diff --git a/README.md b/README.md index 01bbc3ea..89aeb87c 100644 --- a/README.md +++ b/README.md @@ -26,7 +26,7 @@ DataX本身作为数据同步框架,将不同数据源的同步抽象为从源 # Quick Start -##### Download [DataX下载地址](https://datax-opensource.oss-cn-hangzhou.aliyuncs.com/202303/datax.tar.gz) +##### Download [DataX下载地址](https://datax-opensource.oss-cn-hangzhou.aliyuncs.com/202308/datax.tar.gz) ##### 请点击:[Quick Start](https://github.com/alibaba/DataX/blob/master/userGuid.md) @@ -37,47 +37,48 @@ DataX本身作为数据同步框架,将不同数据源的同步抽象为从源 DataX目前已经有了比较全面的插件体系,主流的RDBMS数据库、NOSQL、大数据计算系统都已经接入,目前支持数据如下图,详情请点击:[DataX数据源参考指南](https://github.com/alibaba/DataX/wiki/DataX-all-data-channels) -| 类型 | 数据源 | Reader(读) | Writer(写) | 文档 | +| 类型 | 数据源 | Reader(读) | Writer(写) | 文档 | |--------------|---------------------------|:---------:|:---------:|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------:| -| RDBMS 关系型数据库 | MySQL | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/mysqlreader/doc/mysqlreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/mysqlwriter/doc/mysqlwriter.md) | -| | Oracle | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/oraclereader/doc/oraclereader.md) 、[写](https://github.com/alibaba/DataX/blob/master/oraclewriter/doc/oraclewriter.md) | -| | OceanBase | √ | √ | [读](https://open.oceanbase.com/docs/community/oceanbase-database/V3.1.0/use-datax-to-full-migration-data-to-oceanbase) 、[写](https://open.oceanbase.com/docs/community/oceanbase-database/V3.1.0/use-datax-to-full-migration-data-to-oceanbase) | -| | 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) | -| | Kingbase | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/drdsreader/doc/drdsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/drdswriter/doc/drdswriter.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) | -| | ADB | | √ | [写](https://github.com/alibaba/DataX/blob/master/adbmysqlwriter/doc/adbmysqlwriter.md) | -| | ADS | | √ | [写](https://github.com/alibaba/DataX/blob/master/adswriter/doc/adswriter.md) | -| | OSS | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/ossreader/doc/ossreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/osswriter/doc/osswriter.md) | -| | OCS | | √ | [写](https://github.com/alibaba/DataX/blob/master/ocswriter/doc/ocswriter.md) | -| | Hologres | | √ | [写](https://github.com/alibaba/DataX/blob/master/hologresjdbcwriter/doc/hologresjdbcwriter.md) | -| | AnalyticDB For PostgreSQL | | √ | 写 | -| 阿里云中间件 | datahub | √ | √ | 读 、写 | -| | SLS | √ | √ | 读 、写 | -| 阿里云图数据库 | GDB | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/gdbreader/doc/gdbreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/gdbwriter/doc/gdbwriter.md) | -| NoSQL数据存储 | OTS | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/otsreader/doc/otsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/otswriter/doc/otswriter.md) | -| | Hbase0.94 | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hbase094xreader/doc/hbase094xreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hbase094xwriter/doc/hbase094xwriter.md) | -| | Hbase1.1 | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hbase11xreader/doc/hbase11xreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hbase11xwriter/doc/hbase11xwriter.md) | -| | Phoenix4.x | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hbase11xsqlreader/doc/hbase11xsqlreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hbase11xsqlwriter/doc/hbase11xsqlwriter.md) | -| | Phoenix5.x | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hbase20xsqlreader/doc/hbase20xsqlreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hbase20xsqlwriter/doc/hbase20xsqlwriter.md) | -| | MongoDB | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/mongodbreader/doc/mongodbreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/mongodbwriter/doc/mongodbwriter.md) | -| | Cassandra | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/cassandrareader/doc/cassandrareader.md) 、[写](https://github.com/alibaba/DataX/blob/master/cassandrawriter/doc/cassandrawriter.md) | -| 数仓数据存储 | StarRocks | √ | √ | 读 、[写](https://github.com/alibaba/DataX/blob/master/starrockswriter/doc/starrockswriter.md) | -| | ApacheDoris | | √ | [写](https://github.com/alibaba/DataX/blob/master/doriswriter/doc/doriswriter.md) | -| | ClickHouse | | √ | 写 | -| | Databend | | √ | [写](https://github.com/alibaba/DataX/blob/master/databendwriter/doc/databendwriter.md) | -| | Hive | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hdfsreader/doc/hdfsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hdfswriter/doc/hdfswriter.md) | -| | kudu | | √ | [写](https://github.com/alibaba/DataX/blob/master/hdfswriter/doc/hdfswriter.md) | -| | selectdb | | √ | [写](https://github.com/alibaba/DataX/blob/master/selectdbwriter/doc/selectdbwriter.md) | -| 无结构化数据存储 | TxtFile | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/txtfilereader/doc/txtfilereader.md) 、[写](https://github.com/alibaba/DataX/blob/master/txtfilewriter/doc/txtfilewriter.md) | -| | FTP | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/ftpreader/doc/ftpreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/ftpwriter/doc/ftpwriter.md) | -| | HDFS | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hdfsreader/doc/hdfsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hdfswriter/doc/hdfswriter.md) | -| | Elasticsearch | | √ | [写](https://github.com/alibaba/DataX/blob/master/elasticsearchwriter/doc/elasticsearchwriter.md) | -| 时间序列数据库 | OpenTSDB | √ | | [读](https://github.com/alibaba/DataX/blob/master/opentsdbreader/doc/opentsdbreader.md) | -| | TSDB | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/tsdbreader/doc/tsdbreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/tsdbwriter/doc/tsdbhttpwriter.md) | -| | TDengine | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/tdenginereader/doc/tdenginereader-CN.md) 、[写](https://github.com/alibaba/DataX/blob/master/tdenginewriter/doc/tdenginewriter-CN.md) | +| RDBMS 关系型数据库 | MySQL | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/mysqlreader/doc/mysqlreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/mysqlwriter/doc/mysqlwriter.md) | +| | Oracle | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/oraclereader/doc/oraclereader.md) 、[写](https://github.com/alibaba/DataX/blob/master/oraclewriter/doc/oraclewriter.md) | +| | OceanBase | √ | √ | [读](https://open.oceanbase.com/docs/community/oceanbase-database/V3.1.0/use-datax-to-full-migration-data-to-oceanbase) 、[写](https://open.oceanbase.com/docs/community/oceanbase-database/V3.1.0/use-datax-to-full-migration-data-to-oceanbase) | +| | 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) | +| | Kingbase | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/drdsreader/doc/drdsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/drdswriter/doc/drdswriter.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) | +| | ADB | | √ | [写](https://github.com/alibaba/DataX/blob/master/adbmysqlwriter/doc/adbmysqlwriter.md) | +| | ADS | | √ | [写](https://github.com/alibaba/DataX/blob/master/adswriter/doc/adswriter.md) | +| | OSS | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/ossreader/doc/ossreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/osswriter/doc/osswriter.md) | +| | OCS | | √ | [写](https://github.com/alibaba/DataX/blob/master/ocswriter/doc/ocswriter.md) | +| | Hologres | | √ | [写](https://github.com/alibaba/DataX/blob/master/hologresjdbcwriter/doc/hologresjdbcwriter.md) | +| | AnalyticDB For PostgreSQL | | √ | 写 | +| 阿里云中间件 | datahub | √ | √ | 读 、写 | +| | SLS | √ | √ | 读 、写 | +| 图数据库 | 阿里云 GDB | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/gdbreader/doc/gdbreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/gdbwriter/doc/gdbwriter.md) | +| | Neo4j | | √ | [写](https://github.com/alibaba/DataX/blob/master/neo4jwriter/doc/neo4jwriter.md) | +| NoSQL数据存储 | OTS | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/otsreader/doc/otsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/otswriter/doc/otswriter.md) | +| | Hbase0.94 | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hbase094xreader/doc/hbase094xreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hbase094xwriter/doc/hbase094xwriter.md) | +| | Hbase1.1 | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hbase11xreader/doc/hbase11xreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hbase11xwriter/doc/hbase11xwriter.md) | +| | Phoenix4.x | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hbase11xsqlreader/doc/hbase11xsqlreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hbase11xsqlwriter/doc/hbase11xsqlwriter.md) | +| | Phoenix5.x | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hbase20xsqlreader/doc/hbase20xsqlreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hbase20xsqlwriter/doc/hbase20xsqlwriter.md) | +| | MongoDB | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/mongodbreader/doc/mongodbreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/mongodbwriter/doc/mongodbwriter.md) | +| | Cassandra | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/cassandrareader/doc/cassandrareader.md) 、[写](https://github.com/alibaba/DataX/blob/master/cassandrawriter/doc/cassandrawriter.md) | +| 数仓数据存储 | StarRocks | √ | √ | 读 、[写](https://github.com/alibaba/DataX/blob/master/starrockswriter/doc/starrockswriter.md) | +| | ApacheDoris | | √ | [写](https://github.com/alibaba/DataX/blob/master/doriswriter/doc/doriswriter.md) | +| | ClickHouse | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/clickhousereader/doc/clickhousereader.md) 、[写](https://github.com/alibaba/DataX/blob/master/clickhousewriter/doc/clickhousewriter.md) | +| | Databend | | √ | [写](https://github.com/alibaba/DataX/blob/master/databendwriter/doc/databendwriter.md) | +| | Hive | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hdfsreader/doc/hdfsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hdfswriter/doc/hdfswriter.md) | +| | kudu | | √ | [写](https://github.com/alibaba/DataX/blob/master/hdfswriter/doc/hdfswriter.md) | +| | selectdb | | √ | [写](https://github.com/alibaba/DataX/blob/master/selectdbwriter/doc/selectdbwriter.md) | +| 无结构化数据存储 | TxtFile | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/txtfilereader/doc/txtfilereader.md) 、[写](https://github.com/alibaba/DataX/blob/master/txtfilewriter/doc/txtfilewriter.md) | +| | FTP | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/ftpreader/doc/ftpreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/ftpwriter/doc/ftpwriter.md) | +| | HDFS | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/hdfsreader/doc/hdfsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/hdfswriter/doc/hdfswriter.md) | +| | Elasticsearch | | √ | [写](https://github.com/alibaba/DataX/blob/master/elasticsearchwriter/doc/elasticsearchwriter.md) | +| 时间序列数据库 | OpenTSDB | √ | | [读](https://github.com/alibaba/DataX/blob/master/opentsdbreader/doc/opentsdbreader.md) | +| | TSDB | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/tsdbreader/doc/tsdbreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/tsdbwriter/doc/tsdbhttpwriter.md) | +| | TDengine | √ | √ | [读](https://github.com/alibaba/DataX/blob/master/tdenginereader/doc/tdenginereader-CN.md) 、[写](https://github.com/alibaba/DataX/blob/master/tdenginewriter/doc/tdenginewriter-CN.md) | # 阿里云DataWorks数据集成 @@ -99,7 +100,7 @@ DataX目前已经有了比较全面的插件体系,主流的RDBMS数据库、N - 整库迁移:https://help.aliyun.com/document_detail/137809.html - 批量上云:https://help.aliyun.com/document_detail/146671.html - 更新更多能力请访问:https://help.aliyun.com/document_detail/137663.html - + - # 我要开发新的插件 @@ -109,6 +110,28 @@ DataX目前已经有了比较全面的插件体系,主流的RDBMS数据库、N DataX 后续计划月度迭代更新,也欢迎感兴趣的同学提交 Pull requests,月度更新内容会介绍介绍如下。 +- [datax_v202309](https://github.com/alibaba/DataX/releases/tag/datax_v202309) + - 支持Phoenix 同步数据添加 where条件 + - 支持华为 GuassDB读写插件 + - 修复ClickReader 插件运行报错 Can't find bundle for base name + - 增加 DataX调试模块 + - 修复 orc空文件报错问题 + - 优化obwriter性能 + - txtfilewriter 增加导出为insert语句功能支持 + - HdfsReader/HdfsWriter 支持parquet读写能力 + +- [datax_v202308](https://github.com/alibaba/DataX/releases/tag/datax_v202308) + - OTS 插件更新 + - databend 插件更新 + - Oceanbase驱动修复 + + +- [datax_v202306](https://github.com/alibaba/DataX/releases/tag/datax_v202306) + - 精简代码 + - 新增插件(neo4jwriter、clickhousewriter) + - 优化插件、修复问题(oceanbase、hdfs、databend、txtfile) + + - [datax_v202303](https://github.com/alibaba/DataX/releases/tag/datax_v202303) - 精简代码 - 新增插件(adbmysqlwriter、databendwriter、selectdbwriter) diff --git a/clickhousereader/doc/clickhousereader.md b/clickhousereader/doc/clickhousereader.md new file mode 100644 index 00000000..bf3cd203 --- /dev/null +++ b/clickhousereader/doc/clickhousereader.md @@ -0,0 +1,344 @@ + +# ClickhouseReader 插件文档 + + +___ + + +## 1 快速介绍 + +ClickhouseReader插件实现了从Clickhouse读取数据。在底层实现上,ClickhouseReader通过JDBC连接远程Clickhouse数据库,并执行相应的sql语句将数据从Clickhouse库中SELECT出来。 + +## 2 实现原理 + +简而言之,ClickhouseReader通过JDBC连接器连接到远程的Clickhouse数据库,并根据用户配置的信息生成查询SELECT SQL语句并发送到远程Clickhouse数据库,并将该SQL执行返回结果使用DataX自定义的数据类型拼装为抽象的数据集,并传递给下游Writer处理。 + +对于用户配置Table、Column、Where的信息,ClickhouseReader将其拼接为SQL语句发送到Clickhouse数据库;对于用户配置querySql信息,Clickhouse直接将其发送到Clickhouse数据库。 + + +## 3 功能说明 + +### 3.1 配置样例 + +* 配置一个从Clickhouse数据库同步抽取数据到本地的作业: + +``` +{ + "job": { + "setting": { + "speed": { + //设置传输速度 byte/s 尽量逼近这个速度但是不高于它. + // channel 表示通道数量,byte表示通道速度,如果单通道速度1MB,配置byte为1048576表示一个channel + "byte": 1048576 + }, + //出错限制 + "errorLimit": { + //先选择record + "record": 0, + //百分比 1表示100% + "percentage": 0.02 + } + }, + "content": [ + { + "reader": { + "name": "clickhousereader", + "parameter": { + // 数据库连接用户名 + "username": "root", + // 数据库连接密码 + "password": "root", + "column": [ + "id","name" + ], + "connection": [ + { + "table": [ + "table" + ], + "jdbcUrl": [ + "jdbc:clickhouse://[HOST_NAME]:PORT/[DATABASE_NAME]" + ] + } + ] + } + }, + "writer": { + //writer类型 + "name": "streamwriter", + // 是否打印内容 + "parameter": { + "print": true + } + } + } + ] + } +} + +``` + +* 配置一个自定义SQL的数据库同步任务到本地内容的作业: + +``` +{ + "job": { + "setting": { + "speed": { + "channel": 5 + } + }, + "content": [ + { + "reader": { + "name": "clickhousereader", + "parameter": { + "username": "root", + "password": "root", + "where": "", + "connection": [ + { + "querySql": [ + "select db_id,on_line_flag from db_info where db_id < 10" + ], + "jdbcUrl": [ + "jdbc:clickhouse://1.1.1.1:8123/default" + ] + } + ] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "visible": false, + "encoding": "UTF-8" + } + } + } + ] + } +} +``` + + +### 3.2 参数说明 + +* **jdbcUrl** + + * 描述:描述的是到对端数据库的JDBC连接信息,使用JSON的数组描述,并支持一个库填写多个连接地址。之所以使用JSON数组描述连接信息,是因为阿里集团内部支持多个IP探测,如果配置了多个,ClickhouseReader可以依次探测ip的可连接性,直到选择一个合法的IP。如果全部连接失败,ClickhouseReader报错。 注意,jdbcUrl必须包含在connection配置单元中。对于阿里集团外部使用情况,JSON数组填写一个JDBC连接即可。 + + jdbcUrl按照Clickhouse官方规范,并可以填写连接附件控制信息。具体请参看[Clickhouse官方文档](https://clickhouse.com/docs/en/engines/table-engines/integrations/jdbc)。 + + * 必选:是
+ + * 默认值:无
+ +* **username** + + * 描述:数据源的用户名
+ + * 必选:是
+ + * 默认值:无
+ +* **password** + + * 描述:数据源指定用户名的密码
+ + * 必选:是
+ + * 默认值:无
+ +* **table** + + * 描述:所选取的需要同步的表。使用JSON的数组描述,因此支持多张表同时抽取。当配置为多张表时,用户自己需保证多张表是同一schema结构,ClickhouseReader不予检查表是否同一逻辑表。注意,table必须包含在connection配置单元中。
+ + * 必选:是
+ + * 默认值:无
+ +* **column** + + * 描述:所配置的表中需要同步的列名集合,使用JSON的数组描述字段信息。用户使用\*代表默认使用所有列配置,例如['\*']。 + + 支持列裁剪,即列可以挑选部分列进行导出。 + + 支持列换序,即列可以不按照表schema信息进行导出。 + + 支持常量配置,用户需要按照JSON格式: + ["id", "`table`", "1", "'bazhen.csy'", "null", "to_char(a + 1)", "2.3" , "true"] + id为普通列名,\`table\`为包含保留在的列名,1为整形数字常量,'bazhen.csy'为字符串常量,null为空指针,to_char(a + 1)为表达式,2.3为浮点数,true为布尔值。 + + Column必须显示填写,不允许为空! + + * 必选:是
+ + * 默认值:无
+ +* **splitPk** + + * 描述:ClickhouseReader进行数据抽取时,如果指定splitPk,表示用户希望使用splitPk代表的字段进行数据分片,DataX因此会启动并发任务进行数据同步,这样可以大大提供数据同步的效能。 + + 推荐splitPk用户使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + + 目前splitPk仅支持整形数据切分,`不支持浮点、日期等其他类型`。如果用户指定其他非支持类型,ClickhouseReader将报错! + + splitPk如果不填写,将视作用户不对单表进行切分,ClickhouseReader使用单通道同步全量数据。 + + * 必选:否
+ + * 默认值:无
+ +* **where** + + * 描述:筛选条件,MysqlReader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。
+ + where条件可以有效地进行业务增量同步。 + + * 必选:否
+ + * 默认值:无
+ +* **querySql** + + * 描述:在有些业务场景下,where这一配置项不足以描述所筛选的条件,用户可以通过该配置型来自定义筛选SQL。当用户配置了这一项之后,DataX系统就会忽略table,column这些配置型,直接使用这个配置项的内容对数据进行筛选,例如需要进行多表join后同步数据,使用select a,b from table_a join table_b on table_a.id = table_b.id
+ + `当用户配置querySql时,ClickhouseReader直接忽略table、column、where条件的配置`。 + + * 必选:否
+ + * 默认值:无
+ +* **fetchSize** + + * 描述:该配置项定义了插件和数据库服务器端每次批量数据获取条数,该值决定了DataX和服务器端的网络交互次数,能够较大的提升数据抽取性能。
+ + `注意,该值过大(>2048)可能造成DataX进程OOM。`。 + + * 必选:否
+ + * 默认值:1024
+ +* **session** + + * 描述:控制写入数据的时间格式,时区等的配置,如果表中有时间字段,配置该值以明确告知写入 clickhouse 的时间格式。通常配置的参数为:NLS_DATE_FORMAT,NLS_TIME_FORMAT。其配置的值为 json 格式,例如: +``` +"session": [ + "alter session set NLS_DATE_FORMAT='yyyy-mm-dd hh24:mi:ss'", + "alter session set NLS_TIMESTAMP_FORMAT='yyyy-mm-dd hh24:mi:ss'", + "alter session set NLS_TIMESTAMP_TZ_FORMAT='yyyy-mm-dd hh24:mi:ss'", + "alter session set TIME_ZONE='US/Pacific'" + ] +``` + `(注意"是 " 的转义字符串)`。 + + * 必选:否
+ + * 默认值:无
+ + +### 3.3 类型转换 + +目前ClickhouseReader支持大部分Clickhouse类型,但也存在部分个别类型没有支持的情况,请注意检查你的类型。 + +下面列出ClickhouseReader针对Clickhouse类型转换列表: + + +| DataX 内部类型| Clickhouse 数据类型 | +| -------- |--------------------------------------------------------------------------------------------| +| Long | UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, Int8, Int16, Int32, Int64, Int128, Int256 | +| Double | Float32, Float64, Decimal | +| String | String, FixedString | +| Date | DATE, Date32, DateTime, DateTime64 | +| Boolean | Boolean | +| Bytes | BLOB,BFILE,RAW,LONG RAW | + + + +请注意: + +* `除上述罗列字段类型外,其他类型均不支持`。 + + +## 4 性能报告 + +### 4.1 环境准备 + +#### 4.1.1 数据特征 + +为了模拟线上真实数据,我们设计两个Clickhouse数据表,分别为: + +#### 4.1.2 机器参数 + +* 执行DataX的机器参数为: + +* Clickhouse数据库机器参数为: + +### 4.2 测试报告 + +#### 4.2.1 表1测试报告 + + +| 并发任务数| DataX速度(Rec/s)|DataX流量|网卡流量|DataX运行负载|DB运行负载| +|--------| --------|--------|--------|--------|--------| +|1| DataX 统计速度(Rec/s)|DataX统计流量|网卡流量|DataX运行负载|DB运行负载| + +## 5 约束限制 + +### 5.1 主备同步数据恢复问题 + +主备同步问题指Clickhouse使用主从灾备,备库从主库不间断通过binlog恢复数据。由于主备数据同步存在一定的时间差,特别在于某些特定情况,例如网络延迟等问题,导致备库同步恢复的数据与主库有较大差别,导致从备库同步的数据不是一份当前时间的完整镜像。 + +针对这个问题,我们提供了preSql功能,该功能待补充。 + +### 5.2 一致性约束 + +Clickhouse在数据存储划分中属于RDBMS系统,对外可以提供强一致性数据查询接口。例如当一次同步任务启动运行过程中,当该库存在其他数据写入方写入数据时,ClickhouseReader完全不会获取到写入更新数据,这是由于数据库本身的快照特性决定的。关于数据库快照特性,请参看[MVCC Wikipedia](https://en.wikipedia.org/wiki/Multiversion_concurrency_control) + +上述是在ClickhouseReader单线程模型下数据同步一致性的特性,由于ClickhouseReader可以根据用户配置信息使用了并发数据抽取,因此不能严格保证数据一致性:当ClickhouseReader根据splitPk进行数据切分后,会先后启动多个并发任务完成数据同步。由于多个并发任务相互之间不属于同一个读事务,同时多个并发任务存在时间间隔。因此这份数据并不是`完整的`、`一致的`数据快照信息。 + +针对多线程的一致性快照需求,在技术上目前无法实现,只能从工程角度解决,工程化的方式存在取舍,我们提供几个解决思路给用户,用户可以自行选择: + +1. 使用单线程同步,即不再进行数据切片。缺点是速度比较慢,但是能够很好保证一致性。 + +2. 关闭其他数据写入方,保证当前数据为静态数据,例如,锁表、关闭备库同步等等。缺点是可能影响在线业务。 + +### 5.3 数据库编码问题 + + +ClickhouseReader底层使用JDBC进行数据抽取,JDBC天然适配各类编码,并在底层进行了编码转换。因此ClickhouseReader不需用户指定编码,可以自动获取编码并转码。 + +对于Clickhouse底层写入编码和其设定的编码不一致的混乱情况,ClickhouseReader对此无法识别,对此也无法提供解决方案,对于这类情况,`导出有可能为乱码`。 + +### 5.4 增量数据同步 + +ClickhouseReader使用JDBC SELECT语句完成数据抽取工作,因此可以使用SELECT...WHERE...进行增量数据抽取,方式有多种: + +* 数据库在线应用写入数据库时,填充modify字段为更改时间戳,包括新增、更新、删除(逻辑删)。对于这类应用,ClickhouseReader只需要WHERE条件跟上一同步阶段时间戳即可。 +* 对于新增流水型数据,ClickhouseReader可以WHERE条件后跟上一阶段最大自增ID即可。 + +对于业务上无字段区分新增、修改数据情况,ClickhouseReader也无法进行增量数据同步,只能同步全量数据。 + +### 5.5 Sql安全性 + +ClickhouseReader提供querySql语句交给用户自己实现SELECT抽取语句,ClickhouseReader本身对querySql不做任何安全性校验。这块交由DataX用户方自己保证。 + +## 6 FAQ + +*** + +**Q: ClickhouseReader同步报错,报错信息为XXX** + + A: 网络或者权限问题,请使用Clickhouse命令行测试 + + +如果上述命令也报错,那可以证实是环境问题,请联系你的DBA。 + + +**Q: ClickhouseReader抽取速度很慢怎么办?** + + A: 影响抽取时间的原因大概有如下几个:(来自专业 DBA 卫绾) + 1. 由于SQL的plan异常,导致的抽取时间长; 在抽取时,尽可能使用全表扫描代替索引扫描; + 2. 合理sql的并发度,减少抽取时间; + 3. 抽取sql要简单,尽量不用replace等函数,这个非常消耗cpu,会严重影响抽取速度; diff --git a/clickhousereader/pom.xml b/clickhousereader/pom.xml new file mode 100644 index 00000000..4b095796 --- /dev/null +++ b/clickhousereader/pom.xml @@ -0,0 +1,91 @@ + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + + 4.0.0 + clickhousereader + clickhousereader + jar + + + + ru.yandex.clickhouse + clickhouse-jdbc + 0.2.4 + + + com.alibaba.datax + datax-core + ${datax-project-version} + + + com.alibaba.datax + datax-common + ${datax-project-version} + + + org.slf4j + slf4j-api + + + + ch.qos.logback + logback-classic + + + + com.alibaba.datax + plugin-rdbms-util + ${datax-project-version} + + + + + + + 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/clickhousereader/src/main/assembly/package.xml b/clickhousereader/src/main/assembly/package.xml new file mode 100644 index 00000000..9dc7fc13 --- /dev/null +++ b/clickhousereader/src/main/assembly/package.xml @@ -0,0 +1,35 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + plugin_job_template.json + + plugin/reader/clickhousereader + + + target/ + + clickhousereader-0.0.1-SNAPSHOT.jar + + plugin/reader/clickhousereader + + + + + + false + plugin/reader/clickhousereader/libs + runtime + + + \ No newline at end of file diff --git a/clickhousereader/src/main/java/com/alibaba/datax/plugin/reader/clickhousereader/ClickhouseReader.java b/clickhousereader/src/main/java/com/alibaba/datax/plugin/reader/clickhousereader/ClickhouseReader.java new file mode 100644 index 00000000..cfa6be99 --- /dev/null +++ b/clickhousereader/src/main/java/com/alibaba/datax/plugin/reader/clickhousereader/ClickhouseReader.java @@ -0,0 +1,85 @@ +package com.alibaba.datax.plugin.reader.clickhousereader; + +import java.sql.Array; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Types; +import java.util.List; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.element.StringColumn; +import com.alibaba.datax.common.plugin.RecordSender; +import com.alibaba.datax.common.plugin.TaskPluginCollector; +import com.alibaba.datax.common.spi.Reader; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.common.util.MessageSource; +import com.alibaba.datax.plugin.rdbms.reader.CommonRdbmsReader; +import com.alibaba.datax.plugin.rdbms.util.DataBaseType; +import com.alibaba.fastjson2.JSON; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ClickhouseReader extends Reader { + + private static final DataBaseType DATABASE_TYPE = DataBaseType.ClickHouse; + private static final Logger LOG = LoggerFactory.getLogger(ClickhouseReader.class); + + public static class Job extends Reader.Job { + private Configuration jobConfig = null; + private CommonRdbmsReader.Job commonRdbmsReaderMaster; + + @Override + public void init() { + this.jobConfig = super.getPluginJobConf(); + this.commonRdbmsReaderMaster = new CommonRdbmsReader.Job(DATABASE_TYPE); + this.commonRdbmsReaderMaster.init(this.jobConfig); + } + + @Override + public List split(int mandatoryNumber) { + return this.commonRdbmsReaderMaster.split(this.jobConfig, mandatoryNumber); + } + + @Override + public void post() { + this.commonRdbmsReaderMaster.post(this.jobConfig); + } + + @Override + public void destroy() { + this.commonRdbmsReaderMaster.destroy(this.jobConfig); + } + } + + public static class Task extends Reader.Task { + + private Configuration jobConfig; + private CommonRdbmsReader.Task commonRdbmsReaderSlave; + + @Override + public void init() { + this.jobConfig = super.getPluginJobConf(); + this.commonRdbmsReaderSlave = new CommonRdbmsReader.Task(DATABASE_TYPE, super.getTaskGroupId(), super.getTaskId()); + this.commonRdbmsReaderSlave.init(this.jobConfig); + } + + @Override + public void startRead(RecordSender recordSender) { + int fetchSize = this.jobConfig.getInt(com.alibaba.datax.plugin.rdbms.reader.Constant.FETCH_SIZE, 1000); + + this.commonRdbmsReaderSlave.startRead(this.jobConfig, recordSender, super.getTaskPluginCollector(), fetchSize); + } + + @Override + public void post() { + this.commonRdbmsReaderSlave.post(this.jobConfig); + } + + @Override + public void destroy() { + this.commonRdbmsReaderSlave.destroy(this.jobConfig); + } + } +} diff --git a/clickhousereader/src/main/resources/plugin.json b/clickhousereader/src/main/resources/plugin.json new file mode 100644 index 00000000..5d608f6c --- /dev/null +++ b/clickhousereader/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "clickhousereader", + "class": "com.alibaba.datax.plugin.reader.clickhousereader.ClickhouseReader", + "description": "useScene: prod. mechanism: Jdbc connection using the database, execute select sql.", + "developer": "alibaba" +} \ No newline at end of file diff --git a/clickhousereader/src/main/resources/plugin_job_template.json b/clickhousereader/src/main/resources/plugin_job_template.json new file mode 100644 index 00000000..1814e510 --- /dev/null +++ b/clickhousereader/src/main/resources/plugin_job_template.json @@ -0,0 +1,16 @@ +{ + "name": "clickhousereader", + "parameter": { + "username": "username", + "password": "password", + "column": ["col1", "col2", "col3"], + "connection": [ + { + "jdbcUrl": "jdbc:clickhouse://:[/]", + "table": ["table1", "table2"] + } + ], + "preSql": [], + "postSql": [] + } +} \ No newline at end of file diff --git a/clickhousereader/src/test/resources/basic1.json b/clickhousereader/src/test/resources/basic1.json new file mode 100755 index 00000000..c45a45e7 --- /dev/null +++ b/clickhousereader/src/test/resources/basic1.json @@ -0,0 +1,57 @@ +{ + "job": { + "setting": { + "speed": { + "channel": 5 + } + }, + "content": [ + { + "reader": { + "name": "clickhousereader", + "parameter": { + "username": "XXXX", + "password": "XXXX", + "column": [ + "uint8_col", + "uint16_col", + "uint32_col", + "uint64_col", + "int8_col", + "int16_col", + "int32_col", + "int64_col", + "float32_col", + "float64_col", + "bool_col", + "str_col", + "fixedstr_col", + "uuid_col", + "date_col", + "datetime_col", + "enum_col", + "ary_uint8_col", + "ary_str_col", + "tuple_col", + "nullable_col", + "nested_col.nested_id", + "nested_col.nested_str", + "ipv4_col", + "ipv6_col", + "decimal_col" + ], + "connection": [ + { + "table": [ + "all_type_tbl" + ], + "jdbcUrl":["jdbc:clickhouse://XXXX:8123/default"] + } + ] + } + }, + "writer": {} + } + ] + } +} \ No newline at end of file diff --git a/clickhousereader/src/test/resources/basic1.sql b/clickhousereader/src/test/resources/basic1.sql new file mode 100644 index 00000000..f937b889 --- /dev/null +++ b/clickhousereader/src/test/resources/basic1.sql @@ -0,0 +1,34 @@ +CREATE TABLE IF NOT EXISTS default.all_type_tbl +( +`uint8_col` UInt8, +`uint16_col` UInt16, +uint32_col UInt32, +uint64_col UInt64, +int8_col Int8, +int16_col Int16, +int32_col Int32, +int64_col Int64, +float32_col Float32, +float64_col Float64, +bool_col UInt8, +str_col String, +fixedstr_col FixedString(3), +uuid_col UUID, +date_col Date, +datetime_col DateTime, +enum_col Enum('hello' = 1, 'world' = 2), +ary_uint8_col Array(UInt8), +ary_str_col Array(String), +tuple_col Tuple(UInt8, String), +nullable_col Nullable(UInt8), +nested_col Nested + ( + nested_id UInt32, + nested_str String + ), +ipv4_col IPv4, +ipv6_col IPv6, +decimal_col Decimal(5,3) +) +ENGINE = MergeTree() +ORDER BY (uint8_col); \ No newline at end of file diff --git a/common/src/main/java/com/alibaba/datax/common/element/DateColumn.java b/common/src/main/java/com/alibaba/datax/common/element/DateColumn.java index f688d163..df5e1e4a 100755 --- a/common/src/main/java/com/alibaba/datax/common/element/DateColumn.java +++ b/common/src/main/java/com/alibaba/datax/common/element/DateColumn.java @@ -5,6 +5,7 @@ import com.alibaba.datax.common.exception.DataXException; import java.math.BigDecimal; import java.math.BigInteger; +import java.sql.Time; import java.util.Date; /** @@ -12,18 +13,54 @@ import java.util.Date; */ public class DateColumn extends Column { - private DateType subType = DateType.DATETIME; + private DateType subType = DateType.DATETIME; - public static enum DateType { - DATE, TIME, DATETIME - } + private int nanos = 0; - /** - * 构建值为null的DateColumn,使用Date子类型为DATETIME - * */ - public DateColumn() { - this((Long)null); - } + private int precision = -1; + + public static enum DateType { + DATE, TIME, DATETIME + } + + /** + * 构建值为time(java.sql.Time)的DateColumn,使用Date子类型为TIME,只有时间,没有日期 + */ + public DateColumn(Time time, int nanos, int jdbcPrecision) { + this(time); + if (time != null) { + setNanos(nanos); + } + if (jdbcPrecision == 10) { + setPrecision(0); + } + if (jdbcPrecision >= 12 && jdbcPrecision <= 17) { + setPrecision(jdbcPrecision - 11); + } + } + + public long getNanos() { + return nanos; + } + + public void setNanos(int nanos) { + this.nanos = nanos; + } + + public int getPrecision() { + return precision; + } + + public void setPrecision(int precision) { + this.precision = precision; + } + + /** + * 构建值为null的DateColumn,使用Date子类型为DATETIME + */ + public DateColumn() { + this((Long) null); + } /** * 构建值为stamp(Unix时间戳)的DateColumn,使用Date子类型为DATETIME diff --git a/common/src/main/java/com/alibaba/datax/common/statistics/VMInfo.java b/common/src/main/java/com/alibaba/datax/common/statistics/VMInfo.java index cab42a4b..423c794e 100644 --- a/common/src/main/java/com/alibaba/datax/common/statistics/VMInfo.java +++ b/common/src/main/java/com/alibaba/datax/common/statistics/VMInfo.java @@ -77,8 +77,8 @@ public class VMInfo { garbageCollectorMXBeanList = java.lang.management.ManagementFactory.getGarbageCollectorMXBeans(); memoryPoolMXBeanList = java.lang.management.ManagementFactory.getMemoryPoolMXBeans(); - osInfo = runtimeMXBean.getVmVendor() + " " + runtimeMXBean.getSpecVersion() + " " + runtimeMXBean.getVmVersion(); - jvmInfo = osMXBean.getName() + " " + osMXBean.getArch() + " " + osMXBean.getVersion(); + jvmInfo = runtimeMXBean.getVmVendor() + " " + runtimeMXBean.getSpecVersion() + " " + runtimeMXBean.getVmVersion(); + osInfo = osMXBean.getName() + " " + osMXBean.getArch() + " " + osMXBean.getVersion(); totalProcessorCount = osMXBean.getAvailableProcessors(); //构建startPhyOSStatus diff --git a/common/src/main/java/com/alibaba/datax/common/util/LimitLogger.java b/common/src/main/java/com/alibaba/datax/common/util/LimitLogger.java new file mode 100644 index 00000000..a307e0fb --- /dev/null +++ b/common/src/main/java/com/alibaba/datax/common/util/LimitLogger.java @@ -0,0 +1,34 @@ +package com.alibaba.datax.common.util; + +import org.apache.commons.lang3.StringUtils; + +import java.util.HashMap; +import java.util.Map; + +/** + * @author jitongchen + * @date 2023/9/7 9:47 AM + */ +public class LimitLogger { + + private static Map lastPrintTime = new HashMap<>(); + + public static void limit(String name, long limit, LoggerFunction function) { + if (StringUtils.isBlank(name)) { + name = "__all__"; + } + if (limit <= 0) { + function.apply(); + } else { + if (!lastPrintTime.containsKey(name)) { + lastPrintTime.put(name, System.currentTimeMillis()); + function.apply(); + } else { + if (System.currentTimeMillis() > lastPrintTime.get(name) + limit) { + lastPrintTime.put(name, System.currentTimeMillis()); + function.apply(); + } + } + } + } +} diff --git a/common/src/main/java/com/alibaba/datax/common/util/LoggerFunction.java b/common/src/main/java/com/alibaba/datax/common/util/LoggerFunction.java new file mode 100644 index 00000000..ef24504f --- /dev/null +++ b/common/src/main/java/com/alibaba/datax/common/util/LoggerFunction.java @@ -0,0 +1,10 @@ +package com.alibaba.datax.common.util; + +/** + * @author molin.lxd + * @date 2021-05-09 + */ +public interface LoggerFunction { + + void apply(); +} diff --git a/core/src/main/java/com/alibaba/datax/core/transport/channel/memory/MemoryChannel.java b/core/src/main/java/com/alibaba/datax/core/transport/channel/memory/MemoryChannel.java index e49c7878..5bce085f 100755 --- a/core/src/main/java/com/alibaba/datax/core/transport/channel/memory/MemoryChannel.java +++ b/core/src/main/java/com/alibaba/datax/core/transport/channel/memory/MemoryChannel.java @@ -29,7 +29,7 @@ public class MemoryChannel extends Channel { private ReentrantLock lock; - private Condition notInsufficient, notEmpty; + private Condition notSufficient, notEmpty; public MemoryChannel(final Configuration configuration) { super(configuration); @@ -37,7 +37,7 @@ public class MemoryChannel extends Channel { this.bufferSize = configuration.getInt(CoreConstant.DATAX_CORE_TRANSPORT_EXCHANGER_BUFFERSIZE); lock = new ReentrantLock(); - notInsufficient = lock.newCondition(); + notSufficient = lock.newCondition(); notEmpty = lock.newCondition(); } @@ -75,7 +75,7 @@ public class MemoryChannel extends Channel { lock.lockInterruptibly(); int bytes = getRecordBytes(rs); while (memoryBytes.get() + bytes > this.byteCapacity || rs.size() > this.queue.remainingCapacity()) { - notInsufficient.await(200L, TimeUnit.MILLISECONDS); + notSufficient.await(200L, TimeUnit.MILLISECONDS); } this.queue.addAll(rs); waitWriterTime += System.nanoTime() - startTime; @@ -116,7 +116,7 @@ public class MemoryChannel extends Channel { waitReaderTime += System.nanoTime() - startTime; int bytes = getRecordBytes(rs); memoryBytes.addAndGet(-bytes); - notInsufficient.signalAll(); + notSufficient.signalAll(); } catch (InterruptedException e) { throw DataXException.asDataXException( FrameworkErrorCode.RUNTIME_ERROR, e); diff --git a/core/src/main/java/com/alibaba/datax/core/util/ConfigParser.java b/core/src/main/java/com/alibaba/datax/core/util/ConfigParser.java index 20039864..24f43d55 100755 --- a/core/src/main/java/com/alibaba/datax/core/util/ConfigParser.java +++ b/core/src/main/java/com/alibaba/datax/core/util/ConfigParser.java @@ -168,6 +168,7 @@ public final class ConfigParser { boolean isDefaultPath = StringUtils.isBlank(pluginPath); if (isDefaultPath) { configuration.set("path", path); + configuration.set("loadType","jarLoader"); } Configuration result = Configuration.newDefault(); diff --git a/core/src/main/java/com/alibaba/datax/core/util/container/JarLoader.java b/core/src/main/java/com/alibaba/datax/core/util/container/JarLoader.java index 9fc113dc..ddf22bae 100755 --- a/core/src/main/java/com/alibaba/datax/core/util/container/JarLoader.java +++ b/core/src/main/java/com/alibaba/datax/core/util/container/JarLoader.java @@ -15,7 +15,7 @@ import java.util.List; /** * 提供Jar隔离的加载机制,会把传入的路径、及其子路径、以及路径中的jar文件加入到class path。 */ -public class JarLoader extends URLClassLoader { +public class JarLoader extends URLClassLoader{ public JarLoader(String[] paths) { this(paths, JarLoader.class.getClassLoader()); } diff --git a/core/src/main/java/com/alibaba/datax/core/util/container/LoadUtil.java b/core/src/main/java/com/alibaba/datax/core/util/container/LoadUtil.java index 30e926c3..9a6a8302 100755 --- a/core/src/main/java/com/alibaba/datax/core/util/container/LoadUtil.java +++ b/core/src/main/java/com/alibaba/datax/core/util/container/LoadUtil.java @@ -49,7 +49,7 @@ public class LoadUtil { /** * jarLoader的缓冲 */ - private static Map jarLoaderCenter = new HashMap(); + private static Map jarLoaderCenter = new HashMap(); /** * 设置pluginConfigs,方便后面插件来获取 diff --git a/databendwriter/doc/databendwriter-CN.md b/databendwriter/doc/databendwriter-CN.md index d6a8f1f3..5b26ed7e 100644 --- a/databendwriter/doc/databendwriter-CN.md +++ b/databendwriter/doc/databendwriter-CN.md @@ -79,6 +79,8 @@ create table if not exsits datax.sample1(a string, b int64, c date, d timestamp, "writer": { "name": "databendwriter", "parameter": { + "writeMode": "replace", + "onConflictColumn": ["id"], "username": "databend", "password": "databend", "column": ["a", "b", "c", "d", "e", "f", "g"], @@ -149,6 +151,16 @@ create table if not exsits datax.sample1(a string, b int64, c date, d timestamp, * 必选: 否 * 默认值: 无 * 示例: ["select count(*) from datax.sample1"] +* writeMode + * 描述:写入模式,支持 insert 和 replace 两种模式,默认为 insert。若为 replace,务必填写 onConflictColumn 参数 + * 必选:否 + * 默认值:insert + * 示例:"replace" +* onConflictColumn + * 描述:on conflict 字段,指定 writeMode 为 replace 后,需要此参数 + * 必选:否 + * 默认值:无 + * 示例:["id","user"] ### 3.3 类型转化 DataX中的数据类型可以转换为databend中的相应数据类型。下表显示了两种类型之间的对应关系。 diff --git a/databendwriter/doc/databendwriter.md b/databendwriter/doc/databendwriter.md index 0b57bf13..c92d6387 100644 --- a/databendwriter/doc/databendwriter.md +++ b/databendwriter/doc/databendwriter.md @@ -142,6 +142,16 @@ create table if not exsits datax.sample1(a string, b int64, c date, d timestamp, * Description: A list of SQL statements that will be executed after the write operation. * Required: no * Default: none +* writeMode + * Description:The write mode, support `insert` and `replace` two mode. + * Required:no + * Default:insert + * Example:"replace" +* onConflictColumn + * Description:On conflict fields list. + * Required:no + * Default:none + * Example:["id","user"] ### 3.3 Type Convert Data types in datax can be converted to the corresponding data types in databend. The following table shows the correspondence between the two types. diff --git a/databendwriter/pom.xml b/databendwriter/pom.xml index 9ddc735c..b99ca5d8 100644 --- a/databendwriter/pom.xml +++ b/databendwriter/pom.xml @@ -17,7 +17,7 @@ com.databend databend-jdbc - 0.0.7 + 0.1.0 com.alibaba.datax diff --git a/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/DatabendWriter.java b/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/DatabendWriter.java index a4222f08..ddb8fc9a 100644 --- a/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/DatabendWriter.java +++ b/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/DatabendWriter.java @@ -17,20 +17,17 @@ import java.sql.*; import java.util.List; import java.util.regex.Pattern; -public class DatabendWriter extends Writer -{ +public class DatabendWriter extends Writer { private static final DataBaseType DATABASE_TYPE = DataBaseType.Databend; public static class Job - extends Writer.Job - { + extends Writer.Job { private static final Logger LOG = LoggerFactory.getLogger(Job.class); private Configuration originalConfig; private CommonRdbmsWriter.Job commonRdbmsWriterMaster; @Override - public void init() - { + public void init() throws DataXException { this.originalConfig = super.getPluginJobConf(); this.commonRdbmsWriterMaster = new CommonRdbmsWriter.Job(DATABASE_TYPE); this.commonRdbmsWriterMaster.init(this.originalConfig); @@ -39,8 +36,7 @@ public class DatabendWriter extends Writer } @Override - public void preCheck() - { + public void preCheck() { this.init(); this.commonRdbmsWriterMaster.writerPreCheck(this.originalConfig, DATABASE_TYPE); } @@ -67,8 +63,7 @@ public class DatabendWriter extends Writer } - public static class Task extends Writer.Task - { + public static class Task extends Writer.Task { private static final Logger LOG = LoggerFactory.getLogger(Task.class); private Configuration writerSliceConfig; @@ -76,11 +71,10 @@ public class DatabendWriter extends Writer private CommonRdbmsWriter.Task commonRdbmsWriterSlave; @Override - public void init() - { + public void init() { this.writerSliceConfig = super.getPluginJobConf(); - this.commonRdbmsWriterSlave = new CommonRdbmsWriter.Task(DataBaseType.Databend){ + this.commonRdbmsWriterSlave = new CommonRdbmsWriter.Task(DataBaseType.Databend) { @Override protected PreparedStatement fillPreparedStatementColumnType(PreparedStatement preparedStatement, int columnIndex, int columnSqltype, String typeName, Column column) throws SQLException { try { @@ -177,8 +171,8 @@ public class DatabendWriter extends Writer case Types.BOOLEAN: - // warn: bit(1) -> Types.BIT 可使用setBoolean - // warn: bit(>1) -> Types.VARBINARY 可使用setBytes + // warn: bit(1) -> Types.BIT 可使用setBoolean + // warn: bit(>1) -> Types.VARBINARY 可使用setBytes case Types.BIT: if (this.dataBaseType == DataBaseType.MySql) { Boolean asBoolean = column.asBoolean(); @@ -224,8 +218,7 @@ public class DatabendWriter extends Writer } @Override - public void destroy() - { + public void destroy() { this.commonRdbmsWriterSlave.destroy(this.writerSliceConfig); } @@ -238,9 +231,9 @@ public class DatabendWriter extends Writer public void post() { this.commonRdbmsWriterSlave.post(this.writerSliceConfig); } + @Override - public void startWrite(RecordReceiver lineReceiver) - { + public void startWrite(RecordReceiver lineReceiver) { this.commonRdbmsWriterSlave.startWrite(lineReceiver, this.writerSliceConfig, this.getTaskPluginCollector()); } diff --git a/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/DatabendWriterErrorCode.java b/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/DatabendWriterErrorCode.java new file mode 100644 index 00000000..21cbf428 --- /dev/null +++ b/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/DatabendWriterErrorCode.java @@ -0,0 +1,33 @@ +package com.alibaba.datax.plugin.writer.databendwriter; + +import com.alibaba.datax.common.spi.ErrorCode; + + +public enum DatabendWriterErrorCode implements ErrorCode { + CONF_ERROR("DatabendWriter-00", "配置错误."), + WRITE_DATA_ERROR("DatabendWriter-01", "写入数据时失败."), + ; + + private final String code; + private final String description; + + private DatabendWriterErrorCode(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/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/util/DatabendWriterUtil.java b/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/util/DatabendWriterUtil.java index a862e920..516a75eb 100644 --- a/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/util/DatabendWriterUtil.java +++ b/databendwriter/src/main/java/com/alibaba/datax/plugin/writer/databendwriter/util/DatabendWriterUtil.java @@ -1,40 +1,72 @@ package com.alibaba.datax.plugin.writer.databendwriter.util; + +import com.alibaba.datax.common.exception.DataXException; import com.alibaba.datax.common.util.Configuration; import com.alibaba.datax.plugin.rdbms.writer.Constant; import com.alibaba.datax.plugin.rdbms.writer.Key; +import com.alibaba.datax.plugin.writer.databendwriter.DatabendWriterErrorCode; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.xml.crypto.Data; import java.util.List; import java.util.StringJoiner; -public final class DatabendWriterUtil -{ +public final class DatabendWriterUtil { private static final Logger LOG = LoggerFactory.getLogger(DatabendWriterUtil.class); - private DatabendWriterUtil() {} - public static void dealWriteMode(Configuration originalConfig) - { + private DatabendWriterUtil() { + } + + public static void dealWriteMode(Configuration originalConfig) throws DataXException { List columns = originalConfig.getList(Key.COLUMN, String.class); + List onConflictColumns = originalConfig.getList(Key.ONCONFLICT_COLUMN, String.class); + StringBuilder writeDataSqlTemplate = new StringBuilder(); String jdbcUrl = originalConfig.getString(String.format("%s[0].%s", Constant.CONN_MARK, Key.JDBC_URL, String.class)); String writeMode = originalConfig.getString(Key.WRITE_MODE, "INSERT"); + LOG.info("write mode is {}", writeMode); + if (writeMode.toLowerCase().contains("replace")) { + if (onConflictColumns == null || onConflictColumns.size() == 0) { + throw DataXException + .asDataXException( + DatabendWriterErrorCode.CONF_ERROR, + String.format( + "Replace mode must has onConflictColumn config." + )); + } - StringBuilder writeDataSqlTemplate = new StringBuilder(); - writeDataSqlTemplate.append("INSERT INTO %s"); - StringJoiner columnString = new StringJoiner(","); + // for databend if you want to use replace mode, the writeMode should be: "writeMode": "replace" + writeDataSqlTemplate.append("REPLACE INTO %s (") + .append(StringUtils.join(columns, ",")).append(") ").append(onConFlictDoString(onConflictColumns)) + .append(" VALUES"); - for (String column : columns) { - columnString.add(column); + LOG.info("Replace data [\n{}\n], which jdbcUrl like:[{}]", writeDataSqlTemplate, jdbcUrl); + originalConfig.set(Constant.INSERT_OR_REPLACE_TEMPLATE_MARK, writeDataSqlTemplate); + } else { + writeDataSqlTemplate.append("INSERT INTO %s"); + StringJoiner columnString = new StringJoiner(","); + + for (String column : columns) { + columnString.add(column); + } + writeDataSqlTemplate.append(String.format("(%s)", columnString)); + writeDataSqlTemplate.append(" VALUES"); + + LOG.info("Insert data [\n{}\n], which jdbcUrl like:[{}]", writeDataSqlTemplate, jdbcUrl); + + originalConfig.set(Constant.INSERT_OR_REPLACE_TEMPLATE_MARK, writeDataSqlTemplate); } - writeDataSqlTemplate.append(String.format("(%s)", columnString)); - writeDataSqlTemplate.append(" VALUES"); - LOG.info("Write data [\n{}\n], which jdbcUrl like:[{}]", writeDataSqlTemplate, jdbcUrl); - - originalConfig.set(Constant.INSERT_OR_REPLACE_TEMPLATE_MARK, writeDataSqlTemplate); } -} \ No newline at end of file + + public static String onConFlictDoString(List conflictColumns) { + return " ON " + + "(" + + StringUtils.join(conflictColumns, ",") + ") "; + } +} diff --git a/datax-example/datax-example-core/pom.xml b/datax-example/datax-example-core/pom.xml new file mode 100644 index 00000000..6a2e9e8e --- /dev/null +++ b/datax-example/datax-example-core/pom.xml @@ -0,0 +1,20 @@ + + + 4.0.0 + + com.alibaba.datax + datax-example + 0.0.1-SNAPSHOT + + + datax-example-core + + + 8 + 8 + UTF-8 + + + \ No newline at end of file diff --git a/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/ExampleContainer.java b/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/ExampleContainer.java new file mode 100644 index 00000000..a4229fd1 --- /dev/null +++ b/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/ExampleContainer.java @@ -0,0 +1,26 @@ +package com.alibaba.datax.example; + +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.core.Engine; +import com.alibaba.datax.example.util.ExampleConfigParser; + +/** + * {@code Date} 2023/8/6 11:22 + * + * @author fuyouj + */ + +public class ExampleContainer { + /** + * example对外暴露的启动入口 + * 使用前最好看下 datax-example/doc/README.MD + * @param jobPath 任务json绝对路径 + */ + public static void start(String jobPath) { + + Configuration configuration = ExampleConfigParser.parse(jobPath); + + Engine engine = new Engine(); + engine.start(configuration); + } +} diff --git a/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/Main.java b/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/Main.java new file mode 100644 index 00000000..56bf9f0b --- /dev/null +++ b/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/Main.java @@ -0,0 +1,23 @@ +package com.alibaba.datax.example; + + +import com.alibaba.datax.example.util.PathUtil; + +/** + * @author fuyouj + */ +public class Main { + + /** + * 1.在example模块pom文件添加你依赖的的调试插件, + * 你可以直接打开本模块的pom文件,参考是如何引入streamreader,streamwriter + * 2. 在此处指定你的job文件 + */ + public static void main(String[] args) { + + String classPathJobPath = "/job/stream2stream.json"; + String absJobPath = PathUtil.getAbsolutePathFromClassPath(classPathJobPath); + ExampleContainer.start(absJobPath); + } + +} diff --git a/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/util/ExampleConfigParser.java b/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/util/ExampleConfigParser.java new file mode 100644 index 00000000..6bbb4a23 --- /dev/null +++ b/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/util/ExampleConfigParser.java @@ -0,0 +1,154 @@ +package com.alibaba.datax.example.util; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.core.util.ConfigParser; +import com.alibaba.datax.core.util.FrameworkErrorCode; +import com.alibaba.datax.core.util.container.CoreConstant; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Paths; +import java.util.*; + +/** + * @author fuyouj + */ +public class ExampleConfigParser { + private static final String CORE_CONF = "/example/conf/core.json"; + + private static final String PLUGIN_DESC_FILE = "plugin.json"; + + /** + * 指定Job配置路径,ConfigParser会解析Job、Plugin、Core全部信息,并以Configuration返回 + * 不同于Core的ConfigParser,这里的core,plugin 不依赖于编译后的datax.home,而是扫描程序编译后的target目录 + */ + public static Configuration parse(final String jobPath) { + + Configuration configuration = ConfigParser.parseJobConfig(jobPath); + configuration.merge(coreConfig(), + false); + + Map pluginTypeMap = new HashMap<>(); + String readerName = configuration.getString(CoreConstant.DATAX_JOB_CONTENT_READER_NAME); + String writerName = configuration.getString(CoreConstant.DATAX_JOB_CONTENT_WRITER_NAME); + pluginTypeMap.put(readerName, "reader"); + pluginTypeMap.put(writerName, "writer"); + Configuration pluginsDescConfig = parsePluginsConfig(pluginTypeMap); + configuration.merge(pluginsDescConfig, false); + return configuration; + } + + private static Configuration parsePluginsConfig(Map pluginTypeMap) { + + Configuration configuration = Configuration.newDefault(); + + //最初打算通过user.dir获取工作目录来扫描插件, + //但是user.dir在不同有一些不确定性,所以废弃了这个选择 + + for (File basePackage : runtimeBasePackages()) { + if (pluginTypeMap.isEmpty()) { + break; + } + scanPluginByPackage(basePackage, configuration, basePackage.listFiles(), pluginTypeMap); + } + if (!pluginTypeMap.isEmpty()) { + String failedPlugin = pluginTypeMap.keySet().toString(); + String message = "\nplugin %s load failed :ry to analyze the reasons from the following aspects.。\n" + + "1: Check if the name of the plugin is spelled correctly, and verify whether DataX supports this plugin\n" + + "2:Verify if the tag has been added under section in the pom file of the relevant plugin.\n" + + " src/main/resources\n" + + " \n" + + " **/*.*\n" + + " \n" + + " true\n" + + " \n [Refer to the streamreader pom file] \n" + + "3: Check that the datax-yourPlugin-example module imported your test plugin"; + message = String.format(message, failedPlugin); + throw DataXException.asDataXException(FrameworkErrorCode.PLUGIN_INIT_ERROR, message); + } + return configuration; + } + + /** + * 通过classLoader获取程序编译的输出目录 + * + * @return File[/datax-example/target/classes,xxReader/target/classes,xxWriter/target/classes] + */ + private static File[] runtimeBasePackages() { + List basePackages = new ArrayList<>(); + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + Enumeration resources = null; + try { + resources = classLoader.getResources(""); + } catch (IOException e) { + throw DataXException.asDataXException(e.getMessage()); + } + + while (resources.hasMoreElements()) { + URL resource = resources.nextElement(); + File file = new File(resource.getFile()); + if (file.isDirectory()) { + basePackages.add(file); + } + } + + return basePackages.toArray(new File[0]); + } + + /** + * @param packageFile 编译出来的target/classes根目录 便于找到插件时设置插件的URL目录,设置根目录是最保险的方式 + * @param configuration pluginConfig + * @param files 待扫描文件 + * @param needPluginTypeMap 需要的插件 + */ + private static void scanPluginByPackage(File packageFile, + Configuration configuration, + File[] files, + Map needPluginTypeMap) { + if (files == null) { + return; + } + for (File file : files) { + if (file.isFile() && PLUGIN_DESC_FILE.equals(file.getName())) { + Configuration pluginDesc = Configuration.from(file); + String descPluginName = pluginDesc.getString("name", ""); + + if (needPluginTypeMap.containsKey(descPluginName)) { + + String type = needPluginTypeMap.get(descPluginName); + configuration.merge(parseOnePlugin(packageFile.getAbsolutePath(), type, descPluginName, pluginDesc), false); + needPluginTypeMap.remove(descPluginName); + + } + } else { + scanPluginByPackage(packageFile, configuration, file.listFiles(), needPluginTypeMap); + } + } + } + + + private static Configuration parseOnePlugin(String packagePath, + String pluginType, + String pluginName, + Configuration pluginDesc) { + //设置path 兼容jarLoader的加载方式URLClassLoader + pluginDesc.set("path", packagePath); + Configuration pluginConfInJob = Configuration.newDefault(); + pluginConfInJob.set( + String.format("plugin.%s.%s", pluginType, pluginName), + pluginDesc.getInternal()); + return pluginConfInJob; + } + + private static Configuration coreConfig() { + try { + URL resource = ExampleConfigParser.class.getResource(CORE_CONF); + return Configuration.from(Paths.get(resource.toURI()).toFile()); + } catch (Exception ignore) { + throw DataXException.asDataXException("Failed to load the configuration file core.json. " + + "Please check whether /example/conf/core.json exists!"); + } + } +} diff --git a/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/util/PathUtil.java b/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/util/PathUtil.java new file mode 100644 index 00000000..e197fa73 --- /dev/null +++ b/datax-example/datax-example-core/src/main/java/com/alibaba/datax/example/util/PathUtil.java @@ -0,0 +1,26 @@ +package com.alibaba.datax.example.util; + + +import com.alibaba.datax.common.exception.DataXException; + +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Paths; + +/** + * @author fuyouj + */ +public class PathUtil { + public static String getAbsolutePathFromClassPath(String path) { + URL resource = PathUtil.class.getResource(path); + try { + assert resource != null; + URI uri = resource.toURI(); + return Paths.get(uri).toString(); + } catch (NullPointerException | URISyntaxException e) { + throw DataXException.asDataXException("path error,please check whether the path is correct"); + } + + } +} diff --git a/datax-example/datax-example-core/src/main/resources/example/conf/core.json b/datax-example/datax-example-core/src/main/resources/example/conf/core.json new file mode 100755 index 00000000..33281ac0 --- /dev/null +++ b/datax-example/datax-example-core/src/main/resources/example/conf/core.json @@ -0,0 +1,60 @@ +{ + "entry": { + "jvm": "-Xms1G -Xmx1G", + "environment": {} + }, + "common": { + "column": { + "datetimeFormat": "yyyy-MM-dd HH:mm:ss", + "timeFormat": "HH:mm:ss", + "dateFormat": "yyyy-MM-dd", + "extraFormats":["yyyyMMdd"], + "timeZone": "GMT+8", + "encoding": "utf-8" + } + }, + "core": { + "dataXServer": { + "address": "http://localhost:7001/api", + "timeout": 10000, + "reportDataxLog": false, + "reportPerfLog": false + }, + "transport": { + "channel": { + "class": "com.alibaba.datax.core.transport.channel.memory.MemoryChannel", + "speed": { + "byte": -1, + "record": -1 + }, + "flowControlInterval": 20, + "capacity": 512, + "byteCapacity": 67108864 + }, + "exchanger": { + "class": "com.alibaba.datax.core.plugin.BufferedRecordExchanger", + "bufferSize": 32 + } + }, + "container": { + "job": { + "reportInterval": 10000 + }, + "taskGroup": { + "channel": 5 + }, + "trace": { + "enable": "false" + } + + }, + "statistics": { + "collector": { + "plugin": { + "taskClass": "com.alibaba.datax.core.statistics.plugin.task.StdoutPluginCollector", + "maxDirtyNumber": 10 + } + } + } + } +} diff --git a/datax-example/datax-example-core/src/test/java/com/alibaba/datax/example/util/PathUtilTest.java b/datax-example/datax-example-core/src/test/java/com/alibaba/datax/example/util/PathUtilTest.java new file mode 100644 index 00000000..8985b54c --- /dev/null +++ b/datax-example/datax-example-core/src/test/java/com/alibaba/datax/example/util/PathUtilTest.java @@ -0,0 +1,19 @@ +package com.alibaba.datax.example.util; + +import org.junit.Assert; +import org.junit.Test; + +/** + * {@code Author} FuYouJ + * {@code Date} 2023/8/19 21:38 + */ + +public class PathUtilTest { + + @Test + public void testParseClassPathFile() { + String path = "/pathTest.json"; + String absolutePathFromClassPath = PathUtil.getAbsolutePathFromClassPath(path); + Assert.assertNotNull(absolutePathFromClassPath); + } +} diff --git a/datax-example/datax-example-core/src/test/resources/pathTest.json b/datax-example/datax-example-core/src/test/resources/pathTest.json new file mode 100644 index 00000000..9e26dfee --- /dev/null +++ b/datax-example/datax-example-core/src/test/resources/pathTest.json @@ -0,0 +1 @@ +{} \ No newline at end of file diff --git a/datax-example/datax-example-neo4j/pom.xml b/datax-example/datax-example-neo4j/pom.xml new file mode 100644 index 00000000..303b14a8 --- /dev/null +++ b/datax-example/datax-example-neo4j/pom.xml @@ -0,0 +1,43 @@ + + + 4.0.0 + + com.alibaba.datax + datax-example + 0.0.1-SNAPSHOT + + + datax-example-neo4j + + + 8 + 8 + UTF-8 + 1.17.6 + 4.4.9 + + + + com.alibaba.datax + datax-example-core + 0.0.1-SNAPSHOT + + + org.testcontainers + testcontainers + ${test.container.version} + + + com.alibaba.datax + neo4jwriter + 0.0.1-SNAPSHOT + + + com.alibaba.datax + datax-example-streamreader + 0.0.1-SNAPSHOT + + + \ No newline at end of file diff --git a/datax-example/datax-example-neo4j/src/test/java/com/alibaba/datax/example/neo4j/StreamReader2Neo4jWriterTest.java b/datax-example/datax-example-neo4j/src/test/java/com/alibaba/datax/example/neo4j/StreamReader2Neo4jWriterTest.java new file mode 100644 index 00000000..9cf01253 --- /dev/null +++ b/datax-example/datax-example-neo4j/src/test/java/com/alibaba/datax/example/neo4j/StreamReader2Neo4jWriterTest.java @@ -0,0 +1,138 @@ +package com.alibaba.datax.example.neo4j; + +import com.alibaba.datax.example.ExampleContainer; +import com.alibaba.datax.example.util.PathUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.neo4j.driver.*; +import org.neo4j.driver.types.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; + +import java.net.URI; +import java.util.Arrays; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +/** + * {@code Author} FuYouJ + * {@code Date} 2023/8/19 21:48 + */ + +public class StreamReader2Neo4jWriterTest { + private static final Logger LOGGER = LoggerFactory.getLogger(StreamReader2Neo4jWriterTest.class); + private static final String CONTAINER_IMAGE = "neo4j:5.9.0"; + + private static final String CONTAINER_HOST = "neo4j-host"; + private static final int HTTP_PORT = 7474; + private static final int BOLT_PORT = 7687; + private static final String CONTAINER_NEO4J_USERNAME = "neo4j"; + private static final String CONTAINER_NEO4J_PASSWORD = "Test@12343"; + private static final URI CONTAINER_URI = URI.create("neo4j://localhost:" + BOLT_PORT); + + protected static final Network NETWORK = Network.newNetwork(); + + private GenericContainer container; + protected Driver neo4jDriver; + protected Session neo4jSession; + private static final int CHANNEL = 5; + private static final int READER_NUM = 10; + + @Before + public void init() { + DockerImageName imageName = DockerImageName.parse(CONTAINER_IMAGE); + container = + new GenericContainer<>(imageName) + .withNetwork(NETWORK) + .withNetworkAliases(CONTAINER_HOST) + .withExposedPorts(HTTP_PORT, BOLT_PORT) + .withEnv( + "NEO4J_AUTH", + CONTAINER_NEO4J_USERNAME + "/" + CONTAINER_NEO4J_PASSWORD) + .withEnv("apoc.export.file.enabled", "true") + .withEnv("apoc.import.file.enabled", "true") + .withEnv("apoc.import.file.use_neo4j_config", "true") + .withEnv("NEO4J_PLUGINS", "[\"apoc\"]") + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(CONTAINER_IMAGE))); + container.setPortBindings( + Arrays.asList( + String.format("%s:%s", HTTP_PORT, HTTP_PORT), + String.format("%s:%s", BOLT_PORT, BOLT_PORT))); + Startables.deepStart(Stream.of(container)).join(); + LOGGER.info("container started"); + Awaitility.given() + .ignoreExceptions() + .await() + .atMost(30, TimeUnit.SECONDS) + .untilAsserted(this::initConnection); + } + + //在neo4jWriter模块使用Example测试整个job,方便发现整个流程的代码问题 + @Test + public void streamReader2Neo4j() { + + deleteHistoryIfExist(); + + String path = "/streamreader2neo4j.json"; + String jobPath = PathUtil.getAbsolutePathFromClassPath(path); + + ExampleContainer.start(jobPath); + + //根据channel和reader的mock数据,校验结果集是否符合预期 + verifyWriteResult(); + } + + private void deleteHistoryIfExist() { + String query = "match (n:StreamReader) return n limit 1"; + String delete = "match (n:StreamReader) delete n"; + if (neo4jSession.run(query).hasNext()) { + neo4jSession.run(delete); + } + } + + private void verifyWriteResult() { + int total = CHANNEL * READER_NUM; + String query = "match (n:StreamReader) return n"; + Result run = neo4jSession.run(query); + int count = 0; + while (run.hasNext()) { + Record record = run.next(); + Node node = record.get("n").asNode(); + if (node.hasLabel("StreamReader")) { + count++; + } + } + Assert.assertEquals(count, total); + } + @After + public void destroy() { + if (neo4jSession != null) { + neo4jSession.close(); + } + if (neo4jDriver != null) { + neo4jDriver.close(); + } + if (container != null) { + container.close(); + } + } + + private void initConnection() { + neo4jDriver = + GraphDatabase.driver( + CONTAINER_URI, + AuthTokens.basic(CONTAINER_NEO4J_USERNAME, CONTAINER_NEO4J_PASSWORD)); + neo4jSession = neo4jDriver.session(SessionConfig.forDatabase("neo4j")); + } +} diff --git a/datax-example/datax-example-neo4j/src/test/resources/streamreader2neo4j.json b/datax-example/datax-example-neo4j/src/test/resources/streamreader2neo4j.json new file mode 100644 index 00000000..3d543ce3 --- /dev/null +++ b/datax-example/datax-example-neo4j/src/test/resources/streamreader2neo4j.json @@ -0,0 +1,51 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "sliceRecordCount": 10, + "column": [ + { + "type": "string", + "value": "StreamReader" + }, + { + "type": "string", + "value": "1997" + } + ] + } + }, + "writer": { + "name": "neo4jWriter", + "parameter": { + "uri": "bolt://localhost:7687", + "username":"neo4j", + "password":"Test@12343", + "database":"neo4j", + "cypher": "unwind $batch as row CALL apoc.cypher.doIt( 'create (n:`' + row.Label + '`{id:$id})' ,{id: row.id} ) YIELD value RETURN 1 ", + "batchDataVariableName": "batch", + "batchSize": "3", + "properties": [ + { + "name": "Label", + "type": "string" + }, + { + "name": "id", + "type": "STRING" + } + ] + } + } + } + ], + "setting": { + "speed": { + "channel": 5 + } + } + } +} \ No newline at end of file diff --git a/datax-example/datax-example-streamreader/pom.xml b/datax-example/datax-example-streamreader/pom.xml new file mode 100644 index 00000000..ea70de10 --- /dev/null +++ b/datax-example/datax-example-streamreader/pom.xml @@ -0,0 +1,37 @@ + + + 4.0.0 + + com.alibaba.datax + datax-example + 0.0.1-SNAPSHOT + + + datax-example-streamreader + + + 8 + 8 + UTF-8 + + + + com.alibaba.datax + datax-example-core + 0.0.1-SNAPSHOT + + + com.alibaba.datax + streamreader + 0.0.1-SNAPSHOT + + + com.alibaba.datax + streamwriter + 0.0.1-SNAPSHOT + + + + \ No newline at end of file diff --git a/datax-example/datax-example-streamreader/src/test/java/com/alibaba/datax/example/streamreader/StreamReader2StreamWriterTest.java b/datax-example/datax-example-streamreader/src/test/java/com/alibaba/datax/example/streamreader/StreamReader2StreamWriterTest.java new file mode 100644 index 00000000..71d083d0 --- /dev/null +++ b/datax-example/datax-example-streamreader/src/test/java/com/alibaba/datax/example/streamreader/StreamReader2StreamWriterTest.java @@ -0,0 +1,19 @@ +package com.alibaba.datax.example.streamreader; + +import com.alibaba.datax.example.ExampleContainer; +import com.alibaba.datax.example.util.PathUtil; +import org.junit.Test; + +/** + * {@code Author} FuYouJ + * {@code Date} 2023/8/14 20:16 + */ + +public class StreamReader2StreamWriterTest { + @Test + public void testStreamReader2StreamWriter() { + String path = "/stream2stream.json"; + String jobPath = PathUtil.getAbsolutePathFromClassPath(path); + ExampleContainer.start(jobPath); + } +} diff --git a/datax-example/datax-example-streamreader/src/test/resources/stream2stream.json b/datax-example/datax-example-streamreader/src/test/resources/stream2stream.json new file mode 100644 index 00000000..b2a57395 --- /dev/null +++ b/datax-example/datax-example-streamreader/src/test/resources/stream2stream.json @@ -0,0 +1,36 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "sliceRecordCount": 10, + "column": [ + { + "type": "long", + "value": "10" + }, + { + "type": "string", + "value": "hello,你好,世界-DataX" + } + ] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "encoding": "UTF-8", + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 5 + } + } + } +} \ No newline at end of file diff --git a/datax-example/doc/README.md b/datax-example/doc/README.md new file mode 100644 index 00000000..15f77e87 --- /dev/null +++ b/datax-example/doc/README.md @@ -0,0 +1,107 @@ +## [DataX-Example]调试datax插件的模块 + +### 为什么要开发这个模块 + +一般使用DataX启动数据同步任务是从datax.py 脚本开始,获取程序datax包目录设置到系统变量datax.home里,此后系统核心插件的加载,配置初始化均依赖于变量datax.home,这带来了一些麻烦,以一次本地 DeBug streamreader 插件为例。 + +- maven 打包 datax 生成 datax 目录 +- 在 IDE 中 设置系统环境变量 datax.home,或者在Engine启动类中硬编码设置datax.home。 +- 修改插件 streamreader 代码 +- 再次 maven 打包,使JarLoader 能够加载到最新的 streamreader 代码。 +- 调试代码 + +在以上步骤中,打包完全不必要且最耗时,等待打包也最煎熬。 + +所以我编写一个新的模块(datax-example),此模块特用于本地调试和复现 BUG。如果模块顺利编写完成,那么以上流程将被简化至两步。 + +- 修改插件 streamreader 代码。 +- 调试代码 + +img + +### 目录结构 +该目录结构演示了如何使用datax-example-core编写测试用例,和校验代码流程。 +img + +### 实现原理 + +- 不修改原有的ConfigParer,使用新的ExampleConfigParser,仅用于example模块。他不依赖datax.home,而是依赖ide编译后的target目录 +- 将ide的target目录作为每个插件的目录类加载目录。 + +![img](img/img02.png) + +### 如何使用 +1.修改插件的pom文件,做如下改动。以streamreader为例。
+改动前 +```xml + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + +``` +改动后 +```xml + + + + + src/main/resources + + **/*.* + + true + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + +``` +#### 在测试模块模块使用 +参考datax-example/datax-example-streamreader的StreamReader2StreamWriterTest.java +```java +public class StreamReader2StreamWriterTest { + @Test + public void testStreamReader2StreamWriter() { + String path = "/stream2stream.json"; + String jobPath = PathUtil.getAbsolutePathFromClassPath(path); + ExampleContainer.start(jobPath); + } +} + +``` +参考datax-example/datax-example-neo4j的StreamReader2Neo4jWriterTest +```java +public class StreamReader2Neo4jWriterTest{ +@Test + public void streamReader2Neo4j() { + + deleteHistoryIfExist(); + + String path = "/streamreader2neo4j.json"; + String jobPath = PathUtil.getAbsolutePathFromClassPath(path); + + ExampleContainer.start(jobPath); + + //根据channel和reader的mock数据,校验结果集是否符合预期 + verifyWriteResult(); + } +} +``` \ No newline at end of file diff --git a/datax-example/doc/img/img01.png b/datax-example/doc/img/img01.png new file mode 100644 index 00000000..d0431c1a Binary files /dev/null and b/datax-example/doc/img/img01.png differ diff --git a/datax-example/doc/img/img02.png b/datax-example/doc/img/img02.png new file mode 100644 index 00000000..eec860d4 Binary files /dev/null and b/datax-example/doc/img/img02.png differ diff --git a/datax-example/doc/img/img03.png b/datax-example/doc/img/img03.png new file mode 100644 index 00000000..731f81bd Binary files /dev/null and b/datax-example/doc/img/img03.png differ diff --git a/datax-example/pom.xml b/datax-example/pom.xml new file mode 100644 index 00000000..9c4c9200 --- /dev/null +++ b/datax-example/pom.xml @@ -0,0 +1,68 @@ + + + 4.0.0 + + com.alibaba.datax + datax-all + 0.0.1-SNAPSHOT + + + datax-example + pom + + datax-example-core + datax-example-streamreader + datax-example-neo4j + + + + 8 + 8 + UTF-8 + 4.13.2 + + + + com.alibaba.datax + datax-common + 0.0.1-SNAPSHOT + + + com.alibaba.datax + datax-core + 0.0.1-SNAPSHOT + + + junit + junit + ${junit4.version} + test + + + + + + + src/main/resources + + **/*.* + + true + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + + + \ No newline at end of file diff --git a/dataxPluginDev.md b/dataxPluginDev.md index 4483f270..8c7241bf 100644 --- a/dataxPluginDev.md +++ b/dataxPluginDev.md @@ -447,6 +447,9 @@ DataX的内部类型在实现上会选用不同的java类型: 3. 用户在插件中在`reader`/`writer`配置的`name`字段指定插件名字。框架根据插件的类型(`reader`/`writer`)和插件名称去插件的路径下扫描所有的jar,加入`classpath`。 4. 根据插件配置中定义的入口类,框架通过反射实例化对应的`Job`和`Task`对象。 +### 编写测试用例 +1. 在datax-example工程下新建新的插件测试模块,调用`ExampleContainer.start(jobPath)`方法来检测你的代码逻辑是否正确。[datax-example使用](https://github.com/alibaba/DataX/blob/master/datax-example/doc/README.md) + ## 三、Last but not Least diff --git a/ftpreader/pom.xml b/ftpreader/pom.xml index 7778d491..57bf889d 100755 --- a/ftpreader/pom.xml +++ b/ftpreader/pom.xml @@ -45,7 +45,7 @@ com.jcraft jsch - 0.1.51 + 0.1.54 commons-net @@ -89,4 +89,4 @@ - + diff --git a/ftpreader/src/main/java/com/alibaba/datax/plugin/reader/ftpreader/SftpHelper.java b/ftpreader/src/main/java/com/alibaba/datax/plugin/reader/ftpreader/SftpHelper.java index d25b040c..6e42e10c 100644 --- a/ftpreader/src/main/java/com/alibaba/datax/plugin/reader/ftpreader/SftpHelper.java +++ b/ftpreader/src/main/java/com/alibaba/datax/plugin/reader/ftpreader/SftpHelper.java @@ -64,6 +64,8 @@ public class SftpHelper extends FtpHelper { String message = String.format("请确认连接ftp服务器端口是否正确,错误的端口: [%s] ", port); LOG.error(message); throw DataXException.asDataXException(FtpReaderErrorCode.FAIL_LOGIN, message, e); + }else{ + throw DataXException.asDataXException(FtpReaderErrorCode.COMMAND_FTP_IO_EXCEPTION, "", e); } }else { if("Auth fail".equals(e.getMessage())){ diff --git a/ftpwriter/pom.xml b/ftpwriter/pom.xml index 69ec4a07..bf7ce83d 100644 --- a/ftpwriter/pom.xml +++ b/ftpwriter/pom.xml @@ -45,7 +45,7 @@ com.jcraft jsch - 0.1.51 + 0.1.54 commons-net diff --git a/gaussdbreader/doc/gaussdbreader.md b/gaussdbreader/doc/gaussdbreader.md new file mode 100644 index 00000000..5caa4d59 --- /dev/null +++ b/gaussdbreader/doc/gaussdbreader.md @@ -0,0 +1,297 @@ + +# GaussDbReader 插件文档 + + +___ + + +## 1 快速介绍 + +GaussDbReader插件实现了从GaussDB读取数据。在底层实现上,GaussDbReader通过JDBC连接远程GaussDB数据库,并执行相应的sql语句将数据从GaussDB库中SELECT出来。 + +## 2 实现原理 + +简而言之,GaussDbReader通过JDBC连接器连接到远程的GaussDB数据库,并根据用户配置的信息生成查询SELECT SQL语句并发送到远程GaussDB数据库,并将该SQL执行返回结果使用DataX自定义的数据类型拼装为抽象的数据集,并传递给下游Writer处理。 + +对于用户配置Table、Column、Where的信息,GaussDbReader将其拼接为SQL语句发送到GaussDB数据库;对于用户配置querySql信息,GaussDbReader直接将其发送到GaussDB数据库。 + + +## 3 功能说明 + +### 3.1 配置样例 + +* 配置一个从GaussDB数据库同步抽取数据到本地的作业: + +``` +{ + "job": { + "setting": { + "speed": { + //设置传输速度,单位为byte/s,DataX运行会尽可能达到该速度但是不超过它. + "byte": 1048576 + }, + //出错限制 + "errorLimit": { + //出错的record条数上限,当大于该值即报错。 + "record": 0, + //出错的record百分比上限 1.0表示100%,0.02表示2% + "percentage": 0.02 + } + }, + "content": [ + { + "reader": { + "name": "gaussdbreader", + "parameter": { + // 数据库连接用户名 + "username": "xx", + // 数据库连接密码 + "password": "xx", + "column": [ + "id","name" + ], + //切分主键 + "splitPk": "id", + "connection": [ + { + "table": [ + "table" + ], + "jdbcUrl": [ + "jdbc:opengauss://host:port/database" + ] + } + ] + } + }, + "writer": { + //writer类型 + "name": "streamwriter", + //是否打印内容 + "parameter": { + "print":true, + } + } + } + ] + } +} + +``` + +* 配置一个自定义SQL的数据库同步任务到本地内容的作业: + +```json +{ + "job": { + "setting": { + "speed": 1048576 + }, + "content": [ + { + "reader": { + "name": "gaussdbreader", + "parameter": { + "username": "xx", + "password": "xx", + "where": "", + "connection": [ + { + "querySql": [ + "select db_id,on_line_flag from db_info where db_id < 10;" + ], + "jdbcUrl": [ + "jdbc:opengauss://host:port/database", "jdbc:opengauss://host:port/database" + ] + } + ] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": false, + "encoding": "UTF-8" + } + } + } + ] + } +} +``` + + +### 3.2 参数说明 + +* **jdbcUrl** + + * 描述:描述的是到对端数据库的JDBC连接信息,使用JSON的数组描述,并支持一个库填写多个连接地址。之所以使用JSON数组描述连接信息,是因为阿里集团内部支持多个IP探测,如果配置了多个,GaussDbReader可以依次探测ip的可连接性,直到选择一个合法的IP。如果全部连接失败,GaussDbReader报错。 注意,jdbcUrl必须包含在connection配置单元中。对于阿里集团外部使用情况,JSON数组填写一个JDBC连接即可。 + + jdbcUrl按照GaussDB官方规范,并可以填写连接附件控制信息。具体请参看[GaussDB官方文档](https://docs.opengauss.org/zh/docs/3.1.0/docs/Developerguide/java-sql-Connection.html)。 + + * 必选:是
+ + * 默认值:无
+ +* **username** + + * 描述:数据源的用户名
+ + * 必选:是
+ + * 默认值:无
+ +* **password** + + * 描述:数据源指定用户名的密码
+ + * 必选:是
+ + * 默认值:无
+ +* **table** + + * 描述:所选取的需要同步的表。使用JSON的数组描述,因此支持多张表同时抽取。当配置为多张表时,用户自己需保证多张表是同一schema结构,GaussDbReader不予检查表是否同一逻辑表。注意,table必须包含在connection配置单元中。
+ + * 必选:是
+ + * 默认值:无
+ +* **column** + + * 描述:所配置的表中需要同步的列名集合,使用JSON的数组描述字段信息。用户使用\*代表默认使用所有列配置,例如['\*']。 + + 支持列裁剪,即列可以挑选部分列进行导出。 + + 支持列换序,即列可以不按照表schema信息进行导出。 + + 支持常量配置,用户需要按照GaussDB语法格式: + ["id", "'hello'::varchar", "true", "2.5::real", "power(2,3)"] + id为普通列名,'hello'::varchar为字符串常量,true为布尔值,2.5为浮点数, power(2,3)为函数。 + + **column必须用户显示指定同步的列集合,不允许为空!** + + * 必选:是
+ + * 默认值:无
+ +* **splitPk** + + * 描述:GaussDbReader进行数据抽取时,如果指定splitPk,表示用户希望使用splitPk代表的字段进行数据分片,DataX因此会启动并发任务进行数据同步,这样可以大大提高数据同步的效能。 + + 推荐splitPk用户使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + + 目前splitPk仅支持整形数据切分,`不支持浮点、字符串型、日期等其他类型`。如果用户指定其他非支持类型,GaussDbReader将报错! + + splitPk设置为空,底层将视作用户不允许对单表进行切分,因此使用单通道进行抽取。 + + * 必选:否
+ + * 默认值:空
+ +* **where** + + * 描述:筛选条件,GaussDbReader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。
+ + where条件可以有效地进行业务增量同步。 where条件不配置或者为空,视作全表同步数据。 + + * 必选:否
+ + * 默认值:无
+ +* **querySql** + + * 描述:在有些业务场景下,where这一配置项不足以描述所筛选的条件,用户可以通过该配置型来自定义筛选SQL。当用户配置了这一项之后,DataX系统就会忽略table,column这些配置型,直接使用这个配置项的内容对数据进行筛选,例如需要进行多表join后同步数据,使用select a,b from table_a join table_b on table_a.id = table_b.id
+ + `当用户配置querySql时,GaussDbReader直接忽略table、column、where条件的配置`。 + + * 必选:否
+ + * 默认值:无
+ +* **fetchSize** + + * 描述:该配置项定义了插件和数据库服务器端每次批量数据获取条数,该值决定了DataX和服务器端的网络交互次数,能够较大的提升数据抽取性能。
+ + `注意,该值过大(>2048)可能造成DataX进程OOM。`。 + + * 必选:否
+ + * 默认值:1024
+ + +### 3.3 类型转换 + +目前GaussDbReader支持大部分GaussDB类型,但也存在部分个别类型没有支持的情况,请注意检查你的类型。 + +下面列出GaussDbReader针对GaussDB类型转换列表: + + +| DataX 内部类型| GaussDB 数据类型 | +| -------- | ----- | +| Long |bigint, bigserial, integer, smallint, serial | +| Double |double precision, money, numeric, real | +| String |varchar, char, text, bit, inet| +| Date |date, time, timestamp | +| Boolean |bool| +| Bytes |bytea| + +请注意: + +* `除上述罗列字段类型外,其他类型均不支持; money,inet,bit需用户使用a_inet::varchar类似的语法转换`。 + +## 4 性能报告 + +### 4.1 环境准备 + +#### 4.1.1 数据特征 +建表语句: + +create table pref_test( + id serial, + a_bigint bigint, + a_bit bit(10), + a_boolean boolean, + a_char character(5), + a_date date, + a_double double precision, + a_integer integer, + a_money money, + a_num numeric(10,2), + a_real real, + a_smallint smallint, + a_text text, + a_time time, + a_timestamp timestamp +) + +#### 4.1.2 机器参数 + +* 执行DataX的机器参数为: + 1. cpu: 16核 Intel(R) Xeon(R) CPU E5620 @ 2.40GHz + 2. mem: MemTotal: 24676836kB MemFree: 6365080kB + 3. net: 百兆双网卡 + +* GaussDB数据库机器参数为: + D12 24逻辑核 192G内存 12*480G SSD 阵列 + + +### 4.2 测试报告 + +#### 4.2.1 单表测试报告 + + +| 通道数 | 是否按照主键切分 | DataX速度(Rec/s) | DataX流量(MB/s) | DataX机器运行负载 | +|--------|--------| --------|--------|--------| +|1| 否 | 10211 | 0.63 | 0.2 | +|1| 是 | 10211 | 0.63 | 0.2 | +|4| 否 | 10211 | 0.63 | 0.2 | +|4| 是 | 40000 | 2.48 | 0.5 | +|8| 否 | 10211 | 0.63 | 0.2 | +|8| 是 | 78048 | 4.84 | 0.8 | + + +说明: + +1. 这里的单表,主键类型为 serial,数据分布均匀。 +2. 对单表如果没有按照主键切分,那么配置通道个数不会提升速度,效果与1个通道一样。 diff --git a/gaussdbreader/pom.xml b/gaussdbreader/pom.xml new file mode 100644 index 00000000..ad2e0ba0 --- /dev/null +++ b/gaussdbreader/pom.xml @@ -0,0 +1,86 @@ + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + 4.0.0 + + gaussdbreader + gaussdbreader + 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} + + + + org.opengauss + opengauss-jdbc + 3.0.0 + + + + + + + + + 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/gaussdbreader/src/main/assembly/package.xml b/gaussdbreader/src/main/assembly/package.xml new file mode 100755 index 00000000..65601e45 --- /dev/null +++ b/gaussdbreader/src/main/assembly/package.xml @@ -0,0 +1,35 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + plugin_job_template.json + + plugin/reader/gaussdbreader + + + target/ + + gaussdbreader-0.0.1-SNAPSHOT.jar + + plugin/reader/gaussdbreader + + + + + + false + plugin/reader/gaussdbreader/libs + runtime + + + diff --git a/gaussdbreader/src/main/java/com/alibaba/datax/plugin/reader/gaussdbreader/Constant.java b/gaussdbreader/src/main/java/com/alibaba/datax/plugin/reader/gaussdbreader/Constant.java new file mode 100644 index 00000000..33cdd309 --- /dev/null +++ b/gaussdbreader/src/main/java/com/alibaba/datax/plugin/reader/gaussdbreader/Constant.java @@ -0,0 +1,7 @@ +package com.alibaba.datax.plugin.reader.gaussdbreader; + +public class Constant { + + public static final int DEFAULT_FETCH_SIZE = 1000; + +} diff --git a/gaussdbreader/src/main/java/com/alibaba/datax/plugin/reader/gaussdbreader/GaussDbReader.java b/gaussdbreader/src/main/java/com/alibaba/datax/plugin/reader/gaussdbreader/GaussDbReader.java new file mode 100644 index 00000000..ca158ab7 --- /dev/null +++ b/gaussdbreader/src/main/java/com/alibaba/datax/plugin/reader/gaussdbreader/GaussDbReader.java @@ -0,0 +1,86 @@ +package com.alibaba.datax.plugin.reader.gaussdbreader; + +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.plugin.rdbms.reader.CommonRdbmsReader; +import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; +import com.alibaba.datax.plugin.rdbms.util.DataBaseType; + +import java.util.List; + +public class GaussDbReader extends Reader { + + private static final DataBaseType DATABASE_TYPE = DataBaseType.GaussDB; + + public static class Job extends Reader.Job { + + private Configuration originalConfig; + private CommonRdbmsReader.Job commonRdbmsReaderMaster; + + @Override + public void init() { + this.originalConfig = super.getPluginJobConf(); + int fetchSize = this.originalConfig.getInt(com.alibaba.datax.plugin.rdbms.reader.Constant.FETCH_SIZE, + Constant.DEFAULT_FETCH_SIZE); + if (fetchSize < 1) { + throw DataXException.asDataXException(DBUtilErrorCode.REQUIRED_VALUE, + String.format("您配置的fetchSize有误,根据DataX的设计,fetchSize : [%d] 设置值不能小于 1.", fetchSize)); + } + this.originalConfig.set(com.alibaba.datax.plugin.rdbms.reader.Constant.FETCH_SIZE, fetchSize); + + this.commonRdbmsReaderMaster = new CommonRdbmsReader.Job(DATABASE_TYPE); + this.commonRdbmsReaderMaster.init(this.originalConfig); + } + + @Override + public List split(int adviceNumber) { + return this.commonRdbmsReaderMaster.split(this.originalConfig, adviceNumber); + } + + @Override + public void post() { + this.commonRdbmsReaderMaster.post(this.originalConfig); + } + + @Override + public void destroy() { + this.commonRdbmsReaderMaster.destroy(this.originalConfig); + } + + } + + public static class Task extends Reader.Task { + + private Configuration readerSliceConfig; + private CommonRdbmsReader.Task commonRdbmsReaderSlave; + + @Override + public void init() { + this.readerSliceConfig = super.getPluginJobConf(); + this.commonRdbmsReaderSlave = new CommonRdbmsReader.Task(DATABASE_TYPE,super.getTaskGroupId(), super.getTaskId()); + this.commonRdbmsReaderSlave.init(this.readerSliceConfig); + } + + @Override + public void startRead(RecordSender recordSender) { + int fetchSize = this.readerSliceConfig.getInt(com.alibaba.datax.plugin.rdbms.reader.Constant.FETCH_SIZE); + + this.commonRdbmsReaderSlave.startRead(this.readerSliceConfig, recordSender, + super.getTaskPluginCollector(), fetchSize); + } + + @Override + public void post() { + this.commonRdbmsReaderSlave.post(this.readerSliceConfig); + } + + @Override + public void destroy() { + this.commonRdbmsReaderSlave.destroy(this.readerSliceConfig); + } + + } + +} diff --git a/gaussdbreader/src/main/resources/plugin.json b/gaussdbreader/src/main/resources/plugin.json new file mode 100755 index 00000000..7d4ac8de --- /dev/null +++ b/gaussdbreader/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "gaussdbreader", + "class": "com.alibaba.datax.plugin.reader.gaussdbreader.GaussDbReader", + "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/gaussdbreader/src/main/resources/plugin_job_template.json b/gaussdbreader/src/main/resources/plugin_job_template.json new file mode 100644 index 00000000..e39220eb --- /dev/null +++ b/gaussdbreader/src/main/resources/plugin_job_template.json @@ -0,0 +1,13 @@ +{ + "name": "gaussdbreader", + "parameter": { + "username": "", + "password": "", + "connection": [ + { + "table": [], + "jdbcUrl": [] + } + ] + } +} \ No newline at end of file diff --git a/gaussdbwriter/doc/gaussdbwriter.md b/gaussdbwriter/doc/gaussdbwriter.md new file mode 100644 index 00000000..e65b74d3 --- /dev/null +++ b/gaussdbwriter/doc/gaussdbwriter.md @@ -0,0 +1,267 @@ +# DataX GaussDbWriter + + +--- + + +## 1 快速介绍 + +GaussDbWriter插件实现了写入数据到 GaussDB主库目的表的功能。在底层实现上,GaussDbWriter通过JDBC连接远程 GaussDB 数据库,并执行相应的 insert into ... sql 语句将数据写入 GaussDB,内部会分批次提交入库。 + +GaussDbWriter面向ETL开发工程师,他们使用GaussDbWriter从数仓导入数据到GaussDB。同时 GaussDbWriter亦可以作为数据迁移工具为DBA等用户提供服务。 + + +## 2 实现原理 + +GaussDbWriter通过 DataX 框架获取 Reader 生成的协议数据,根据你配置生成相应的SQL插入语句 + + +* `insert into...`(当主键/唯一性索引冲突时会写不进去冲突的行) + +
+ + 注意: + 1. 目的表所在数据库必须是主库才能写入数据;整个任务至少需具备 insert into...的权限,是否需要其他权限,取决于你任务配置中在 preSql 和 postSql 中指定的语句。 + 2. GaussDbWriter和MysqlWriter不同,不支持配置writeMode参数。 + + +## 3 功能说明 + +### 3.1 配置样例 + +* 这里使用一份从内存产生到 GaussDbWriter导入的数据。 + +```json +{ + "job": { + "setting": { + "speed": { + "channel": 1 + } + }, + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column" : [ + { + "value": "DataX", + "type": "string" + }, + { + "value": 19880808, + "type": "long" + }, + { + "value": "1988-08-08 08:08:08", + "type": "date" + }, + { + "value": true, + "type": "bool" + }, + { + "value": "test", + "type": "bytes" + } + ], + "sliceRecordCount": 1000 + } + }, + "writer": { + "name": "gaussdbwriter", + "parameter": { + "username": "xx", + "password": "xx", + "column": [ + "id", + "name" + ], + "preSql": [ + "delete from test" + ], + "connection": [ + { + "jdbcUrl": "jdbc:opengauss://127.0.0.1:3002/datax", + "table": [ + "test" + ] + } + ] + } + } + } + ] + } +} + +``` + + +### 3.2 参数说明 + +* **jdbcUrl** + + * 描述:目的数据库的 JDBC 连接信息 ,jdbcUrl必须包含在connection配置单元中。 + + 注意:1、在一个数据库上只能配置一个值。 + 2、jdbcUrl按照GaussDB官方规范,并可以填写连接附加参数信息。具体请参看GaussDB官方文档或者咨询对应 DBA。 + + +* 必选:是
+ +* 默认值:无
+ +* **username** + + * 描述:目的数据库的用户名
+ + * 必选:是
+ + * 默认值:无
+ +* **password** + + * 描述:目的数据库的密码
+ + * 必选:是
+ + * 默认值:无
+ +* **table** + + * 描述:目的表的表名称。支持写入一个或者多个表。当配置为多张表时,必须确保所有表结构保持一致。 + + 注意:table 和 jdbcUrl 必须包含在 connection 配置单元中 + + * 必选:是
+ + * 默认值:无
+ +* **column** + + * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。如果要依次写入全部列,使用\*表示, 例如: "column": ["\*"] + + 注意:1、我们强烈不推荐你这样配置,因为当你目的表字段个数、类型等有改动时,你的任务可能运行不正确或者失败 + 2、此处 column 不能配置任何常量值 + + * 必选:是
+ + * 默认值:否
+ +* **preSql** + + * 描述:写入数据到目的表前,会先执行这里的标准语句。如果 Sql 中有你需要操作到的表名称,请使用 `@table` 表示,这样在实际执行 Sql 语句时,会对变量按照实际表名称进行替换。比如你的任务是要写入到目的端的100个同构分表(表名称为:datax_00,datax01, ... datax_98,datax_99),并且你希望导入数据前,先对表中数据进行删除操作,那么你可以这样配置:`"preSql":["delete from @table"]`,效果是:在执行到每个表写入数据前,会先执行对应的 delete from 对应表名称
+ + * 必选:否
+ + * 默认值:无
+ +* **postSql** + + * 描述:写入数据到目的表后,会执行这里的标准语句。(原理同 preSql )
+ + * 必选:否
+ + * 默认值:无
+ +* **batchSize** + + * 描述:一次性批量提交的记录数大小,该值可以极大减少DataX与GaussDB的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成DataX运行进程OOM情况。
+ + * 必选:否
+ + * 默认值:1024
+ +### 3.3 类型转换 + +目前 GaussDbWriter支持大部分 GaussDB类型,但也存在部分没有支持的情况,请注意检查你的类型。 + +下面列出 GaussDbWriter针对 GaussDB类型转换列表: + +| DataX 内部类型| GaussDB 数据类型 | +| -------- | ----- | +| Long |bigint, bigserial, integer, smallint, serial | +| Double |double precision, money, numeric, real | +| String |varchar, char, text, bit| +| Date |date, time, timestamp | +| Boolean |bool| +| Bytes |bytea| + +## 4 性能报告 + +### 4.1 环境准备 + +#### 4.1.1 数据特征 +建表语句: + +create table pref_test( +id serial, +a_bigint bigint, +a_bit bit(10), +a_boolean boolean, +a_char character(5), +a_date date, +a_double double precision, +a_integer integer, +a_money money, +a_num numeric(10,2), +a_real real, +a_smallint smallint, +a_text text, +a_time time, +a_timestamp timestamp +) + +#### 4.1.2 机器参数 + +* 执行DataX的机器参数为: + 1. cpu: 16核 Intel(R) Xeon(R) CPU E5620 @ 2.40GHz + 2. mem: MemTotal: 24676836kB MemFree: 6365080kB + 3. net: 百兆双网卡 + +* GaussDB数据库机器参数为: + D12 24逻辑核 192G内存 12*480G SSD 阵列 + + +### 4.2 测试报告 + +#### 4.2.1 单表测试报告 + +| 通道数| 批量提交batchSize | DataX速度(Rec/s)| DataX流量(M/s) | DataX机器运行负载 +|--------|--------| --------|--------|--------|--------| +|1| 128 | 9259 | 0.55 | 0.3 +|1| 512 | 10869 | 0.653 | 0.3 +|1| 2048 | 9803 | 0.589 | 0.8 +|4| 128 | 30303 | 1.82 | 1 +|4| 512 | 36363 | 2.18 | 1 +|4| 2048 | 36363 | 2.18 | 1 +|8| 128 | 57142 | 3.43 | 2 +|8| 512 | 66666 | 4.01 | 1.5 +|8| 2048 | 66666 | 4.01 | 1.1 +|16| 128 | 88888 | 5.34 | 1.8 +|16| 2048 | 94117 | 5.65 | 2.5 +|32| 512 | 76190 | 4.58 | 3 + +#### 4.2.2 性能测试小结 +1. `channel数对性能影响很大` +2. `通常不建议写入数据库时,通道个数 > 32` + + +## FAQ + +*** + +**Q: GaussDbWriter 执行 postSql 语句报错,那么数据导入到目标数据库了吗?** + +A: DataX 导入过程存在三块逻辑,pre 操作、导入操作、post 操作,其中任意一环报错,DataX 作业报错。由于 DataX 不能保证在同一个事务完成上述几个操作,因此有可能数据已经落入到目标端。 + +*** + +**Q: 按照上述说法,那么有部分脏数据导入数据库,如果影响到线上数据库怎么办?** + +A: 目前有两种解法,第一种配置 pre 语句,该 sql 可以清理当天导入数据, DataX 每次导入时候可以把上次清理干净并导入完整数据。 +第二种,向临时表导入数据,完成后再 rename 到线上表。 + +*** diff --git a/gaussdbwriter/pom.xml b/gaussdbwriter/pom.xml new file mode 100644 index 00000000..9da02eff --- /dev/null +++ b/gaussdbwriter/pom.xml @@ -0,0 +1,86 @@ + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + 4.0.0 + + gaussdbwriter + gaussdbwriter + 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} + + + + org.opengauss + opengauss-jdbc + 3.0.0 + + + + + + + + + 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/gaussdbwriter/src/main/assembly/package.xml b/gaussdbwriter/src/main/assembly/package.xml new file mode 100755 index 00000000..7167c89d --- /dev/null +++ b/gaussdbwriter/src/main/assembly/package.xml @@ -0,0 +1,35 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + plugin_job_template.json + + plugin/writer/gaussdbwriter + + + target/ + + gaussdbwriter-0.0.1-SNAPSHOT.jar + + plugin/writer/gaussdbwriter + + + + + + false + plugin/writer/gaussdbwriter/libs + runtime + + + diff --git a/gaussdbwriter/src/main/java/com/alibaba/datax/plugin/reader/gaussdbwriter/GaussDbWriter.java b/gaussdbwriter/src/main/java/com/alibaba/datax/plugin/reader/gaussdbwriter/GaussDbWriter.java new file mode 100644 index 00000000..3f758ee7 --- /dev/null +++ b/gaussdbwriter/src/main/java/com/alibaba/datax/plugin/reader/gaussdbwriter/GaussDbWriter.java @@ -0,0 +1,103 @@ +package com.alibaba.datax.plugin.reader.gaussdbwriter; + +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.DBUtilErrorCode; +import com.alibaba.datax.plugin.rdbms.util.DataBaseType; +import com.alibaba.datax.plugin.rdbms.writer.CommonRdbmsWriter; +import com.alibaba.datax.plugin.rdbms.writer.Key; + +import java.util.List; + +public class GaussDbWriter extends Writer { + + private static final DataBaseType DATABASE_TYPE = DataBaseType.GaussDB; + + public static class Job extends Writer.Job { + private Configuration originalConfig = null; + private CommonRdbmsWriter.Job commonRdbmsWriterMaster; + + @Override + public void init() { + this.originalConfig = super.getPluginJobConf(); + + // warn:not like mysql, GaussDB only support insert mode, don't use + String writeMode = this.originalConfig.getString(Key.WRITE_MODE); + if (null != writeMode) { + throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, + String.format("写入模式(writeMode)配置有误. 因为GaussDB不支持配置参数项 writeMode: %s, GaussDB仅使用insert sql 插入数据. 请检查您的配置并作出修改.", writeMode)); + } + + this.commonRdbmsWriterMaster = new CommonRdbmsWriter.Job(DATABASE_TYPE); + this.commonRdbmsWriterMaster.init(this.originalConfig); + } + + @Override + public void prepare() { + this.commonRdbmsWriterMaster.prepare(this.originalConfig); + } + + @Override + public List split(int mandatoryNumber) { + return this.commonRdbmsWriterMaster.split(this.originalConfig, mandatoryNumber); + } + + @Override + public void post() { + this.commonRdbmsWriterMaster.post(this.originalConfig); + } + + @Override + public void destroy() { + this.commonRdbmsWriterMaster.destroy(this.originalConfig); + } + + } + + public static class Task extends Writer.Task { + private Configuration writerSliceConfig; + private CommonRdbmsWriter.Task commonRdbmsWriterSlave; + + @Override + public void init() { + this.writerSliceConfig = super.getPluginJobConf(); + this.commonRdbmsWriterSlave = new CommonRdbmsWriter.Task(DATABASE_TYPE){ + @Override + public String calcValueHolder(String columnType){ + if("serial".equalsIgnoreCase(columnType)){ + return "?::int"; + }else if("bigserial".equalsIgnoreCase(columnType)){ + return "?::int8"; + }else if("bit".equalsIgnoreCase(columnType)){ + return "?::bit varying"; + } + return "?::" + columnType; + } + }; + this.commonRdbmsWriterSlave.init(this.writerSliceConfig); + } + + @Override + public void prepare() { + this.commonRdbmsWriterSlave.prepare(this.writerSliceConfig); + } + + public void startWrite(RecordReceiver recordReceiver) { + this.commonRdbmsWriterSlave.startWrite(recordReceiver, this.writerSliceConfig, super.getTaskPluginCollector()); + } + + @Override + public void post() { + this.commonRdbmsWriterSlave.post(this.writerSliceConfig); + } + + @Override + public void destroy() { + this.commonRdbmsWriterSlave.destroy(this.writerSliceConfig); + } + + } + +} diff --git a/gaussdbwriter/src/main/resources/plugin.json b/gaussdbwriter/src/main/resources/plugin.json new file mode 100755 index 00000000..2f52a167 --- /dev/null +++ b/gaussdbwriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "gaussdbwriter", + "class": "com.alibaba.datax.plugin.writer.gaussdbwriter.GaussDbWriter", + "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": "alibaba" +} \ No newline at end of file diff --git a/gaussdbwriter/src/main/resources/plugin_job_template.json b/gaussdbwriter/src/main/resources/plugin_job_template.json new file mode 100644 index 00000000..539fa46f --- /dev/null +++ b/gaussdbwriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,16 @@ +{ + "name": "gaussdbwriter", + "parameter": { + "username": "", + "password": "", + "column": [], + "connection": [ + { + "jdbcUrl": "", + "table": [] + } + ], + "preSql": [], + "postSql": [] + } +} \ No newline at end of file diff --git a/hbase11xsqlreader/doc/hbase11xsqlreader.md b/hbase11xsqlreader/doc/hbase11xsqlreader.md index 03261a1f..9f70077f 100644 --- a/hbase11xsqlreader/doc/hbase11xsqlreader.md +++ b/hbase11xsqlreader/doc/hbase11xsqlreader.md @@ -60,12 +60,16 @@ hbase11xsqlreader插件实现了从Phoenix(HBase SQL)读取数据。在底层实 //填写连接Phoenix的hbase集群zk地址 "hbaseConfig": { "hbase.zookeeper.quorum": "hb-proxy-xxx-002.hbase.rds.aliyuncs.com,hb-proxy-xxx-001.hbase.rds.aliyuncs.com,hb-proxy-xxx-003.hbase.rds.aliyuncs.com" - }, + }, + //填写要读取的phoenix的命名空间 + "schema": "TAG", //填写要读取的phoenix的表名 "table": "US_POPULATION", //填写要读取的列名,不填读取所有列 "column": [ - ] + ], + //查询条件 + "where": "id=" } }, "writer": { @@ -92,11 +96,18 @@ hbase11xsqlreader插件实现了从Phoenix(HBase SQL)读取数据。在底层实 * 必选:是
+ * 默认值:无
+* **schema** + + * 描述:编写Phoenix中的namespace,该值设置为'' + + * 必选:是
+ * 默认值:无
* **table** - * 描述:编写Phoenix中的表名,如果有namespace,该值设置为'namespace.tablename' + * 描述:编写Phoenix中的表名,该值设置为'tablename' * 必选:是
@@ -109,7 +120,13 @@ hbase11xsqlreader插件实现了从Phoenix(HBase SQL)读取数据。在底层实 * 必选:是
* 默认值:无
+* **where** + + * 描述:填写需要从phoenix表中读取条件判断。 + * 可选:是
+ + * 默认值:无
### 3.3 类型转换 @@ -172,11 +189,14 @@ hbase11xsqlreader插件实现了从Phoenix(HBase SQL)读取数据。在底层实 "hbaseConfig": { "hbase.zookeeper.quorum": "hb-proxy-xxx-002.hbase.rds.aliyuncs.com,hb-proxy-xxx-001.hbase.rds.aliyuncs.com,hb-proxy-xxx-003.hbase.rds.aliyuncs.com" }, + "schema": "TAG", //填写要读取的phoenix的表名 "table": "US_POPULATION", //填写要读取的列名,不填读取所有列 "column": [ - ] + ], + //查询条件 + "where": "id=" } }, "writer": { @@ -204,7 +224,13 @@ hbase11xsqlreader插件实现了从Phoenix(HBase SQL)读取数据。在底层实 * 必选:是
* 默认值:无
- +* **schema** + + * 描述:编写Phoenix中的namespace,该值设置为'' + + * 必选:是
+ + * 默认值:无
* **table** * 描述:编写Phoenix中的表名,如果有namespace,该值设置为'namespace.tablename' @@ -220,7 +246,13 @@ hbase11xsqlreader插件实现了从Phoenix(HBase SQL)读取数据。在底层实 * 必选:是
* 默认值:无
+ * **where** + * 描述:填写需要从phoenix表中读取条件判断。 + + * 可选:是
+ + * 默认值:无
### 3.3 类型转换 diff --git a/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLHelper.java b/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLHelper.java index 71665a6b..cf4304ee 100644 --- a/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLHelper.java +++ b/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLHelper.java @@ -26,9 +26,7 @@ import java.io.IOException; import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; +import java.util.*; public class HbaseSQLHelper { @@ -50,11 +48,15 @@ public class HbaseSQLHelper { String zkUrl = readerConfig.getZkUrl(); PhoenixConfigurationUtil.setInputClass(conf, PhoenixRecordWritable.class); - PhoenixConfigurationUtil.setInputTableName(conf, table); + + PhoenixConfigurationUtil.setInputTableName(conf, readerConfig.getSchema()+"."+table); if (!columns.isEmpty()) { PhoenixConfigurationUtil.setSelectColumnNames(conf, columns.toArray(new String[columns.size()])); } + if(Objects.nonNull(readerConfig.getWhere())){ + PhoenixConfigurationUtil.setInputTableConditions(conf,readerConfig.getWhere()); + } PhoenixEmbeddedDriver.ConnectionInfo info = null; try { info = PhoenixEmbeddedDriver.ConnectionInfo.create(zkUrl); @@ -67,15 +69,19 @@ public class HbaseSQLHelper { conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, info.getPort()); if (info.getRootNode() != null) conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, info.getRootNode()); + conf.set(Key.NAME_SPACE_MAPPING_ENABLED,"true"); + conf.set(Key.SYSTEM_TABLES_TO_NAMESPACE,"true"); return conf; } - public static List getPColumnNames(String connectionString, String tableName) throws SQLException { - Connection con = - DriverManager.getConnection(connectionString); + public static List getPColumnNames(String connectionString, String tableName,String schema) throws SQLException { + Properties pro = new Properties(); + pro.put(Key.NAME_SPACE_MAPPING_ENABLED, true); + pro.put(Key.SYSTEM_TABLES_TO_NAMESPACE, true); + Connection con = DriverManager.getConnection(connectionString,pro); PhoenixConnection phoenixConnection = con.unwrap(PhoenixConnection.class); MetaDataClient metaDataClient = new MetaDataClient(phoenixConnection); - PTable table = metaDataClient.updateCache("", tableName).getTable(); + PTable table = metaDataClient.updateCache(schema, tableName).getTable(); List columnNames = new ArrayList(); for (PColumn pColumn : table.getColumns()) { if (!pColumn.getName().getString().equals(SaltingUtil.SALTING_COLUMN_NAME)) diff --git a/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLReaderConfig.java b/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLReaderConfig.java index ab06f6e1..37060986 100644 --- a/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLReaderConfig.java +++ b/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLReaderConfig.java @@ -9,6 +9,7 @@ import org.slf4j.LoggerFactory; import java.sql.SQLException; import java.util.List; +import java.util.StringJoiner; public class HbaseSQLReaderConfig { private final static Logger LOG = LoggerFactory.getLogger(HbaseSQLReaderConfig.class); @@ -27,6 +28,9 @@ public class HbaseSQLReaderConfig { private String tableName; private List columns; // 目的表的所有列的列名,包括主键和非主键,不包括时间列 + private String where;//条件 + + private String schema;// /** * @return 获取原始的datax配置 */ @@ -96,22 +100,27 @@ public class HbaseSQLReaderConfig { } String zkQuorum = zkCfg.getFirst(); String znode = zkCfg.getSecond(); + if (zkQuorum == null || zkQuorum.isEmpty()) { throw DataXException.asDataXException( HbaseSQLReaderErrorCode.ILLEGAL_VALUE, "HBase的hbase.zookeeper.quorum配置不能为空" ); } // 生成sql使用的连接字符串, 格式: jdbc:hbase:zk_quorum:2181:/znode_parent - cfg.connectionString = "jdbc:phoenix:" + zkQuorum; - cfg.zkUrl = zkQuorum + ":2181"; + StringBuilder connectionString=new StringBuilder("jdbc:phoenix:"); + connectionString.append(zkQuorum); + cfg.connectionString = connectionString.toString(); + StringBuilder zkUrl =new StringBuilder(zkQuorum); + cfg.zkUrl = zkUrl.append(":2181").toString(); if (!znode.isEmpty()) { - cfg.connectionString += cfg.connectionString + ":" + znode; - cfg.zkUrl += cfg.zkUrl + ":" + znode; + cfg.connectionString = connectionString.append(":").append(znode).toString(); + cfg.zkUrl=zkUrl.append(":").append(znode).toString(); } } private static void parseTableConfig(HbaseSQLReaderConfig cfg, Configuration dataxCfg) { // 解析并检查表名 cfg.tableName = dataxCfg.getString(Key.TABLE); + cfg.schema = dataxCfg.getString(Key.SCHEMA); if (cfg.tableName == null || cfg.tableName.isEmpty()) { throw DataXException.asDataXException( HbaseSQLReaderErrorCode.ILLEGAL_VALUE, "HBase的tableName配置不能为空,请检查并修改配置." ); @@ -124,13 +133,14 @@ public class HbaseSQLReaderConfig { HbaseSQLReaderErrorCode.ILLEGAL_VALUE, "您配置的tableName含有非法字符{0},请检查您的配置."); } else if (cfg.columns.isEmpty()) { try { - cfg.columns = HbaseSQLHelper.getPColumnNames(cfg.connectionString, cfg.tableName); + cfg.columns = HbaseSQLHelper.getPColumnNames(cfg.connectionString, cfg.tableName,cfg.schema); dataxCfg.set(Key.COLUMN, cfg.columns); } catch (SQLException e) { throw DataXException.asDataXException( HbaseSQLReaderErrorCode.GET_PHOENIX_COLUMN_ERROR, "HBase的columns配置不能为空,请添加目标表的列名配置." + e.getMessage(), e); } } + cfg.where=dataxCfg.getString(Key.WHERE); } @Override @@ -151,6 +161,8 @@ public class HbaseSQLReaderConfig { ret.append(","); } ret.setLength(ret.length() - 1); + ret.append("[where=]").append(getWhere()); + ret.append("[schema=]").append(getSchema()); ret.append("\n"); return ret.toString(); @@ -161,4 +173,20 @@ public class HbaseSQLReaderConfig { */ private HbaseSQLReaderConfig() { } + + public String getWhere() { + return where; + } + + public void setWhere(String where) { + this.where = where; + } + + public String getSchema() { + return schema; + } + + public void setSchema(String schema) { + this.schema = schema; + } } diff --git a/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLReaderTask.java b/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLReaderTask.java index 1ca22c6f..461649d1 100644 --- a/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLReaderTask.java +++ b/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/HbaseSQLReaderTask.java @@ -19,10 +19,8 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.math.BigDecimal; import java.sql.*; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; +import java.sql.Date; +import java.util.*; /** * Created by admin on 1/3/18. @@ -42,11 +40,14 @@ public class HbaseSQLReaderTask { } private void getPColumns() throws SQLException { + Properties pro = new Properties(); + pro.put(Key.NAME_SPACE_MAPPING_ENABLED, true); + pro.put(Key.SYSTEM_TABLES_TO_NAMESPACE, true); Connection con = - DriverManager.getConnection(this.readerConfig.getConnectionString()); + DriverManager.getConnection(this.readerConfig.getConnectionString(),pro); PhoenixConnection phoenixConnection = con.unwrap(PhoenixConnection.class); MetaDataClient metaDataClient = new MetaDataClient(phoenixConnection); - PTable table = metaDataClient.updateCache("", this.readerConfig.getTableName()).getTable(); + PTable table = metaDataClient.updateCache(this.readerConfig.getSchema(), this.readerConfig.getTableName()).getTable(); List columnNames = this.readerConfig.getColumns(); for (PColumn pColumn : table.getColumns()) { if (columnNames.contains(pColumn.getName().getString())) { diff --git a/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/Key.java b/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/Key.java index 7987d6c8..f8453add 100644 --- a/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/Key.java +++ b/hbase11xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase11xsqlreader/Key.java @@ -24,5 +24,18 @@ public final class Key { * 【必选】列配置 */ public final static String COLUMN = "column"; + /** + * + */ + public static final String WHERE = "where"; + + /** + * 【可选】Phoenix表所属schema,默认为空 + */ + public static final String SCHEMA = "schema"; + + public static final String NAME_SPACE_MAPPING_ENABLED = "phoenix.schema.isNamespaceMappingEnabled"; + + public static final String SYSTEM_TABLES_TO_NAMESPACE = "phoenix.schema.mapSystemTablesToNamespace"; } diff --git a/hdfsreader/pom.xml b/hdfsreader/pom.xml index a5c2da2c..de7c0e21 100644 --- a/hdfsreader/pom.xml +++ b/hdfsreader/pom.xml @@ -1,5 +1,6 @@ - + datax-all com.alibaba.datax @@ -111,6 +112,42 @@ ${datax-project-version}
+ + org.apache.parquet + parquet-column + 1.12.0 + + + org.apache.parquet + parquet-avro + 1.12.0 + + + org.apache.parquet + parquet-common + 1.12.0 + + + org.apache.parquet + parquet-format + 2.3.0 + + + org.apache.parquet + parquet-jackson + 1.12.0 + + + org.apache.parquet + parquet-encoding + 1.12.0 + + + org.apache.parquet + parquet-hadoop + 1.12.0 + + diff --git a/hdfsreader/src/main/assembly/package.xml b/hdfsreader/src/main/assembly/package.xml index 3f1393b7..a5f28e5c 100644 --- a/hdfsreader/src/main/assembly/package.xml +++ b/hdfsreader/src/main/assembly/package.xml @@ -37,6 +37,28 @@ + + + + + + + + + + src/main/libs + + *.* + + plugin/reader/ossreader/libs + + + src/main/libs + + *.* + + plugin/reader/hivereader/libs + diff --git a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/Constant.java b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/Constant.java index 6bfb9bf7..061c55a0 100644 --- a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/Constant.java +++ b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/Constant.java @@ -10,4 +10,5 @@ public class Constant { public static final String CSV = "CSV"; public static final String SEQ = "SEQ"; public static final String RC = "RC"; + public static final String PARQUET = "PARQUET"; } diff --git a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/DFSUtil.java b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/DFSUtil.java index 5ba572e1..720f8bf6 100644 --- a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/DFSUtil.java +++ b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/DFSUtil.java @@ -9,12 +9,16 @@ import com.alibaba.datax.plugin.unstructuredstorage.reader.ColumnEntry; import com.alibaba.datax.plugin.unstructuredstorage.reader.UnstructuredStorageReaderErrorCode; import com.alibaba.datax.plugin.unstructuredstorage.reader.UnstructuredStorageReaderUtil; import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONArray; import com.alibaba.fastjson2.JSONObject; +import org.apache.commons.lang3.BooleanUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.ql.io.RCFile; import org.apache.hadoop.hive.ql.io.RCFileRecordReader; import org.apache.hadoop.hive.ql.io.orc.OrcFile; @@ -29,14 +33,30 @@ import org.apache.hadoop.io.*; import org.apache.hadoop.mapred.*; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ReflectionUtils; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.example.GroupReadSupport; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.apache.parquet.schema.PrimitiveType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; import java.text.SimpleDateFormat; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * Created by mingya.wmy on 2015/8/12. @@ -56,6 +76,10 @@ public class DFSUtil { public static final String HDFS_DEFAULTFS_KEY = "fs.defaultFS"; public static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication"; + private Boolean skipEmptyOrcFile = false; + + private Integer orcFileEmptySize = null; + public DFSUtil(Configuration taskConfig) { hadoopConf = new org.apache.hadoop.conf.Configuration(); @@ -79,6 +103,7 @@ public class DFSUtil { this.hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, "kerberos"); } this.kerberosAuthentication(this.kerberosPrincipal, this.kerberosKeytabFilePath); + this.skipEmptyOrcFile = taskConfig.getBool(Key.SKIP_EMPTY_ORCFILE, false); LOG.info(String.format("hadoopConfig details:%s", JSON.toJSONString(this.hadoopConf))); } @@ -102,10 +127,11 @@ public class DFSUtil { * @param srcPaths 路径列表 * @param specifiedFileType 指定文件类型 */ - public HashSet getAllFiles(List srcPaths, String specifiedFileType) { + public HashSet getAllFiles(List srcPaths, String specifiedFileType, Boolean skipEmptyOrcFile, Integer orcFileEmptySize) { this.specifiedFileType = specifiedFileType; - + this.skipEmptyOrcFile = skipEmptyOrcFile; + this.orcFileEmptySize = orcFileEmptySize; if (!srcPaths.isEmpty()) { for (String eachPath : srcPaths) { LOG.info(String.format("get HDFS all files in path = [%s]", eachPath)); @@ -127,9 +153,13 @@ public class DFSUtil { FileStatus stats[] = hdfs.globStatus(path); for (FileStatus f : stats) { if (f.isFile()) { - if (f.getLen() == 0) { + long fileLength = f.getLen(); + if (fileLength == 0) { String message = String.format("文件[%s]长度为0,将会跳过不作处理!", hdfsPath); LOG.warn(message); + } else if (BooleanUtils.isTrue(this.skipEmptyOrcFile) && this.orcFileEmptySize != null && fileLength <= this.orcFileEmptySize) { + String message = String.format("The orc file [%s] is empty, file size: %s, DataX will skip it !", f.getPath().toString(), fileLength); + LOG.warn(message); } else { addSourceFileByType(f.getPath().toString()); } @@ -167,7 +197,16 @@ public class DFSUtil { LOG.info(String.format("[%s] 是目录, 递归获取该目录下的文件", f.getPath().toString())); getHDFSAllFilesNORegex(f.getPath().toString(), hdfs); } else if (f.isFile()) { - + long fileLength = f.getLen(); + if (fileLength == 0) { + String message = String.format("The file [%s] is empty, DataX will skip it !", f.getPath().toString()); + LOG.warn(message); + continue; + } else if (BooleanUtils.isTrue(this.skipEmptyOrcFile) && this.orcFileEmptySize != null && fileLength <= this.orcFileEmptySize) { + String message = String.format("The orc file [%s] is empty, file size: %s, DataX will skip it !", f.getPath().toString(), fileLength); + LOG.warn(message); + continue; + } addSourceFileByType(f.getPath().toString()); } else { String message = String.format("该路径[%s]文件类型既不是目录也不是文件,插件自动忽略。", @@ -332,7 +371,19 @@ public class DFSUtil { //Each file as a split //TODO multy threads // OrcInputFormat getSplits params numSplits not used, splits size = block numbers - InputSplit[] splits = in.getSplits(conf, -1); + InputSplit[] splits; + try { + splits = in.getSplits(conf, 1); + } catch (Exception splitException) { + if (Boolean.TRUE.equals(this.skipEmptyOrcFile)) { + boolean isOrcFileEmptyException = checkIsOrcEmptyFileExecption(splitException); + if (isOrcFileEmptyException) { + LOG.info("skipEmptyOrcFile: true, \"{}\" is an empty orc file, skip it!", sourceOrcFilePath); + return; + } + } + throw splitException; + } for (InputSplit split : splits) { { RecordReader reader = in.getRecordReader(split, conf, Reporter.NULL); @@ -349,8 +400,11 @@ public class DFSUtil { Object field = inspector.getStructFieldData(value, fields.get(i)); recordFields.add(field); } + List hivePartitionColumnEntrys = UnstructuredStorageReaderUtil.getListColumnEntry(readerSliceConfig, com.alibaba.datax.plugin.unstructuredstorage.reader.Key.HIVE_PARTION_COLUMN); + ArrayList hivePartitionColumns = new ArrayList<>(); + hivePartitionColumns = UnstructuredStorageReaderUtil.getHivePartitionColumns(sourceOrcFilePath, hivePartitionColumnEntrys); transportOneRecord(column, recordFields, recordSender, - taskPluginCollector, isReadAllColumns, nullFormat); + taskPluginCollector, isReadAllColumns, nullFormat,hivePartitionColumns); } reader.close(); } @@ -367,8 +421,20 @@ public class DFSUtil { } } + private boolean checkIsOrcEmptyFileExecption(Exception e) { + if (e == null) { + return false; + } + + String fullStackTrace = ExceptionUtils.getStackTrace(e); + if (fullStackTrace.contains("org.apache.orc.impl.ReaderImpl.getRawDataSizeOfColumn") && fullStackTrace.contains("Caused by: java.lang.IndexOutOfBoundsException: Index: 1, Size: 1")) { + return true; + } + return false; + } + private Record transportOneRecord(List columnConfigs, List recordFields - , RecordSender recordSender, TaskPluginCollector taskPluginCollector, boolean isReadAllColumns, String nullFormat) { + , RecordSender recordSender, TaskPluginCollector taskPluginCollector, boolean isReadAllColumns, String nullFormat, ArrayList hiveParitionColumns) { Record record = recordSender.createRecord(); Column columnGenerated; try { @@ -555,8 +621,9 @@ public class DFSUtil { } else if (StringUtils.equalsIgnoreCase(specifiedFileType, Constant.SEQ)) { return isSequenceFile(filepath, in); + } else if (StringUtils.equalsIgnoreCase(specifiedFileType, Constant.PARQUET)) { + return true; } - } catch (Exception e) { String message = String.format("检查文件[%s]类型失败,目前支持ORC,SEQUENCE,RCFile,TEXT,CSV五种格式的文件," + "请检查您文件类型和文件是否正确。", filepath); @@ -693,4 +760,332 @@ public class DFSUtil { return false; } + public void parquetFileStartRead(String sourceParquetFilePath, Configuration readerSliceConfig, RecordSender recordSender, TaskPluginCollector taskPluginCollector) { + String schemaString = readerSliceConfig.getString(Key.PARQUET_SCHEMA); + if (StringUtils.isNotBlank(schemaString)) { + LOG.info("You config parquet schema, use it {}", schemaString); + } else { + schemaString = getParquetSchema(sourceParquetFilePath, hadoopConf); + LOG.info("Parquet schema parsed from: {} , schema is {}", sourceParquetFilePath, schemaString); + if (StringUtils.isBlank(schemaString)) { + throw DataXException.asDataXException("ParquetSchema is required, please check your config"); + } + } + MessageType parquetSchema = null; + List parquetTypes = null; + Map parquetMetaMap = null; + int fieldCount = 0; + try { + parquetSchema = MessageTypeParser.parseMessageType(schemaString); + fieldCount = parquetSchema.getFieldCount(); + parquetTypes = parquetSchema.getFields(); + parquetMetaMap = ParquetMessageHelper.parseParquetTypes(parquetTypes); + } catch (Exception e) { + String message = String.format("Error parsing to MessageType via Schema string [%s]", schemaString); + LOG.error(message); + throw DataXException.asDataXException(HdfsReaderErrorCode.PARSE_MESSAGE_TYPE_FROM_SCHEMA_ERROR, e); + } + List column = UnstructuredStorageReaderUtil.getListColumnEntry(readerSliceConfig, com.alibaba.datax.plugin.unstructuredstorage.reader.Key.COLUMN); + String nullFormat = readerSliceConfig.getString(com.alibaba.datax.plugin.unstructuredstorage.reader.Key.NULL_FORMAT); + boolean isUtcTimestamp = readerSliceConfig.getBool(Key.PARQUET_UTC_TIMESTAMP, false); + boolean isReadAllColumns = (column == null || column.size() == 0) ? true : false; + LOG.info("ReadingAllColums: " + isReadAllColumns); + + /** + * 支持 hive 表中间加列场景 + * + * 开关默认 false,在 hive表存在中间加列的场景打开,需要根据 name排序 + * 不默认打开的原因 + * 1、存量hdfs任务,只根据 index获取字段,无name字段配置 + * 2、中间加列场景比较少 + * 3、存量任务可能存在列错位的问题,不能随意纠正 + */ + boolean supportAddMiddleColumn = readerSliceConfig.getBool(Key.SUPPORT_ADD_MIDDLE_COLUMN, false); + + boolean printNullValueException = readerSliceConfig.getBool("printNullValueException", false); + List ignoreIndex = readerSliceConfig.getList("ignoreIndex", new ArrayList(), Integer.class); + + JobConf conf = new JobConf(hadoopConf); + ParquetReader reader = null; + try { + Path parquetFilePath = new Path(sourceParquetFilePath); + GroupReadSupport readSupport = new GroupReadSupport(); + readSupport.init(conf, null, parquetSchema); + // 这里初始化parquetReader的时候,会getFileSystem,如果是HA集群,期间会根据hadoopConfig中区加载failover类,这里初始化builder带上conf + ParquetReader.Builder parquetReaderBuilder = ParquetReader.builder(readSupport, parquetFilePath); + parquetReaderBuilder.withConf(hadoopConf); + reader = parquetReaderBuilder.build(); + Group g = null; + + // 从文件名中解析分区信息 + List hivePartitionColumnEntrys = UnstructuredStorageReaderUtil.getListColumnEntry(readerSliceConfig, com.alibaba.datax.plugin.unstructuredstorage.reader.Key.HIVE_PARTION_COLUMN); + ArrayList hivePartitionColumns = new ArrayList<>(); + hivePartitionColumns = UnstructuredStorageReaderUtil.getHivePartitionColumns(sourceParquetFilePath, hivePartitionColumnEntrys); + List schemaFieldList = null; + Map colNameIndexMap = null; + Map indexMap = null; + if (supportAddMiddleColumn) { + boolean nonName = column.stream().anyMatch(columnEntry -> StringUtils.isEmpty(columnEntry.getName())); + if (nonName) { + throw new DataXException("You configured column item without name, please correct it"); + } + List parquetFileFields = getParquetFileFields(parquetFilePath, hadoopConf); + schemaFieldList = parquetFileFields.stream().map(org.apache.parquet.schema.Type::getName).collect(Collectors.toList()); + colNameIndexMap = new ConcurrentHashMap<>(); + Map finalColNameIndexMap = colNameIndexMap; + column.forEach(columnEntry -> finalColNameIndexMap.put(columnEntry.getIndex(), columnEntry.getName())); + Iterator> iterator = finalColNameIndexMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry next = iterator.next(); + if (!schemaFieldList.contains(next.getValue())) { + finalColNameIndexMap.remove((next.getKey())); + } + } + LOG.info("SupportAddMiddleColumn is true, fields from parquet file is {}, " + + "colNameIndexMap is {}", JSON.toJSONString(schemaFieldList), JSON.toJSONString(colNameIndexMap)); + fieldCount = column.size(); + indexMap = new HashMap<>(); + for (int j = 0; j < fieldCount; j++) { + if (colNameIndexMap.containsKey(j)) { + int index = findIndex(schemaFieldList, findEleInMap(colNameIndexMap, j)); + indexMap.put(j, index); + } + } + } + while ((g = reader.read()) != null) { + List formattedRecord = new ArrayList(fieldCount); + try { + for (int j = 0; j < fieldCount; j++) { + Object data = null; + try { + if (null != ignoreIndex && !ignoreIndex.isEmpty() && ignoreIndex.contains(j)) { + data = null; + } else { + if (supportAddMiddleColumn) { + if (!colNameIndexMap.containsKey(j)) { + formattedRecord.add(null); + continue; + } else { + data = DFSUtil.this.readFields(g, parquetTypes.get(indexMap.get(j)), indexMap.get(j), parquetMetaMap, isUtcTimestamp); + } + } else { + data = DFSUtil.this.readFields(g, parquetTypes.get(j), j, parquetMetaMap, isUtcTimestamp); + } + } + } catch (RuntimeException e) { + if (printNullValueException) { + LOG.warn(e.getMessage()); + } + } + formattedRecord.add(data); + } + transportOneRecord(column, formattedRecord, recordSender, taskPluginCollector, isReadAllColumns, nullFormat, hivePartitionColumns); + } catch (Exception e) { + throw DataXException.asDataXException(HdfsReaderErrorCode.READ_PARQUET_ERROR, e); + } + } + } catch (Exception e) { + throw DataXException.asDataXException(HdfsReaderErrorCode.READ_PARQUET_ERROR, e); + } finally { + org.apache.commons.io.IOUtils.closeQuietly(reader); + } + } + + private String findEleInMap(Map map, Integer key) { + Iterator> iterator = map.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry next = iterator.next(); + if (key.equals(next.getKey())) { + return next.getValue(); + } + } + return null; + } + + private int findIndex(List schemaFieldList, String colName) { + for (int i = 0; i < schemaFieldList.size(); i++) { + if (schemaFieldList.get(i).equals(colName)) { + return i; + } + } + return -1; + } + + private List getParquetFileFields(Path filePath, org.apache.hadoop.conf.Configuration configuration) { + try (org.apache.parquet.hadoop.ParquetFileReader reader = org.apache.parquet.hadoop.ParquetFileReader.open(HadoopInputFile.fromPath(filePath, configuration))) { + org.apache.parquet.schema.MessageType schema = reader.getFooter().getFileMetaData().getSchema(); + List fields = schema.getFields(); + return fields; + } catch (IOException e) { + LOG.error("Fetch parquet field error", e); + throw new DataXException(String.format("Fetch parquet field error, msg is %s", e.getMessage())); + } + } + + private String getParquetSchema(String sourceParquetFilePath, org.apache.hadoop.conf.Configuration hadoopConf) { + GroupReadSupport readSupport = new GroupReadSupport(); + ParquetReader.Builder parquetReaderBuilder = ParquetReader.builder(readSupport, new Path(sourceParquetFilePath)); + ParquetReader reader = null; + try { + parquetReaderBuilder.withConf(hadoopConf); + reader = parquetReaderBuilder.build(); + Group g = null; + if ((g = reader.read()) != null) { + return g.getType().toString(); + } + } catch (Throwable e) { + LOG.error("Inner error, getParquetSchema failed, message is {}", e.getMessage()); + } finally { + org.apache.commons.io.IOUtils.closeQuietly(reader); + } + return null; + } + + /** + * parquet 相关 + */ + private static final int JULIAN_EPOCH_OFFSET_DAYS = 2440588; + private static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1); + private static final long NANOS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1); + + private long julianDayToMillis(int julianDay) { + return (julianDay - JULIAN_EPOCH_OFFSET_DAYS) * MILLIS_IN_DAY; + } + + private org.apache.parquet.schema.OriginalType getOriginalType(org.apache.parquet.schema.Type type, Map parquetMetaMap) { + ParquetMeta meta = parquetMetaMap.get(type.getName()); + return meta.getOriginalType(); + } + + private org.apache.parquet.schema.PrimitiveType asPrimitiveType(org.apache.parquet.schema.Type type, Map parquetMetaMap) { + ParquetMeta meta = parquetMetaMap.get(type.getName()); + return meta.getPrimitiveType(); + } + + private Object readFields(Group g, org.apache.parquet.schema.Type type, int index, Map parquetMetaMap, boolean isUtcTimestamp) { + if (this.getOriginalType(type, parquetMetaMap) == org.apache.parquet.schema.OriginalType.MAP) { + Group groupData = g.getGroup(index, 0); + List parquetTypes = groupData.getType().getFields(); + JSONObject data = new JSONObject(); + for (int i = 0; i < parquetTypes.size(); i++) { + int j = groupData.getFieldRepetitionCount(i); + // map key value 的对数 + for (int k = 0; k < j; k++) { + Group groupDataK = groupData.getGroup(0, k); + List parquetTypesK = groupDataK.getType().getFields(); + if (2 != parquetTypesK.size()) { + // warn: 不是key value成对出现 + throw new RuntimeException(String.format("bad parquet map type: %s", groupData.getValueToString(index, 0))); + } + Object subDataKey = this.readFields(groupDataK, parquetTypesK.get(0), 0, parquetMetaMap, isUtcTimestamp); + Object subDataValue = this.readFields(groupDataK, parquetTypesK.get(1), 1, parquetMetaMap, isUtcTimestamp); + if (StringUtils.equalsIgnoreCase("key", parquetTypesK.get(0).getName())) { + ((JSONObject) data).put(subDataKey.toString(), subDataValue); + } else { + ((JSONObject) data).put(subDataValue.toString(), subDataKey); + } + } + } + return data; + } else if (this.getOriginalType(type, parquetMetaMap) == org.apache.parquet.schema.OriginalType.MAP_KEY_VALUE) { + Group groupData = g.getGroup(index, 0); + List parquetTypes = groupData.getType().getFields(); + JSONObject data = new JSONObject(); + for (int i = 0; i < parquetTypes.size(); i++) { + int j = groupData.getFieldRepetitionCount(i); + // map key value 的对数 + for (int k = 0; k < j; k++) { + Group groupDataK = groupData.getGroup(0, k); + List parquetTypesK = groupDataK.getType().getFields(); + if (2 != parquetTypesK.size()) { + // warn: 不是key value成对出现 + throw new RuntimeException(String.format("bad parquet map type: %s", groupData.getValueToString(index, 0))); + } + Object subDataKey = this.readFields(groupDataK, parquetTypesK.get(0), 0, parquetMetaMap, isUtcTimestamp); + Object subDataValue = this.readFields(groupDataK, parquetTypesK.get(1), 1, parquetMetaMap, isUtcTimestamp); + if (StringUtils.equalsIgnoreCase("key", parquetTypesK.get(0).getName())) { + ((JSONObject) data).put(subDataKey.toString(), subDataValue); + } else { + ((JSONObject) data).put(subDataValue.toString(), subDataKey); + } + } + } + return data; + } else if (this.getOriginalType(type, parquetMetaMap) == org.apache.parquet.schema.OriginalType.LIST) { + Group groupData = g.getGroup(index, 0); + List parquetTypes = groupData.getType().getFields(); + JSONArray data = new JSONArray(); + for (int i = 0; i < parquetTypes.size(); i++) { + Object subData = this.readFields(groupData, parquetTypes.get(i), i, parquetMetaMap, isUtcTimestamp); + data.add(subData); + } + return data; + } else if (this.getOriginalType(type, parquetMetaMap) == org.apache.parquet.schema.OriginalType.DECIMAL) { + Binary binaryDate = g.getBinary(index, 0); + if (null == binaryDate) { + return null; + } else { + org.apache.hadoop.hive.serde2.io.HiveDecimalWritable decimalWritable = new org.apache.hadoop.hive.serde2.io.HiveDecimalWritable(binaryDate.getBytes(), this.asPrimitiveType(type, parquetMetaMap).getDecimalMetadata().getScale()); + // g.getType().getFields().get(1).asPrimitiveType().getDecimalMetadata().getScale() + HiveDecimal hiveDecimal = decimalWritable.getHiveDecimal(); + if (null == hiveDecimal) { + return null; + } else { + return hiveDecimal.bigDecimalValue(); + } + // return decimalWritable.doubleValue(); + } + } else if (this.getOriginalType(type, parquetMetaMap) == org.apache.parquet.schema.OriginalType.DATE) { + return java.sql.Date.valueOf(LocalDate.ofEpochDay(g.getInteger(index, 0))); + } else if (this.getOriginalType(type, parquetMetaMap) == org.apache.parquet.schema.OriginalType.UTF8) { + return g.getValueToString(index, 0); + } else { + if (type.isPrimitive()) { + PrimitiveType.PrimitiveTypeName primitiveTypeName = this.asPrimitiveType(type, parquetMetaMap).getPrimitiveTypeName(); + if (PrimitiveType.PrimitiveTypeName.BINARY == primitiveTypeName) { + return g.getValueToString(index, 0); + } else if (PrimitiveType.PrimitiveTypeName.BOOLEAN == primitiveTypeName) { + return g.getValueToString(index, 0); + } else if (PrimitiveType.PrimitiveTypeName.DOUBLE == primitiveTypeName) { + return g.getValueToString(index, 0); + } else if (PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY == primitiveTypeName) { + return g.getValueToString(index, 0); + } else if (PrimitiveType.PrimitiveTypeName.FLOAT == primitiveTypeName) { + return g.getValueToString(index, 0); + } else if (PrimitiveType.PrimitiveTypeName.INT32 == primitiveTypeName) { + return g.getValueToString(index, 0); + } else if (PrimitiveType.PrimitiveTypeName.INT64 == primitiveTypeName) { + return g.getValueToString(index, 0); + } else if (PrimitiveType.PrimitiveTypeName.INT96 == primitiveTypeName) { + Binary dataInt96 = g.getInt96(index, 0); + if (null == dataInt96) { + return null; + } else { + ByteBuffer buf = dataInt96.toByteBuffer(); + buf.order(ByteOrder.LITTLE_ENDIAN); + long timeOfDayNanos = buf.getLong(); + int julianDay = buf.getInt(); + if (isUtcTimestamp) { + // UTC + LocalDate localDate = LocalDate.ofEpochDay(julianDay - JULIAN_EPOCH_OFFSET_DAYS); + LocalTime localTime = LocalTime.ofNanoOfDay(timeOfDayNanos); + return Timestamp.valueOf(LocalDateTime.of(localDate, localTime)); + } else { + // local time + long mills = julianDayToMillis(julianDay) + (timeOfDayNanos / NANOS_PER_MILLISECOND); + Timestamp timestamp = new Timestamp(mills); + timestamp.setNanos((int) (timeOfDayNanos % TimeUnit.SECONDS.toNanos(1))); + return timestamp; + } + } + } else { + return g.getValueToString(index, 0); + } + } else { + return g.getValueToString(index, 0); + } + } + } + + } diff --git a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsPathFilter.java b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsPathFilter.java new file mode 100644 index 00000000..88dd1fa7 --- /dev/null +++ b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsPathFilter.java @@ -0,0 +1,21 @@ +package com.alibaba.datax.plugin.reader.hdfsreader; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +/** + * Created by wmy on 16/11/29. + */ +public class HdfsPathFilter implements PathFilter { + + private String regex = null; + + public HdfsPathFilter(String regex) { + this.regex = regex; + } + + @Override + public boolean accept(Path path) { + return regex != null ? path.getName().matches(regex) : true; + } +} diff --git a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsReader.java b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsReader.java index c953ef16..08c630fc 100644 --- a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsReader.java +++ b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsReader.java @@ -41,6 +41,8 @@ public class HdfsReader extends Reader { private String specifiedFileType = null; private DFSUtil dfsUtil = null; private List path = null; + private boolean skipEmptyOrcFile = false; + private Integer orcFileEmptySize = null; @Override public void init() { @@ -81,9 +83,10 @@ public class HdfsReader extends Reader { !specifiedFileType.equalsIgnoreCase(Constant.TEXT) && !specifiedFileType.equalsIgnoreCase(Constant.CSV) && !specifiedFileType.equalsIgnoreCase(Constant.SEQ) && - !specifiedFileType.equalsIgnoreCase(Constant.RC)){ - String message = "HdfsReader插件目前支持ORC, TEXT, CSV, SEQUENCE, RC五种格式的文件," + - "请将fileType选项的值配置为ORC, TEXT, CSV, SEQUENCE 或者 RC"; + !specifiedFileType.equalsIgnoreCase(Constant.RC) && + !specifiedFileType.equalsIgnoreCase(Constant.PARQUET)){ + String message = "HdfsReader插件目前支持ORC, TEXT, CSV, SEQUENCE, RC, PARQUET 六种格式的文件," + + "请将fileType选项的值配置为ORC, TEXT, CSV, SEQUENCE,RC 和 PARQUET"; throw DataXException.asDataXException(HdfsReaderErrorCode.FILE_TYPE_ERROR, message); } @@ -115,6 +118,16 @@ public class HdfsReader extends Reader { UnstructuredStorageReaderUtil.validateCompress(this.readerOriginConfig); UnstructuredStorageReaderUtil.validateCsvReaderConfig(this.readerOriginConfig); } + if (this.specifiedFileType.equalsIgnoreCase(Constant.ORC)) { + skipEmptyOrcFile = this.readerOriginConfig.getBool(Key.SKIP_EMPTY_ORCFILE, false); + orcFileEmptySize = this.readerOriginConfig.getInt(Key.ORCFILE_EMPTYSIZE); + //将orcFileEmptySize必填项检查去掉,仅需要配置skipEmptyOrcFile即可,考虑历史任务兼容性(For中华保险),保留orcFileEmptySize参数配置 + //if (skipEmptyOrcFile && orcFileEmptySize == null) { + // throw new IllegalArgumentException("When \"skipEmptyOrcFile\" is configured, " + // + "parameter \"orcFileEmptySize\" cannot be null."); + //} + } + LOG.info("skipEmptyOrcFile: {}, orcFileEmptySize: {}", skipEmptyOrcFile, orcFileEmptySize); } @@ -166,7 +179,7 @@ public class HdfsReader extends Reader { @Override public void prepare() { LOG.info("prepare(), start to getAllFiles..."); - this.sourceFiles = dfsUtil.getAllFiles(path, specifiedFileType); + this.sourceFiles = dfsUtil.getAllFiles(path, specifiedFileType,skipEmptyOrcFile, orcFileEmptySize); LOG.info(String.format("您即将读取的文件数为: [%s], 列表为: [%s]", this.sourceFiles.size(), StringUtils.join(this.sourceFiles, ","))); @@ -273,7 +286,9 @@ public class HdfsReader extends Reader { }else if(specifiedFileType.equalsIgnoreCase(Constant.RC)){ dfsUtil.rcFileStartRead(sourceFile, this.taskConfig, recordSender, this.getTaskPluginCollector()); - }else { + } else if (specifiedFileType.equalsIgnoreCase(Constant.PARQUET)) { + dfsUtil.parquetFileStartRead(sourceFile, this.taskConfig, recordSender, this.getTaskPluginCollector()); + } else { String message = "HdfsReader插件目前支持ORC, TEXT, CSV, SEQUENCE, RC五种格式的文件," + "请将fileType选项的值配置为ORC, TEXT, CSV, SEQUENCE 或者 RC"; diff --git a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsReaderErrorCode.java b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsReaderErrorCode.java index 8dd3f370..f2caa1a8 100644 --- a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsReaderErrorCode.java +++ b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/HdfsReaderErrorCode.java @@ -19,7 +19,12 @@ public enum HdfsReaderErrorCode implements ErrorCode { FILE_TYPE_UNSUPPORT("HdfsReader-12", "文件类型目前不支持"), KERBEROS_LOGIN_ERROR("HdfsReader-13", "KERBEROS认证失败"), READ_SEQUENCEFILE_ERROR("HdfsReader-14", "读取SequenceFile文件出错"), - READ_RCFILE_ERROR("HdfsReader-15", "读取RCFile文件出错"),; + READ_RCFILE_ERROR("HdfsReader-15", "读取RCFile文件出错"), + INIT_RCFILE_SERDE_ERROR("HdfsReader-16", "Deserialize RCFile, initialization failed!"), + PARSE_MESSAGE_TYPE_FROM_SCHEMA_ERROR("HdfsReader-17", "Error parsing ParquetSchema"), + INVALID_PARQUET_SCHEMA("HdfsReader-18", "ParquetSchema is invalid"), + READ_PARQUET_ERROR("HdfsReader-19", "Error reading Parquet file"), + CONNECT_HDFS_IO_ERROR("HdfsReader-20", "I/O exception in establishing connection with HDFS"); private final String code; private final String description; diff --git a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/Key.java b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/Key.java index 7b985a88..7f9b3a0a 100644 --- a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/Key.java +++ b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/Key.java @@ -7,9 +7,60 @@ public final class Key { */ public final static String PATH = "path"; public final static String DEFAULT_FS = "defaultFS"; + public final static String HIVE_VERSION = "hiveVersion"; public static final String FILETYPE = "fileType"; public static final String HADOOP_CONFIG = "hadoopConfig"; public static final String HAVE_KERBEROS = "haveKerberos"; public static final String KERBEROS_KEYTAB_FILE_PATH = "kerberosKeytabFilePath"; + public static final String KERBEROS_CONF_FILE_PATH = "kerberosConfFilePath"; public static final String KERBEROS_PRINCIPAL = "kerberosPrincipal"; + public static final String PATH_FILTER = "pathFilter"; + public static final String PARQUET_SCHEMA = "parquetSchema"; + /** + * hive 3.x 或 cdh高版本,使用UTC时区存储时间戳,如果发现时区偏移,该配置项要配置成 true + */ + public static final String PARQUET_UTC_TIMESTAMP = "parquetUtcTimestamp"; + public static final String SUCCESS_ON_NO_FILE = "successOnNoFile"; + public static final String PROTECTION = "protection"; + + /** + * 用于显示地指定hdfs客户端的用户名 + */ + public static final String HDFS_USERNAME = "hdfsUsername"; + + /** + * ORC FILE空文件大小 + */ + public static final String ORCFILE_EMPTYSIZE = "orcFileEmptySize"; + + /** + * 是否跳过空的OrcFile + */ + public static final String SKIP_EMPTY_ORCFILE = "skipEmptyOrcFile"; + + /** + * 是否跳过 orc meta 信息 + */ + + public static final String SKIP_ORC_META = "skipOrcMetaInfo"; + /** + * 过滤_或者.开头的文件 + */ + public static final String REGEX_PATTERN = "^.*[/][^._].*"; + + public static final String FILTER_TAG_FILE = "filterTagFile"; + + // high level params refs https://github.com/aliyun/alibabacloud-jindodata/blob/master/docs/user/4.x/4.4.0/oss/configuration/jindosdk_configuration_list.md + // + public static final String FS_OSS_DOWNLOAD_QUEUE_SIZE = "ossDownloadQueueSize"; + + // + public static final String FS_OSS_DOWNLOAD_THREAD_CONCURRENCY = "ossDownloadThreadConcurrency"; + + public static final String FS_OSS_READ_READAHEAD_BUFFER_COUNT = "ossDownloadBufferCount"; + + public static final String FILE_SYSTEM_TYPE = "fileSystemType"; + public static final String CDH_3_X_HIVE_VERSION = "3.1.3-cdh"; + + public static final String SUPPORT_ADD_MIDDLE_COLUMN = "supportAddMiddleColumn"; } diff --git a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/ParquetMessageHelper.java b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/ParquetMessageHelper.java new file mode 100644 index 00000000..e5838d6e --- /dev/null +++ b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/ParquetMessageHelper.java @@ -0,0 +1,33 @@ +package com.alibaba.datax.plugin.reader.hdfsreader; + +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * @author jitongchen + * @date 2023/9/7 10:20 AM + */ +public class ParquetMessageHelper { + public static Map parseParquetTypes(List parquetTypes) { + int fieldCount = parquetTypes.size(); + Map parquetMetaMap = new HashMap(); + for (int i = 0; i < fieldCount; i++) { + org.apache.parquet.schema.Type type = parquetTypes.get(i); + String name = type.getName(); + ParquetMeta parquetMeta = new ParquetMeta(); + parquetMeta.setName(name); + OriginalType originalType = type.getOriginalType(); + parquetMeta.setOriginalType(originalType); + if (type.isPrimitive()) { + PrimitiveType primitiveType = type.asPrimitiveType(); + parquetMeta.setPrimitiveType(primitiveType); + } + parquetMetaMap.put(name, parquetMeta); + } + return parquetMetaMap; + } +} diff --git a/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/ParquetMeta.java b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/ParquetMeta.java new file mode 100644 index 00000000..6f99e9b5 --- /dev/null +++ b/hdfsreader/src/main/java/com/alibaba/datax/plugin/reader/hdfsreader/ParquetMeta.java @@ -0,0 +1,38 @@ +package com.alibaba.datax.plugin.reader.hdfsreader; + +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; + +/** + * @author jitongchen + * @date 2023/9/7 10:20 AM + */ +public class ParquetMeta { + private String name; + private OriginalType originalType; + private PrimitiveType primitiveType; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public OriginalType getOriginalType() { + return originalType; + } + + public void setOriginalType(OriginalType originalType) { + this.originalType = originalType; + } + + public PrimitiveType getPrimitiveType() { + return primitiveType; + } + + public void setPrimitiveType(PrimitiveType primitiveType) { + this.primitiveType = primitiveType; + } +} \ No newline at end of file diff --git a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsHelper.java b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsHelper.java index a9e157b7..09fd2723 100644 --- a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsHelper.java +++ b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsHelper.java @@ -27,9 +27,8 @@ import org.apache.hadoop.mapred.*; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import parquet.schema.OriginalType; -import parquet.schema.PrimitiveType; -import parquet.schema.Types; +import parquet.hadoop.metadata.CompressionCodecName; +import parquet.schema.*; import java.io.IOException; import java.text.SimpleDateFormat; @@ -626,4 +625,61 @@ public class HdfsHelper { } return typeBuilder.named("m").toString(); } + + public void parquetFileStartWrite(RecordReceiver lineReceiver, Configuration config, String fileName, TaskPluginCollector taskPluginCollector, Configuration taskConfig) { + MessageType messageType = null; + ParquetFileProccessor proccessor = null; + Path outputPath = new Path(fileName); + String schema = config.getString(Key.PARQUET_SCHEMA); + try { + messageType = MessageTypeParser.parseMessageType(schema); + } catch (Exception e) { + String message = String.format("Error parsing the Schema string [%s] into MessageType", schema); + LOG.error(message); + throw DataXException.asDataXException(HdfsWriterErrorCode.PARSE_MESSAGE_TYPE_FROM_SCHEMA_ERROR, e); + } + + // determine the compression codec + String compress = config.getString(Key.COMPRESS, null); + // be compatible with the old NONE + if ("NONE".equalsIgnoreCase(compress)) { + compress = "UNCOMPRESSED"; + } + CompressionCodecName compressionCodecName = CompressionCodecName.fromConf(compress); + LOG.info("The compression codec used for parquet writing is: {}", compressionCodecName, compress); + try { + proccessor = new ParquetFileProccessor(outputPath, messageType, compressionCodecName, false, taskConfig, taskPluginCollector, hadoopConf); + } catch (Exception e) { + String message = String.format("Initializing ParquetFileProccessor based on Schema[%s] failed.", schema); + LOG.error(message); + throw DataXException.asDataXException(HdfsWriterErrorCode.INIT_PROCCESSOR_FAILURE, e); + } + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyyMMddHHmm"); + String attempt = "attempt_" + dateFormat.format(new Date()) + "_0001_m_000000_0"; + conf.set(JobContext.TASK_ATTEMPT_ID, attempt); + FileOutputFormat outFormat = new TextOutputFormat(); + outFormat.setOutputPath(conf, outputPath); + outFormat.setWorkOutputPath(conf, outputPath); + try { + Record record = null; + while ((record = lineReceiver.getFromReader()) != null) { + proccessor.write(record); + } + } catch (Exception e) { + String message = String.format("An exception occurred while writing the file file [%s]", fileName); + LOG.error(message); + Path path = new Path(fileName); + deleteDir(path.getParent()); + throw DataXException.asDataXException(HdfsWriterErrorCode.Write_FILE_IO_ERROR, e); + } finally { + if (proccessor != null) { + try { + proccessor.close(); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + } + } + } + } + } diff --git a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsWriter.java b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsWriter.java index 59ec6d18..e7707461 100644 --- a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsWriter.java +++ b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsWriter.java @@ -53,8 +53,8 @@ public class HdfsWriter extends Writer { this.defaultFS = this.writerSliceConfig.getNecessaryValue(Key.DEFAULT_FS, HdfsWriterErrorCode.REQUIRED_VALUE); //fileType check this.fileType = this.writerSliceConfig.getNecessaryValue(Key.FILE_TYPE, HdfsWriterErrorCode.REQUIRED_VALUE); - if( !fileType.equalsIgnoreCase("ORC") && !fileType.equalsIgnoreCase("TEXT")){ - String message = "HdfsWriter插件目前只支持ORC和TEXT两种格式的文件,请将filetype选项的值配置为ORC或者TEXT"; + if (!fileType.equalsIgnoreCase("ORC") && !fileType.equalsIgnoreCase("TEXT") && !fileType.equalsIgnoreCase("PARQUET")) { + String message = "HdfsWriter插件目前只支持ORC、TEXT、PARQUET三种格式的文件,请将filetype选项的值配置为ORC、TEXT或PARQUET"; throw DataXException.asDataXException(HdfsWriterErrorCode.ILLEGAL_VALUE, message); } //path @@ -415,6 +415,9 @@ public class HdfsWriter extends Writer { //写ORC FILE hdfsHelper.orcFileStartWrite(lineReceiver,this.writerSliceConfig, this.fileName, this.getTaskPluginCollector()); + } else if (fileType.equalsIgnoreCase("PARQUET")) { + //写PARQUET FILE + hdfsHelper.parquetFileStartWrite(lineReceiver, this.writerSliceConfig, this.fileName, this.getTaskPluginCollector(), this.writerSliceConfig); } LOG.info("end do write"); diff --git a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsWriterErrorCode.java b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsWriterErrorCode.java index a9e1cb30..8a729f97 100644 --- a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsWriterErrorCode.java +++ b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/HdfsWriterErrorCode.java @@ -16,7 +16,11 @@ public enum HdfsWriterErrorCode implements ErrorCode { CONNECT_HDFS_IO_ERROR("HdfsWriter-06", "与HDFS建立连接时出现IO异常."), COLUMN_REQUIRED_VALUE("HdfsWriter-07", "您column配置中缺失了必须填写的参数值."), HDFS_RENAME_FILE_ERROR("HdfsWriter-08", "将文件移动到配置路径失败."), - KERBEROS_LOGIN_ERROR("HdfsWriter-09", "KERBEROS认证失败"); + KERBEROS_LOGIN_ERROR("HdfsWriter-09", "KERBEROS认证失败"), + PARSE_MESSAGE_TYPE_FROM_SCHEMA_ERROR("HdfsWriter-10", "Parse parquet schema error"), + + INIT_PROCCESSOR_FAILURE("HdfsWriter-11", "Init processor failed"); + private final String code; private final String description; diff --git a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/Key.java b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/Key.java index 2b1fab98..05f4cd0a 100644 --- a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/Key.java +++ b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/Key.java @@ -46,4 +46,32 @@ public class Key { public static final String PARQUET_SCHEMA = "parquetSchema"; public static final String PARQUET_MERGE_RESULT = "parquetMergeResult"; + + /** + * hive 3.x 或 cdh高版本,使用UTC时区存储时间戳,如果发现时区偏移,该配置项要配置成 true + */ + public static final String PARQUET_UTC_TIMESTAMP = "parquetUtcTimestamp"; + + // Kerberos + public static final String KERBEROS_CONF_FILE_PATH = "kerberosConfFilePath"; + + // PanguFS + public final static String PANGU_FS_CONFIG = "panguFSConfig"; + public final static String PANGU_FS_CONFIG_NUWA_CLUSTER = "nuwaCluster"; + public final static String PANGU_FS_CONFIG_NUWA_SERVERS = "nuwaServers"; + public final static String PANGU_FS_CONFIG_NUWA_PROXIES = "nuwaProxies"; + public final static String PANGU_FS_CONFIG_CAPABILITY = "capability"; + + + public static final String FS_OSS_UPLOAD_THREAD_CONCURRENCY = "ossUploadConcurrency"; + // + public static final String FS_OSS_UPLOAD_QUEUE_SIZE = "ossUploadQueueSize"; + // + public static final String FS_OSS_UPLOAD_MAX_PENDING_TASKS_PER_STREAM = "ossUploadMaxPendingTasksPerStream"; + + public static final String FS_OSS_BLOCKLET_SIZE_MB = "ossBlockSize"; + + public static final String FILE_SYSTEM_TYPE = "fileSystemType"; + public static final String ENABLE_COLUMN_EXCHANGE = "enableColumnExchange"; + public static final String SUPPORT_HIVE_DATETIME = "supportHiveDateTime"; } diff --git a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/ParquetFileProccessor.java b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/ParquetFileProccessor.java new file mode 100644 index 00000000..90d0f6e5 --- /dev/null +++ b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/ParquetFileProccessor.java @@ -0,0 +1,30 @@ +package com.alibaba.datax.plugin.writer.hdfswriter; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.plugin.TaskPluginCollector; +import com.alibaba.datax.common.util.Configuration; +import org.apache.hadoop.fs.Path; +import parquet.hadoop.ParquetWriter; +import parquet.hadoop.metadata.CompressionCodecName; +import parquet.schema.MessageType; + +import java.io.IOException; + +/** + * @author jitongchen + * @date 2023/9/7 9:41 AM + */ +public class ParquetFileProccessor extends ParquetWriter { + + public ParquetFileProccessor(Path file, MessageType schema, boolean enableDictionary, Configuration taskConfig, TaskPluginCollector taskPluginCollector, org.apache.hadoop.conf.Configuration configuration) throws IOException { + this(file, schema, CompressionCodecName.UNCOMPRESSED, enableDictionary, taskConfig, taskPluginCollector, configuration); + } + + public ParquetFileProccessor(Path file, MessageType schema, CompressionCodecName codecName, boolean enableDictionary, Configuration taskConfig, TaskPluginCollector taskPluginCollector) throws IOException { + super(file, new ParquetFileSupport(schema, taskConfig, taskPluginCollector), codecName, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE, DEFAULT_PAGE_SIZE, enableDictionary, false, DEFAULT_WRITER_VERSION); + } + + public ParquetFileProccessor(Path file, MessageType schema, CompressionCodecName codecName, boolean enableDictionary, Configuration taskConfig, TaskPluginCollector taskPluginCollector, org.apache.hadoop.conf.Configuration configuration) throws IOException { + super(file, new ParquetFileSupport(schema, taskConfig, taskPluginCollector), codecName, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE, DEFAULT_PAGE_SIZE, enableDictionary, false, DEFAULT_WRITER_VERSION, configuration); + } +} diff --git a/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/ParquetFileSupport.java b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/ParquetFileSupport.java new file mode 100644 index 00000000..410d5231 --- /dev/null +++ b/hdfswriter/src/main/java/com/alibaba/datax/plugin/writer/hdfswriter/ParquetFileSupport.java @@ -0,0 +1,642 @@ +package com.alibaba.datax.plugin.writer.hdfswriter; + +import com.alibaba.datax.common.element.*; +import com.alibaba.datax.common.plugin.TaskPluginCollector; +import com.alibaba.datax.common.util.LimitLogger; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONArray; +import com.alibaba.fastjson2.JSONObject; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import parquet.column.ColumnDescriptor; +import parquet.hadoop.api.WriteSupport; +import parquet.io.api.Binary; +import parquet.io.api.RecordConsumer; +import parquet.schema.*; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoField; +import java.util.Arrays; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * @author jitongchen + * @date 2023/9/7 9:41 AM + */ +public class ParquetFileSupport extends WriteSupport { + public static final Logger LOGGER = LoggerFactory.getLogger(ParquetFileSupport.class); + private MessageType schema; + private List columns; + private RecordConsumer recordConsumer; + private boolean useRawDataTransf = true; + private boolean printStackTrace = true; + + // 不通类型的nullFormat + private String nullFormat; + + private String dateFormat; + private boolean isUtcTimestamp; + private SimpleDateFormat dateParse; + private Binary binaryForNull; + private TaskPluginCollector taskPluginCollector; + private String dataxParquetMode; + + public ParquetFileSupport(MessageType schema, com.alibaba.datax.common.util.Configuration taskConfig, TaskPluginCollector taskPluginCollector) { + this.schema = schema; + this.columns = schema.getColumns(); + this.useRawDataTransf = taskConfig.getBool(Key.PARQUET_FILE_USE_RAW_DATA_TRANSF, true); + + // 不通类型的nullFormat + this.nullFormat = taskConfig.getString(Key.NULL_FORMAT, Constant.DEFAULT_NULL_FORMAT); + this.binaryForNull = Binary.fromString(this.nullFormat); + + this.dateFormat = taskConfig.getString(Key.DATE_FORMAT, null); + if (StringUtils.isNotBlank(this.dateFormat)) { + this.dateParse = new SimpleDateFormat(dateFormat); + } + + this.isUtcTimestamp = taskConfig.getBool(Key.PARQUET_UTC_TIMESTAMP, false); + + this.taskPluginCollector = taskPluginCollector; + if (taskConfig.getKeys().contains("dataxParquetMode")) { + this.dataxParquetMode = taskConfig.getString("dataxParquetMode"); + } else { + // 默认值是columns + this.dataxParquetMode = "columns"; + } + } + + @Override + public WriteContext init(Configuration configuration) { + return new WriteContext(schema, new HashMap()); + } + + @Override + public void prepareForWrite(RecordConsumer recordConsumer) { + this.recordConsumer = recordConsumer; + } + + @Override + public void write(Record values) { + if (dataxParquetMode.equalsIgnoreCase("fields")) { + writeBaseOnFields(values); + return; + } + + // NOTE: 下面的实现其实是不对的,只是看代码注释貌似有用户已经在用 + // 所以暂时不动下面的逻辑。 + // 默认走的就是下面的这条代码路径 + if (values != null && columns != null && values.getColumnNumber() == columns.size()) { + recordConsumer.startMessage(); + for (int i = 0; i < columns.size(); i++) { + Column value = values.getColumn(i); + ColumnDescriptor columnDescriptor = columns.get(i); + Type type = this.schema.getFields().get(i); + if (value != null) { + try { + if (this.useRawDataTransf) { + if (value.getRawData() == null) { + continue; + } + recordConsumer.startField(columnDescriptor.getPath()[0], i); + // 原来使用Column->RawData的方法其实是错误的类型转换策略,会将DataX的数据内部表示形象序列化出去 + // 但是 Parquet 已经有用户使用了,故暂时只是配置项切换 + String rawData = value.getRawData().toString(); + switch (columnDescriptor.getType()) { + case BOOLEAN: + recordConsumer.addBoolean(Boolean.parseBoolean(rawData)); + break; + case FLOAT: + recordConsumer.addFloat(Float.parseFloat(rawData)); + break; + case DOUBLE: + recordConsumer.addDouble(Double.parseDouble(rawData)); + break; + case INT32: + OriginalType originalType = type.getOriginalType(); + if (originalType != null && StringUtils.equalsIgnoreCase("DATE", originalType.name())) { + int realVal = (int) (new java.sql.Date(Long.parseLong(rawData)).toLocalDate().toEpochDay()); + recordConsumer.addInteger(realVal); + } else { + recordConsumer.addInteger(Integer.parseInt(rawData)); + } + break; + case INT64: + recordConsumer.addLong(Long.valueOf(rawData)); + break; + case INT96: + recordConsumer.addBinary(timestampColToBinary(value)); + break; + case BINARY: + recordConsumer.addBinary(Binary.fromString(rawData)); + break; + case FIXED_LEN_BYTE_ARRAY: + PrimitiveType primitiveType = type.asPrimitiveType(); + if (primitiveType.getDecimalMetadata() != null) { + // decimal + recordConsumer.addBinary(decimalToBinary(value, primitiveType.getDecimalMetadata().getPrecision(), primitiveType.getDecimalMetadata().getScale())); + break; + } + /* fall through */ + default: + recordConsumer.addBinary(Binary.fromString(rawData)); + break; + } + + recordConsumer.endField(columnDescriptor.getPath()[0], i); + } else { + boolean isNull = null == value.getRawData(); + + if (!isNull) { + recordConsumer.startField(columnDescriptor.getPath()[0], i); + + // no skip: empty fields are illegal, the field should be ommited completely instead + switch (columnDescriptor.getType()) { + case BOOLEAN: + recordConsumer.addBoolean(value.asBoolean()); + break; + case FLOAT: + recordConsumer.addFloat(value.asDouble().floatValue()); + break; + case DOUBLE: + recordConsumer.addDouble(value.asDouble()); + break; + case INT32: + OriginalType originalType = type.getOriginalType(); + if (originalType != null && StringUtils.equalsIgnoreCase("DATE", originalType.name())) { + int realVal = (int) (new java.sql.Date(value.asLong()).toLocalDate().toEpochDay()); + recordConsumer.addInteger(realVal); + } else { + recordConsumer.addInteger(value.asLong().intValue()); + } + break; + case INT64: + recordConsumer.addLong(value.asLong()); + break; + case INT96: + recordConsumer.addBinary(timestampColToBinary(value)); + break; + case BINARY: + String valueAsString2Write = null; + if (Column.Type.DATE == value.getType() && null != this.dateParse) { + valueAsString2Write = dateParse.format(value.asDate()); + } else { + valueAsString2Write = value.asString(); + } + recordConsumer.addBinary(Binary.fromString(valueAsString2Write)); + break; + case FIXED_LEN_BYTE_ARRAY: + PrimitiveType primitiveType = type.asPrimitiveType(); + if (primitiveType.getDecimalMetadata() != null) { + // decimal + recordConsumer.addBinary(decimalToBinary(value, primitiveType.getDecimalMetadata().getPrecision(), primitiveType.getDecimalMetadata().getScale())); + break; + } + /* fall through */ + default: + recordConsumer.addBinary(Binary.fromString(value.asString())); + break; + } + recordConsumer.endField(columnDescriptor.getPath()[0], i); + } + } + } catch (Exception e) { + if (printStackTrace) { + printStackTrace = false; + LOGGER.warn("write to parquet error: {}", e.getMessage(), e); + } + // dirty data + if (null != this.taskPluginCollector) { + // job post 里面的merge taskPluginCollector 为null + this.taskPluginCollector.collectDirtyRecord(values, e, e.getMessage()); + } + } + } else { + recordConsumer.addBinary(this.binaryForNull); + } + } + recordConsumer.endMessage(); + } + } + + private Binary decimalToBinary(Column value, int precision, int scale) { + BigDecimal bigDecimal = value.asBigDecimal(); + bigDecimal = bigDecimal.setScale(scale, RoundingMode.HALF_UP); + byte[] decimalBytes = bigDecimal.unscaledValue().toByteArray(); + + int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[precision - 1]; + if (precToBytes == decimalBytes.length) { + // No padding needed. + return Binary.fromByteArray(decimalBytes); + } + + byte[] tgt = new byte[precToBytes]; + + // padding -1 for negative number + if (bigDecimal.compareTo(new BigDecimal("0")) < 0) { + Arrays.fill(tgt, 0, precToBytes - decimalBytes.length, (byte) -1); + } + + System.arraycopy(decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); + return Binary.fromByteArray(tgt); + } + + private static final int JULIAN_EPOCH_OFFSET_DAYS = 2_440_588; + private static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1); + private static final long MILLS_PER_SECOND = TimeUnit.SECONDS.toMillis(1); + private static final long NANOS_PER_DAY = TimeUnit.DAYS.toNanos(1); + private static final long NANOS_PER_SECOND = TimeUnit.SECONDS.toNanos(1); + private static final ZoneOffset defaultOffset = OffsetDateTime.now().getOffset(); + + /** + * int 96 is timestamp in parquet + * + * @param valueColumn + * @return + */ + private Binary timestampColToBinary(Column valueColumn) { + if (valueColumn.getRawData() == null) { + return Binary.EMPTY; + } + long mills; + long nanos = 0; + if (valueColumn instanceof DateColumn) { + DateColumn dateColumn = (DateColumn) valueColumn; + mills = dateColumn.asLong(); + nanos = dateColumn.getNanos(); + } else { + mills = valueColumn.asLong(); + } + int julianDay; + long nanosOfDay; + if (isUtcTimestamp) { + // utc ignore current timezone (task should set timezone same as hive/hdfs) + long seconds = mills >= 0 ? mills / MILLS_PER_SECOND : (mills / MILLS_PER_SECOND - 1); + LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(seconds, (int) nanos, defaultOffset); + julianDay = (int) (localDateTime.getLong(ChronoField.EPOCH_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + nanosOfDay = localDateTime.getLong(ChronoField.NANO_OF_DAY); + } else { + // local date + julianDay = (int) ((mills / MILLIS_IN_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + if (mills >= 0) { + nanosOfDay = ((mills % MILLIS_IN_DAY) / MILLS_PER_SECOND) * NANOS_PER_SECOND + nanos; + } else { + julianDay--; + nanosOfDay = (((mills % MILLIS_IN_DAY) / MILLS_PER_SECOND) - 1) * NANOS_PER_SECOND + nanos; + nanosOfDay += NANOS_PER_DAY; + } + } + + ByteBuffer buf = ByteBuffer.allocate(12); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.putLong(nanosOfDay); + buf.putInt(julianDay); + buf.flip(); + return Binary.fromByteBuffer(buf); + } + + private void writeBaseOnFields(Record values) { + //LOGGER.info("Writing parquet data using fields mode(The correct mode.)"); + List types = this.schema.getFields(); + + if (values != null && types != null && values.getColumnNumber() == types.size()) { + recordConsumer.startMessage(); + writeFields(types, values); + recordConsumer.endMessage(); + } + } + + private void writeFields(List types, Record values) { + for (int i = 0; i < types.size(); i++) { + Type type = types.get(i); + Column value = values.getColumn(i); + if (value != null) { + try { + if (type.isPrimitive()) { + writePrimitiveType(type, value, i); + } else { + writeGroupType(type, (JSON) JSON.parse(value.asString()), i); + } + } catch (Exception e) { + if (printStackTrace) { + printStackTrace = false; + LOGGER.warn("write to parquet error: {}", e.getMessage(), e); + } + // dirty data + if (null != this.taskPluginCollector) { + // job post 里面的merge taskPluginCollector 为null + this.taskPluginCollector.collectDirtyRecord(values, e, e.getMessage()); + } + } + } + } + } + + private void writeFields(List types, JSONObject values) { + for (int i = 0; i < types.size(); i++) { + Type type = types.get(i); + Object value = values.get(type.getName()); + + if (value != null) { + try { + if (type.isPrimitive()) { + writePrimitiveType(type, value, i); + } else { + writeGroupType(type, (JSON) value, i); + } + } catch (Exception e) { + if (printStackTrace) { + printStackTrace = false; + LOGGER.warn("write to parquet error: {}", e.getMessage(), e); + } + } + } else { + recordConsumer.addBinary(this.binaryForNull); + } + } + } + + private void writeGroupType(Type type, JSON value, int index) { + GroupType groupType = type.asGroupType(); + OriginalType originalType = groupType.getOriginalType(); + if (originalType != null) { + switch (originalType) { + case MAP: + writeMap(groupType, value, index); + break; + case LIST: + writeList(groupType, value, index); + break; + default: + break; + } + } else { + // struct + writeStruct(groupType, value, index); + } + } + + private void writeMap(GroupType groupType, JSON value, int index) { + if (value == null) { + return; + } + + JSONObject json = (JSONObject) value; + + if (json.isEmpty()) { + return; + } + + recordConsumer.startField(groupType.getName(), index); + + recordConsumer.startGroup(); + + // map + // key_value start + recordConsumer.startField("key_value", 0); + recordConsumer.startGroup(); + + List keyValueFields = groupType.getFields().get(0).asGroupType().getFields(); + Type keyType = keyValueFields.get(0); + Type valueType = keyValueFields.get(1); + for (String key : json.keySet()) { + // key + writePrimitiveType(keyType, key, 0); + + // value + if (valueType.isPrimitive()) { + writePrimitiveType(valueType, json.get(key), 1); + } else { + writeGroupType(valueType, (JSON) json.get(key), 1); + } + } + + recordConsumer.endGroup(); + recordConsumer.endField("key_value", 0); + // key_value end + + recordConsumer.endGroup(); + recordConsumer.endField(groupType.getName(), index); + } + + private void writeList(GroupType groupType, JSON value, int index) { + if (value == null) { + return; + } + + JSONArray json = (JSONArray) value; + + if (json.isEmpty()) { + return; + } + + recordConsumer.startField(groupType.getName(), index); + // list + recordConsumer.startGroup(); + + // list start + recordConsumer.startField("list", 0); + recordConsumer.startGroup(); + + Type elementType = groupType.getFields().get(0).asGroupType().getFields().get(0); + + if (elementType.isPrimitive()) { + for (Object elementValue : json) { + writePrimitiveType(elementType, elementValue, 0); + } + } else { + for (Object elementValue : json) { + writeGroupType(elementType, (JSON) elementValue, 0); + } + } + + recordConsumer.endGroup(); + recordConsumer.endField("list", 0); + // list end + recordConsumer.endGroup(); + + recordConsumer.endField(groupType.getName(), index); + } + + private void writeStruct(GroupType groupType, JSON value, int index) { + if (value == null) { + return; + } + JSONObject json = (JSONObject) value; + if (json.isEmpty()) { + return; + } + + recordConsumer.startField(groupType.getName(), index); + // struct start + recordConsumer.startGroup(); + + writeFields(groupType.getFields(), json); + recordConsumer.endGroup(); + // struct end + recordConsumer.endField(groupType.getName(), index); + } + + private void writePrimitiveType(Type type, Object value, int index) { + if (value == null) { + return; + } + + recordConsumer.startField(type.getName(), index); + PrimitiveType primitiveType = type.asPrimitiveType(); + + switch (primitiveType.getPrimitiveTypeName()) { + case BOOLEAN: + recordConsumer.addBoolean((Boolean) value); + break; + case FLOAT: + if (value instanceof Float) { + recordConsumer.addFloat(((Float) value).floatValue()); + } else if (value instanceof Double) { + recordConsumer.addFloat(((Double) value).floatValue()); + } else if (value instanceof Long) { + recordConsumer.addFloat(((Long) value).floatValue()); + } else if (value instanceof Integer) { + recordConsumer.addFloat(((Integer) value).floatValue()); + } + break; + case DOUBLE: + if (value instanceof Float) { + recordConsumer.addDouble(((Float) value).doubleValue()); + } else if (value instanceof Double) { + recordConsumer.addDouble(((Double) value).doubleValue()); + } else if (value instanceof Long) { + recordConsumer.addDouble(((Long) value).doubleValue()); + } else if (value instanceof Integer) { + recordConsumer.addDouble(((Integer) value).doubleValue()); + } + break; + case INT32: + if (value instanceof Integer) { + recordConsumer.addInteger((Integer) value); + } else if (value instanceof Long) { + recordConsumer.addInteger(((Long) value).intValue()); + } else { + // 之前代码写的有问题,导致这里丢列了没抛异常,先收集,后续看看有没有任务命中在决定怎么改 + LimitLogger.limit("dirtyDataHiveWriterParquet", TimeUnit.MINUTES.toMillis(1), () -> LOGGER.warn("dirtyDataHiveWriterParquet {}", String.format("Invalid value: %s(clazz: %s) for field: %s", value, value.getClass(), type.getName()))); + } + break; + case INT64: + if (value instanceof Integer) { + recordConsumer.addLong(((Integer) value).longValue()); + } else if (value instanceof Long) { + recordConsumer.addInteger(((Long) value).intValue()); + } else { + // 之前代码写的有问题,导致这里丢列了没抛异常,先收集,后续看看有没有任务命中在决定怎么改 + LimitLogger.limit("dirtyDataHiveWriterParquet", TimeUnit.MINUTES.toMillis(1), () -> LOGGER.warn("dirtyDataHiveWriterParquet {}", String.format("Invalid value: %s(clazz: %s) for field: %s", value, value.getClass(), type.getName()))); + } + break; + case INT96: + if (value instanceof Integer) { + recordConsumer.addBinary(timestampColToBinary(new LongColumn((Integer) value))); + } else if (value instanceof Long) { + recordConsumer.addBinary(timestampColToBinary(new LongColumn((Long) value))); + } else if (value instanceof Timestamp) { + recordConsumer.addBinary(timestampColToBinary(new DateColumn((Timestamp) value))); + } else if (value instanceof Date) { + recordConsumer.addBinary(timestampColToBinary(new DateColumn((Date) value))); + } else { + recordConsumer.addBinary(timestampColToBinary(new StringColumn(value.toString()))); + } + break; + case FIXED_LEN_BYTE_ARRAY: + if (primitiveType.getDecimalMetadata() != null) { + // decimal + Column column; + if (value instanceof Integer) { + column = new LongColumn((Integer) value); + } else if (value instanceof Long) { + column = new LongColumn((Long) value); + } else if (value instanceof Double) { + column = new DoubleColumn((Double) value); + } else if (value instanceof BigDecimal) { + column = new DoubleColumn((BigDecimal) value); + } else { + column = new StringColumn(value.toString()); + } + recordConsumer.addBinary(decimalToBinary(column, primitiveType.getDecimalMetadata().getPrecision(), primitiveType.getDecimalMetadata().getScale())); + break; + } + /* fall through */ + case BINARY: + default: + recordConsumer.addBinary(Binary.fromString((String) value)); + break; + } + recordConsumer.endField(type.getName(), index); + } + + private void writePrimitiveType(Type type, Column value, int index) { + if (value == null || value.getRawData() == null) { + return; + } + + recordConsumer.startField(type.getName(), index); + PrimitiveType primitiveType = type.asPrimitiveType(); + switch (primitiveType.getPrimitiveTypeName()) { + case BOOLEAN: + recordConsumer.addBoolean(value.asBoolean()); + break; + case FLOAT: + recordConsumer.addFloat(value.asDouble().floatValue()); + break; + case DOUBLE: + recordConsumer.addDouble(value.asDouble()); + break; + case INT32: + OriginalType originalType = type.getOriginalType(); + if (OriginalType.DATE.equals(originalType)) { + int realVal = (int) (new java.sql.Date(value.asLong()).toLocalDate().toEpochDay()); + recordConsumer.addInteger(realVal); + } else { + recordConsumer.addInteger(value.asLong().intValue()); + } + break; + case INT64: + recordConsumer.addLong(value.asLong()); + break; + case INT96: + recordConsumer.addBinary(timestampColToBinary(value)); + break; + case BINARY: + String valueAsString2Write = null; + if (Column.Type.DATE == value.getType() && null != this.dateParse) { + valueAsString2Write = dateParse.format(value.asDate()); + } else { + valueAsString2Write = value.asString(); + } + recordConsumer.addBinary(Binary.fromString(valueAsString2Write)); + break; + case FIXED_LEN_BYTE_ARRAY: + if (primitiveType.getDecimalMetadata() != null) { + // decimal + recordConsumer.addBinary(decimalToBinary(value, primitiveType.getDecimalMetadata().getPrecision(), primitiveType.getDecimalMetadata().getScale())); + break; + } + /* fall through */ + default: + recordConsumer.addBinary(Binary.fromString(value.asString())); + break; + } + recordConsumer.endField(type.getName(), index); + } +} diff --git a/neo4jwriter/doc/neo4jwriter.md b/neo4jwriter/doc/neo4jwriter.md new file mode 100644 index 00000000..0c6e356c --- /dev/null +++ b/neo4jwriter/doc/neo4jwriter.md @@ -0,0 +1,193 @@ +# DataX neo4jWriter 插件文档 + +## 功能简介 + +本目前市面上的neo4j 批量导入主要有Cypher Create,Load CSV,第三方或者官方提供的Batch Import。Load CSV支持节点10W级别一下,Batch Import 需要对数据库进行停机。要想实现不停机的数据写入,Cypher是最好的方式。 + +## 支持版本 + +支持Neo4j 4 和Neo4j 5,如果是Neo4j 3,需要自行将驱动降低至相对应的版本进行编译。 + +## 实现原理 + +将datax的数据转换成了neo4j驱动能识别的对象,利用 unwind 语法进行批量插入。 + +## 如何配置 + +### 配置项介绍 + +| 配置 | 说明 | 是否必须 | 默认值 | 示例 | +|:-------------------------------|--------------------| -------- | ------ | ---------------------------------------------------- | +| database | 数据库名字 | 是 | - | neo4j | +| uri | 数据库访问链接 | 是 | - | bolt://localhost:7687 | +| username | 访问用户名 | 是 | - | neo4j | +| password | 访问密码 | 是 | - | neo4j | +| bearerToken | 权限相关 | 否 | - | - | +| kerberosTicket | 权限相关 | 否 | - | - | +| cypher | 同步语句 | 是 | - | unwind $batch as row create(p) set p.name = row.name | +| batchDataVariableName | unwind 携带的数据变量名 | | | batch | +| properties | 定义neo4j中数据的属性名字和类型 | 是 | - | 见后续案例 | +| batchSize | 一批写入数据量 | 否 | 1000 | | +| maxTransactionRetryTimeSeconds | 事务运行最长时间 | 否 | 30秒 | 30 | +| maxConnectionTimeoutSeconds | 驱动最长链接时间 | 否 | 30秒 | 30 | +| retryTimes | 发生错误的重试次数 | 否 | 3次 | 3 | +| retrySleepMills | 重试失败后的等待时间 | 否 | 3秒 | 3 | + +### 支持的数据类型 +> 配置时均忽略大小写 +``` +BOOLEAN, +STRING, +LONG, +SHORT, +INTEGER, +DOUBLE, +FLOAT, +LOCAL_DATE, +LOCAL_TIME, +LOCAL_DATE_TIME, +LIST, +//map类型支持 . 属性表达式取值 +MAP, +CHAR_ARRAY, +BYTE_ARRAY, +BOOLEAN_ARRAY, +STRING_ARRAY, +LONG_ARRAY, +INT_ARRAY, +SHORT_ARRAY, +DOUBLE_ARRAY, +FLOAT_ARRAY, +Object_ARRAY +``` + +### 写节点 + +这里提供了一个写节点包含很多类型属性的例子。你可以在我的测试方法中运行。 + +```json +"writer": { + "name": "neo4jWriter", + "parameter": { + "uri": "neo4j://localhost:7687", + "username": "neo4j", + "password": "Test@12343", + "database": "neo4j", + "cypher": "unwind $batch as row create(p:Person) set p.pbool = row.pbool,p.pstring = row.pstring,p.plong = row.plong,p.pshort = row.pshort,p.pdouble=row.pdouble,p.pstringarr=row.pstringarr,p.plocaldate=row.plocaldate", + "batchDataVariableName": "batch", + "batchSize": "33", + "properties": [ + { + "name": "pbool", + "type": "BOOLEAN" + }, + { + "name": "pstring", + "type": "STRING" + }, + { + "name": "plong", + "type": "LONG" + }, + { + "name": "pshort", + "type": "SHORT" + }, + { + "name": "pdouble", + "type": "DOUBLE" + }, + { + "name": "pstringarr", + "type": "STRING_ARRAY", + "split": "," + }, + { + "name": "plocaldate", + "type": "LOCAL_DATE", + "dateFormat": "yyyy-MM-dd" + } + ] + } + } +``` + +### 写关系 + +```json +"writer": { + "name": "neo4jWriter", + "parameter": { + "uri": "neo4j://localhost:7687", + "username": "neo4j", + "password": "Test@12343", + "database": "neo4j", + "cypher": "unwind $batch as row match(p1:Person) where p1.id = row.startNodeId match(p2:Person) where p2.id = row.endNodeId create (p1)-[:LINK]->(p2)", + "batchDataVariableName": "batch", + "batch_size": "33", + "properties": [ + { + "name": "startNodeId", + "type": "STRING" + }, + { + "name": "endNodeId", + "type": "STRING" + } + ] + } + } +``` + +### 节点/关系类型动态写 + +> 需要使用AOPC函数拓展,如果你的数据库没有,请安装APOC函数拓展 + +```json + "writer": { + "name": "neo4jWriter", + "parameter": { + "uri": "bolt://localhost:7687", + "username": "yourUserName", + "password": "yourPassword", + "database": "yourDataBase", + "cypher": "unwind $batch as row CALL apoc.cypher.doIt( 'create (n:`' + row.Label + '`{id:$id})' ,{id: row.id} ) YIELD value RETURN 1 ", + "batchDataVariableName": "batch", + "batch_size": "1", + "properties": [ + { + "name": "Label", + "type": "STRING" + }, + { + "name": "id", + "type": "STRING" + } + ] + } + } +``` + +## 注意事项 + +* properties定义的顺序需要与reader端顺序一一对应。 +* 灵活使用map类型,可以免去很多数据加工的烦恼。在cypher中,可以根据 . 属性访问符号一直取值。比如 unwind $batch as row create (p) set p.name = row.prop.name,set p.age = row.prop.age,在这个例子中,prop是map类型,包含name和age两个属性。 +* 如果提示事务超时,建议调大事务运行时间或者调小batchSize +* 如果用于更新场景,遇到死锁问题影响写入,建议二开源码加入死锁异常检测,并进行重试。 + +## 性能报告 + +**JVM参数** + +16G G1垃圾收集器 8核心 + +**Neo4j数据库配置** + +32核心,256G + +**datax 配置** + +* Channel 20 batchsize = 1000 +* 任务平均流量:15.23MB/s +* 记录写入速度:44440 rec/s +* 读出记录总数:2222013 diff --git a/neo4jwriter/pom.xml b/neo4jwriter/pom.xml new file mode 100644 index 00000000..2ff0f550 --- /dev/null +++ b/neo4jwriter/pom.xml @@ -0,0 +1,100 @@ + + + + com.alibaba.datax + datax-all + 0.0.1-SNAPSHOT + + 4.0.0 + + neo4jwriter + neo4jwriter + jar + + + 8 + 8 + UTF-8 + 4.4.9 + 4.13.2 + 1.17.6 + + + + org.slf4j + slf4j-api + + + ch.qos.logback + logback-classic + + + org.neo4j.driver + neo4j-java-driver + ${neo4j-java-driver.version} + + + com.alibaba.datax + datax-common + ${datax-project-version} + + + + org.testcontainers + testcontainers + ${test.container.version} + + + + junit + junit + ${junit4.version} + test + + + + + + + src/main/resources + + **/*.* + + true + + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + + maven-assembly-plugin + + + src/main/assembly/package.xml + + datax + + + + dwzip + package + + single + + + + + + + diff --git a/neo4jwriter/src/main/assembly/package.xml b/neo4jwriter/src/main/assembly/package.xml new file mode 100644 index 00000000..3acbe674 --- /dev/null +++ b/neo4jwriter/src/main/assembly/package.xml @@ -0,0 +1,35 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + plugin_job_template.json + + plugin/writer/neo4jwriter + + + target/ + + neo4jwriter-0.0.1-SNAPSHOT.jar + + plugin/writer/neo4jwriter + + + + + + false + plugin/writer/neo4jwriter/libs + runtime + + + diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/Neo4jClient.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/Neo4jClient.java new file mode 100644 index 00000000..4451bbdf --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/Neo4jClient.java @@ -0,0 +1,256 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter; + + +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.TaskPluginCollector; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.common.util.RetryUtil; +import com.alibaba.datax.plugin.writer.neo4jwriter.adapter.DateAdapter; +import com.alibaba.datax.plugin.writer.neo4jwriter.adapter.ValueAdapter; +import com.alibaba.datax.plugin.writer.neo4jwriter.config.Neo4jProperty; +import com.alibaba.datax.plugin.writer.neo4jwriter.exception.Neo4jErrorCode; +import com.alibaba.fastjson2.JSON; +import org.apache.commons.lang3.StringUtils; +import org.neo4j.driver.*; +import org.neo4j.driver.exceptions.Neo4jException; +import org.neo4j.driver.internal.value.MapValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.concurrent.TimeUnit; + +import static com.alibaba.datax.plugin.writer.neo4jwriter.config.ConfigConstants.*; +import static com.alibaba.datax.plugin.writer.neo4jwriter.exception.Neo4jErrorCode.DATABASE_ERROR; + +public class Neo4jClient { + private static final Logger LOGGER = LoggerFactory.getLogger(Neo4jClient.class); + private Driver driver; + + private WriteConfig writeConfig; + private RetryConfig retryConfig; + private TaskPluginCollector taskPluginCollector; + + private Session session; + + private List writerBuffer; + + + public Neo4jClient(Driver driver, + WriteConfig writeConfig, + RetryConfig retryConfig, + TaskPluginCollector taskPluginCollector) { + this.driver = driver; + this.writeConfig = writeConfig; + this.retryConfig = retryConfig; + this.taskPluginCollector = taskPluginCollector; + this.writerBuffer = new ArrayList<>(writeConfig.batchSize); + } + + public void init() { + String database = writeConfig.database; + //neo4j 3.x 没有数据库 + if (null != database && !"".equals(database)) { + this.session = driver.session(SessionConfig.forDatabase(database)); + } else { + this.session = driver.session(); + } + } + + public static Neo4jClient build(Configuration config, TaskPluginCollector taskPluginCollector) { + + Driver driver = buildNeo4jDriver(config); + String cypher = checkCypher(config); + String database = config.getString(DATABASE.getKey()); + String batchVariableName = config.getString(BATCH_DATA_VARIABLE_NAME.getKey(), + BATCH_DATA_VARIABLE_NAME.getDefaultValue()); + List neo4jProperties = JSON.parseArray(config.getString(NEO4J_PROPERTIES.getKey()), Neo4jProperty.class); + int batchSize = config.getInt(BATCH_SIZE.getKey(), BATCH_SIZE.getDefaultValue()); + int retryTimes = config.getInt(RETRY_TIMES.getKey(), RETRY_TIMES.getDefaultValue()); + + return new Neo4jClient(driver, + new WriteConfig(cypher, database, batchVariableName, neo4jProperties, batchSize), + new RetryConfig(retryTimes, config.getLong(RETRY_SLEEP_MILLS.getKey(), RETRY_SLEEP_MILLS.getDefaultValue())), + taskPluginCollector + ); + } + + private static String checkCypher(Configuration config) { + String cypher = config.getString(CYPHER.getKey()); + if (StringUtils.isBlank(cypher)) { + throw DataXException.asDataXException(Neo4jErrorCode.CONFIG_INVALID, "cypher must not null or empty"); + } + return cypher; + } + + private static Driver buildNeo4jDriver(Configuration config) { + + Config.ConfigBuilder configBuilder = Config.builder().withMaxConnectionPoolSize(1); + String uri = checkUriConfig(config); + + //connection timeout + //连接超时时间 + Long maxConnTime = config.getLong(MAX_CONNECTION_TIMEOUT_SECONDS.getKey(), MAX_TRANSACTION_RETRY_TIME.getDefaultValue()); + configBuilder + .withConnectionAcquisitionTimeout( + maxConnTime * 2, TimeUnit.SECONDS) + .withConnectionTimeout(maxConnTime, TimeUnit.SECONDS); + + + //transaction timeout + //事务运行超时时间 + Long txRetryTime = config.getLong(MAX_TRANSACTION_RETRY_TIME.getKey(), MAX_TRANSACTION_RETRY_TIME.getDefaultValue()); + configBuilder.withMaxTransactionRetryTime(txRetryTime, TimeUnit.SECONDS); + String username = config.getString(USERNAME.getKey()); + String password = config.getString(PASSWORD.getKey()); + String bearerToken = config.getString(BEARER_TOKEN.getKey()); + String kerberosTicket = config.getString(KERBEROS_TICKET.getKey()); + + if (StringUtils.isNotBlank(username) && StringUtils.isNotBlank(password)) { + + return GraphDatabase.driver(uri, AuthTokens.basic(username, password), configBuilder.build()); + + } else if (StringUtils.isNotBlank(bearerToken)) { + + return GraphDatabase.driver(uri, AuthTokens.bearer(bearerToken), configBuilder.build()); + + } else if (StringUtils.isNotBlank(kerberosTicket)) { + + return GraphDatabase.driver(uri, AuthTokens.kerberos(kerberosTicket), configBuilder.build()); + + } + + throw DataXException.asDataXException(Neo4jErrorCode.CONFIG_INVALID, "Invalid Auth config."); + } + + private static String checkUriConfig(Configuration config) { + String uri = config.getString(URI.getKey()); + if (null == uri || uri.length() == 0) { + throw DataXException.asDataXException(Neo4jErrorCode.CONFIG_INVALID, "Invalid uri configuration"); + } + return uri; + } + + public void destroy() { + tryFlushBuffer(); + if (driver != null) { + driver.close(); + } + if (session != null) { + session.close(); + } + DateAdapter.destroy(); + } + + private void tryFlushBuffer() { + if (!writerBuffer.isEmpty()) { + doWrite(writerBuffer); + writerBuffer.clear(); + } + } + + private void tryBatchWrite() { + if (!writerBuffer.isEmpty() && writerBuffer.size() >= writeConfig.batchSize) { + doWrite(writerBuffer); + writerBuffer.clear(); + } + } + + private void doWrite(List values) { + Value batchValues = Values.parameters(this.writeConfig.batchVariableName, values); + Query query = new Query(this.writeConfig.cypher, batchValues); +// LOGGER.debug("query:{}", query.text()); +// LOGGER.debug("batch:{}", toUnwindStr(values)); + try { + RetryUtil.executeWithRetry(() -> { + session.writeTransaction(tx -> tx.run(query)); + return null; + }, this.retryConfig.retryTimes, retryConfig.retrySleepMills, true, + Collections.singletonList(Neo4jException.class)); + } catch (Exception e) { + LOGGER.error("an exception occurred while writing to the database,message:{}", e.getMessage()); + throw DataXException.asDataXException(DATABASE_ERROR, e.getMessage()); + } + + + } + + private String toUnwindStr(List values) { + StringJoiner joiner = new StringJoiner(","); + for (MapValue value : values) { + joiner.add(value.toString()); + } + return "[" + joiner + "]"; + } + + public void tryWrite(Record record) { + MapValue neo4jValue = checkAndConvert(record); + writerBuffer.add(neo4jValue); + tryBatchWrite(); + } + + private MapValue checkAndConvert(Record record) { + int sourceColNum = record.getColumnNumber(); + List neo4jProperties = writeConfig.neo4jProperties; + if (neo4jProperties == null || neo4jProperties.size() != sourceColNum) { + throw new DataXException(Neo4jErrorCode.CONFIG_INVALID, "the read and write columns do not match!"); + } + Map data = new HashMap<>(sourceColNum * 4 / 3); + for (int i = 0; i < sourceColNum; i++) { + Column column = record.getColumn(i); + Neo4jProperty neo4jProperty = neo4jProperties.get(i); + try { + + Value value = ValueAdapter.column2Value(column, neo4jProperty); + data.put(neo4jProperty.getName(), value); + } catch (Exception e) { + LOGGER.info("dirty record:{},message :{}", column, e.getMessage()); + this.taskPluginCollector.collectDirtyRecord(record, e.getMessage()); + } + } + return new MapValue(data); + } + + public List getNeo4jFields() { + return this.writeConfig.neo4jProperties; + } + + + static class RetryConfig { + int retryTimes; + long retrySleepMills; + + RetryConfig(int retryTimes, long retrySleepMills) { + this.retryTimes = retryTimes; + this.retrySleepMills = retrySleepMills; + } + } + + static class WriteConfig { + String cypher; + + String database; + + String batchVariableName; + + List neo4jProperties; + + int batchSize; + + public WriteConfig(String cypher, + String database, + String batchVariableName, + List neo4jProperties, + int batchSize) { + this.cypher = cypher; + this.database = database; + this.batchVariableName = batchVariableName; + this.neo4jProperties = neo4jProperties; + this.batchSize = batchSize; + } + + + } +} diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/Neo4jWriter.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/Neo4jWriter.java new file mode 100644 index 00000000..6a589c1d --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/Neo4jWriter.java @@ -0,0 +1,64 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter; + +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.element.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +public class Neo4jWriter extends Writer { + public static class Job extends Writer.Job { + private static final Logger LOGGER = LoggerFactory.getLogger(Job.class); + + private Configuration jobConf = null; + @Override + public void init() { + LOGGER.info("Neo4jWriter Job init success"); + this.jobConf = getPluginJobConf(); + } + + @Override + public void destroy() { + LOGGER.info("Neo4jWriter Job destroyed"); + } + + @Override + public List split(int mandatoryNumber) { + List configurations = new ArrayList(mandatoryNumber); + for (int i = 0; i < mandatoryNumber; i++) { + configurations.add(this.jobConf.clone()); + } + return configurations; + } + } + + public static class Task extends Writer.Task { + private static final Logger TASK_LOGGER = LoggerFactory.getLogger(Task.class); + private Neo4jClient neo4jClient; + @Override + public void init() { + Configuration taskConf = super.getPluginJobConf(); + this.neo4jClient = Neo4jClient.build(taskConf,getTaskPluginCollector()); + this.neo4jClient.init(); + TASK_LOGGER.info("neo4j writer task init success."); + } + + @Override + public void destroy() { + this.neo4jClient.destroy(); + TASK_LOGGER.info("neo4j writer task destroyed."); + } + + @Override + public void startWrite(RecordReceiver receiver) { + Record record; + while ((record = receiver.getFromReader()) != null){ + this.neo4jClient.tryWrite(record); + } + } + } +} diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/adapter/DateAdapter.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/adapter/DateAdapter.java new file mode 100644 index 00000000..51b214bd --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/adapter/DateAdapter.java @@ -0,0 +1,70 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter.adapter; + + +import com.alibaba.datax.plugin.writer.neo4jwriter.config.Neo4jProperty; +import org.testcontainers.shaded.com.google.common.base.Supplier; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; + +/** + * @author fuyouj + */ +public class DateAdapter { + private static final ThreadLocal LOCAL_DATE_FORMATTER_MAP = new ThreadLocal<>(); + private static final ThreadLocal LOCAL_TIME_FORMATTER_MAP = new ThreadLocal<>(); + private static final ThreadLocal LOCAL_DATE_TIME_FORMATTER_MAP = new ThreadLocal<>(); + private static final String DEFAULT_LOCAL_DATE_FORMATTER = "yyyy-MM-dd"; + private static final String DEFAULT_LOCAL_TIME_FORMATTER = "HH:mm:ss"; + private static final String DEFAULT_LOCAL_DATE_TIME_FORMATTER = "yyyy-MM-dd HH:mm:ss"; + + + public static LocalDate localDate(String text, Neo4jProperty neo4jProperty) { + if (LOCAL_DATE_FORMATTER_MAP.get() != null) { + return LocalDate.parse(text, LOCAL_DATE_FORMATTER_MAP.get()); + } + + String format = getOrDefault(neo4jProperty::getDateFormat, DEFAULT_LOCAL_DATE_FORMATTER); + DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(format); + LOCAL_DATE_FORMATTER_MAP.set(dateTimeFormatter); + return LocalDate.parse(text, dateTimeFormatter); + } + + public static String getOrDefault(Supplier dateFormat, String defaultFormat) { + String format = dateFormat.get(); + if (null == format || "".equals(format)) { + return defaultFormat; + } else { + return format; + } + } + + public static void destroy() { + LOCAL_DATE_FORMATTER_MAP.remove(); + LOCAL_TIME_FORMATTER_MAP.remove(); + LOCAL_DATE_TIME_FORMATTER_MAP.remove(); + } + + public static LocalTime localTime(String text, Neo4jProperty neo4JProperty) { + if (LOCAL_TIME_FORMATTER_MAP.get() != null) { + return LocalTime.parse(text, LOCAL_TIME_FORMATTER_MAP.get()); + } + + String format = getOrDefault(neo4JProperty::getDateFormat, DEFAULT_LOCAL_TIME_FORMATTER); + DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(format); + LOCAL_TIME_FORMATTER_MAP.set(dateTimeFormatter); + return LocalTime.parse(text, dateTimeFormatter); + } + + public static LocalDateTime localDateTime(String text, Neo4jProperty neo4JProperty) { + if (LOCAL_DATE_TIME_FORMATTER_MAP.get() != null){ + return LocalDateTime.parse(text,LOCAL_DATE_TIME_FORMATTER_MAP.get()); + } + String format = getOrDefault(neo4JProperty::getDateFormat, DEFAULT_LOCAL_DATE_TIME_FORMATTER); + DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(format); + LOCAL_DATE_TIME_FORMATTER_MAP.set(dateTimeFormatter); + return LocalDateTime.parse(text, dateTimeFormatter); + } +} diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/adapter/ValueAdapter.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/adapter/ValueAdapter.java new file mode 100644 index 00000000..d0f4044d --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/adapter/ValueAdapter.java @@ -0,0 +1,95 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter.adapter; + + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.plugin.writer.neo4jwriter.config.Neo4jProperty; +import com.alibaba.datax.plugin.writer.neo4jwriter.element.PropertyType; +import com.alibaba.fastjson2.JSON; +import org.neo4j.driver.Value; +import org.neo4j.driver.Values; +import org.neo4j.driver.internal.value.NullValue; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +/** + * @author fuyouj + */ +public class ValueAdapter { + + + public static Value column2Value(final Column column, final Neo4jProperty neo4JProperty) { + + String typeStr = neo4JProperty.getType(); + PropertyType type = PropertyType.fromStrIgnoreCase(typeStr); + if (column.asString() == null) { + return NullValue.NULL; + } + + switch (type) { + case NULL: + return NullValue.NULL; + case MAP: + return Values.value(JSON.parseObject(column.asString(), Map.class)); + case BOOLEAN: + return Values.value(column.asBoolean()); + case STRING: + return Values.value(column.asString()); + case INTEGER: + case LONG: + return Values.value(column.asLong()); + case SHORT: + return Values.value(Short.valueOf(column.asString())); + case FLOAT: + case DOUBLE: + return Values.value(column.asDouble()); + case BYTE_ARRAY: + return Values.value(parseArrayType(neo4JProperty, column.asString(), Byte::valueOf)); + case CHAR_ARRAY: + return Values.value(parseArrayType(neo4JProperty, column.asString(), (s) -> s.charAt(0))); + case BOOLEAN_ARRAY: + return Values.value(parseArrayType(neo4JProperty, column.asString(), Boolean::valueOf)); + case STRING_ARRAY: + case Object_ARRAY: + case LIST: + return Values.value(parseArrayType(neo4JProperty, column.asString(), Function.identity())); + case LONG_ARRAY: + return Values.value(parseArrayType(neo4JProperty, column.asString(), Long::valueOf)); + case INT_ARRAY: + return Values.value(parseArrayType(neo4JProperty, column.asString(), Integer::valueOf)); + case SHORT_ARRAY: + return Values.value(parseArrayType(neo4JProperty, column.asString(), Short::valueOf)); + case DOUBLE_ARRAY: + case FLOAT_ARRAY: + return Values.value(parseArrayType(neo4JProperty, column.asString(), Double::valueOf)); + case LOCAL_DATE: + return Values.value(DateAdapter.localDate(column.asString(), neo4JProperty)); + case LOCAL_TIME: + return Values.value(DateAdapter.localTime(column.asString(), neo4JProperty)); + case LOCAL_DATE_TIME: + return Values.value(DateAdapter.localDateTime(column.asString(), neo4JProperty)); + default: + return Values.value(column.getRawData()); + + } + } + + + private static List parseArrayType(final Neo4jProperty neo4JProperty, + final String strValue, + final Function convertFunc) { + if (null == strValue || "".equals(strValue)) { + return Collections.emptyList(); + } + String split = neo4JProperty.getSplitOrDefault(); + String[] strArr = strValue.split(split); + List ans = new ArrayList<>(); + for (String s : strArr) { + ans.add(convertFunc.apply(s)); + } + return ans; + } +} diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/ConfigConstants.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/ConfigConstants.java new file mode 100644 index 00000000..eed3588e --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/ConfigConstants.java @@ -0,0 +1,116 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter.config; + + +import java.util.List; + +/** + * @author fuyouj + */ +public final class ConfigConstants { + + public static final Long DEFAULT_MAX_TRANSACTION_RETRY_SECONDS = 30L; + + public static final Long DEFAULT_MAX_CONNECTION_SECONDS = 30L; + + + + public static final Option RETRY_TIMES = + Option.builder() + .key("retryTimes") + .defaultValue(3) + .desc("The number of overwrites when an error occurs") + .build(); + + public static final Option RETRY_SLEEP_MILLS = + Option.builder() + .key("retrySleepMills") + .defaultValue(3000L) + .build(); + + /** + * cluster mode please reference + * how to connect cluster mode + */ + public static final Option URI = + Option.builder() + .key("uri") + .noDefaultValue() + .desc("uir of neo4j database") + .build(); + + public static final Option USERNAME = + Option.builder() + .key("username") + .noDefaultValue() + .desc("username for accessing the neo4j database") + .build(); + + public static final Option PASSWORD = + Option.builder() + .key("password") + .noDefaultValue() + .desc("password for accessing the neo4j database") + .build(); + + public static final Option BEARER_TOKEN = + Option.builder() + .key("bearerToken") + .noDefaultValue() + .desc("base64 encoded bearer token of the Neo4j. for Auth.") + .build(); + + public static final Option KERBEROS_TICKET = + Option.builder() + .key("kerberosTicket") + .noDefaultValue() + .desc("base64 encoded kerberos ticket of the Neo4j. for Auth.") + .build(); + + public static final Option DATABASE = + Option.builder() + .key("database") + .noDefaultValue() + .desc("database name.") + .build(); + + public static final Option CYPHER = + Option.builder() + .key("cypher") + .noDefaultValue() + .desc("cypher query.") + .build(); + + public static final Option MAX_TRANSACTION_RETRY_TIME = + Option.builder() + .key("maxTransactionRetryTimeSeconds") + .defaultValue(DEFAULT_MAX_TRANSACTION_RETRY_SECONDS) + .desc("maximum transaction retry time(seconds). transaction fail if exceeded.") + .build(); + public static final Option MAX_CONNECTION_TIMEOUT_SECONDS = + Option.builder() + .key("maxConnectionTimeoutSeconds") + .defaultValue(DEFAULT_MAX_CONNECTION_SECONDS) + .desc("The maximum amount of time to wait for a TCP connection to be established (seconds).") + .build(); + + public static final Option BATCH_DATA_VARIABLE_NAME = + Option.builder() + .key("batchDataVariableName") + .defaultValue("batch") + .desc("in a cypher statement, a variable name that represents a batch of data") + .build(); + + public static final Option> NEO4J_PROPERTIES = + Option.>builder() + .key("properties") + .noDefaultValue() + .desc("neo4j node or relation`s props") + .build(); + + public static final Option BATCH_SIZE = + Option.builder(). + key("batchSize") + .defaultValue(1000) + .desc("max batch size") + .build(); +} diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/Neo4jProperty.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/Neo4jProperty.java new file mode 100644 index 00000000..5c5867b3 --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/Neo4jProperty.java @@ -0,0 +1,82 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter.config; + +/** + * 由于dataX并不能传输数据的元数据,所以只能在writer端定义每列数据的名字 + * datax does not support data metadata, + * only the name of each column of data can be defined on neo4j writer + * + * @author fuyouj + */ +public class Neo4jProperty { + public static final String DEFAULT_SPLIT = ","; + + /** + * name of neo4j field + */ + private String name; + + /** + * neo4j type + * reference by org.neo4j.driver.Values + */ + private String type; + + /** + * for date + */ + private String dateFormat; + + /** + * for array type + */ + private String split; + + public Neo4jProperty() { + } + + public Neo4jProperty(String name, String type, String format, String split) { + this.name = name; + this.type = type; + this.dateFormat = format; + this.split = split; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String getDateFormat() { + return dateFormat; + } + + public void setDateFormat(String dateFormat) { + this.dateFormat = dateFormat; + } + + public String getSplit() { + return getSplitOrDefault(); + } + + public String getSplitOrDefault() { + if (split == null || "".equals(split)) { + return DEFAULT_SPLIT; + } + return split; + } + + public void setSplit(String split) { + this.split = split; + } +} diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/Option.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/Option.java new file mode 100644 index 00000000..f22bd205 --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/config/Option.java @@ -0,0 +1,65 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter.config; + + +public class Option { + + public static class Builder { + private String key; + private String desc; + + private T defaultValue; + + public Builder key(String key) { + this.key = key; + return this; + } + + public Builder desc(String desc) { + this.desc = desc; + return this; + } + + public Builder defaultValue(T defaultValue) { + this.defaultValue = defaultValue; + return this; + } + + public Builder noDefaultValue() { + return this; + } + + public Option build() { + return new Option<>(this.key, this.desc, this.defaultValue); + } + } + + private final String key; + private final String desc; + + private final T defaultValue; + + public Option(String key, String desc, T defaultValue) { + this.key = key; + this.desc = desc; + this.defaultValue = defaultValue; + } + + public static Builder builder(){ + return new Builder<>(); + } + + public String getKey() { + return key; + } + + public String getDesc() { + return desc; + } + + public T getDefaultValue() { + if (defaultValue == null){ + throw new IllegalStateException(key + ":defaultValue is null"); + } + return defaultValue; + } +} diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/element/PropertyType.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/element/PropertyType.java new file mode 100644 index 00000000..b3446de7 --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/element/PropertyType.java @@ -0,0 +1,40 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter.element; + +import java.util.Arrays; + +/** + * @see org.neo4j.driver.Values + * @author fuyouj + */ +public enum PropertyType { + NULL, + BOOLEAN, + STRING, + LONG, + SHORT, + INTEGER, + DOUBLE, + FLOAT, + LOCAL_DATE, + LOCAL_TIME, + LOCAL_DATE_TIME, + LIST, + MAP, + CHAR_ARRAY, + BYTE_ARRAY, + BOOLEAN_ARRAY, + STRING_ARRAY, + LONG_ARRAY, + INT_ARRAY, + SHORT_ARRAY, + DOUBLE_ARRAY, + FLOAT_ARRAY, + Object_ARRAY; + + public static PropertyType fromStrIgnoreCase(String typeStr) { + return Arrays.stream(PropertyType.values()) + .filter(e -> e.name().equalsIgnoreCase(typeStr)) + .findFirst() + .orElse(PropertyType.STRING); + } +} diff --git a/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/exception/Neo4jErrorCode.java b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/exception/Neo4jErrorCode.java new file mode 100644 index 00000000..d7df79ff --- /dev/null +++ b/neo4jwriter/src/main/java/com/alibaba/datax/plugin/writer/neo4jwriter/exception/Neo4jErrorCode.java @@ -0,0 +1,37 @@ +package com.alibaba.datax.plugin.writer.neo4jwriter.exception; + +import com.alibaba.datax.common.spi.ErrorCode; + + +public enum Neo4jErrorCode implements ErrorCode { + + /** + * Invalid configuration + * 配置校验异常 + */ + CONFIG_INVALID("NEO4J_ERROR_01","invalid configuration"), + /** + * database error + * 在执行写入到数据库时抛出的异常,可能是权限异常,也可能是连接超时,或者是配置到了从节点。 + * 如果是更新操作,还会有死锁异常。具体原因根据报错信息确定,但是这与dataX无关。 + */ + DATABASE_ERROR("NEO4J_ERROR_02","database error"); + + private final String code; + private final String description; + + @Override + public String getCode() { + return code; + } + + @Override + public String getDescription() { + return description; + } + + Neo4jErrorCode(String code, String description) { + this.code = code; + this.description = description; + } +} diff --git a/neo4jwriter/src/main/resources/plugin.json b/neo4jwriter/src/main/resources/plugin.json new file mode 100644 index 00000000..3c8878f6 --- /dev/null +++ b/neo4jwriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "neo4jWriter", + "class": "com.alibaba.datax.plugin.writer.neo4jwriter.Neo4jWriter", + "description": "dataX neo4j 写插件", + "developer": "付有杰" +} \ No newline at end of file diff --git a/neo4jwriter/src/main/resources/plugin_job_template.json b/neo4jwriter/src/main/resources/plugin_job_template.json new file mode 100644 index 00000000..45bf3c88 --- /dev/null +++ b/neo4jwriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,42 @@ +{ + "uri": "neo4j://localhost:7687", + "username": "neo4j", + "password": "Test@12343", + "database": "neo4j", + "cypher": "unwind $batch as row create(p:Person) set p.pbool = row.pbool,p.pstring = row.pstring,p.plong = row.plong,p.pshort = row.pshort,p.pdouble=row.pdouble,p.pstringarr=row.pstringarr,p.plocaldate=row.plocaldate", + "batchDataVariableName": "batch", + "batchSize": "33", + "properties": [ + { + "name": "pbool", + //type 忽略大小写 + "type": "BOOLEAN" + }, + { + "name": "pstring", + "type": "STRING" + }, + { + "name": "plong", + "type": "LONG" + }, + { + "name": "pshort", + "type": "SHORT" + }, + { + "name": "pdouble", + "type": "DOUBLE" + }, + { + "name": "pstringarr", + "type": "STRING_ARRAY", + "split": "," + }, + { + "name": "plocaldate", + "type": "LOCAL_DATE", + "dateFormat": "yyyy-MM-dd" + } + ] +} \ No newline at end of file diff --git a/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/Neo4jWriterTest.java b/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/Neo4jWriterTest.java new file mode 100644 index 00000000..53c9235e --- /dev/null +++ b/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/Neo4jWriterTest.java @@ -0,0 +1,257 @@ +package com.alibaba.datax.plugin.writer; + + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.element.StringColumn; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.writer.mock.MockRecord; +import com.alibaba.datax.plugin.writer.mock.MockUtil; +import com.alibaba.datax.plugin.writer.neo4jwriter.Neo4jClient; +import com.alibaba.datax.plugin.writer.neo4jwriter.config.Neo4jProperty; +import com.alibaba.datax.plugin.writer.neo4jwriter.element.PropertyType; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.neo4j.driver.*; +import org.neo4j.driver.types.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; + +import java.io.File; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + + +public class Neo4jWriterTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(Neo4jWriterTest.class); + private static final int MOCK_NUM = 100; + private static final String CONTAINER_IMAGE = "neo4j:5.9.0"; + + private static final String CONTAINER_HOST = "neo4j-host"; + private static final int HTTP_PORT = 7474; + private static final int BOLT_PORT = 7687; + private static final String CONTAINER_NEO4J_USERNAME = "neo4j"; + private static final String CONTAINER_NEO4J_PASSWORD = "Test@12343"; + private static final URI CONTAINER_URI = URI.create("neo4j://localhost:" + BOLT_PORT); + + protected static final Network NETWORK = Network.newNetwork(); + + private GenericContainer container; + private Driver neo4jDriver; + private Session neo4jSession; + + @Before + public void init() { + DockerImageName imageName = DockerImageName.parse(CONTAINER_IMAGE); + container = + new GenericContainer<>(imageName) + .withNetwork(NETWORK) + .withNetworkAliases(CONTAINER_HOST) + .withExposedPorts(HTTP_PORT, BOLT_PORT) + .withEnv( + "NEO4J_AUTH", + CONTAINER_NEO4J_USERNAME + "/" + CONTAINER_NEO4J_PASSWORD) + .withEnv("apoc.export.file.enabled", "true") + .withEnv("apoc.import.file.enabled", "true") + .withEnv("apoc.import.file.use_neo4j_config", "true") + .withEnv("NEO4J_PLUGINS", "[\"apoc\"]") + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(CONTAINER_IMAGE))); + container.setPortBindings( + Arrays.asList( + String.format("%s:%s", HTTP_PORT, HTTP_PORT), + String.format("%s:%s", BOLT_PORT, BOLT_PORT))); + Startables.deepStart(Stream.of(container)).join(); + LOGGER.info("container started"); + Awaitility.given() + .ignoreExceptions() + .await() + .atMost(30, TimeUnit.SECONDS) + .untilAsserted(this::initConnection); + } + + @Test + public void testCreateNodeAllTypeField() { + final Result checkExists = neo4jSession.run("MATCH (p:Person) RETURN p limit 1"); + if (checkExists.hasNext()) { + neo4jSession.run("MATCH (p:Person) delete p"); + } + + Configuration configuration = Configuration.from(new File("src/test/resources/allTypeFieldNode.json")); + Neo4jClient neo4jClient = Neo4jClient.build(configuration, null); + + neo4jClient.init(); + for (int i = 0; i < MOCK_NUM; i++) { + neo4jClient.tryWrite(mockAllTypeFieldTestNode(neo4jClient.getNeo4jFields())); + } + neo4jClient.destroy(); + + + Result result = neo4jSession.run("MATCH (p:Person) return p"); + // nodes + assertTrue(result.hasNext()); + int cnt = 0; + while (result.hasNext()) { + org.neo4j.driver.Record record = result.next(); + record.get("p").get("pbool").asBoolean(); + record.get("p").get("pstring").asString(); + record.get("p").get("plong").asLong(); + record.get("p").get("pshort").asInt(); + record.get("p").get("pdouble").asDouble(); + List list = (List) record.get("p").get("pstringarr").asObject(); + record.get("p").get("plocaldate").asLocalDate(); + cnt++; + + } + assertEquals(cnt, MOCK_NUM); + } + + + /** + * 创建关系 必须先有节点 + * 所以先创建节点再模拟关系 + */ + @Test + public void testCreateRelation() { + final Result checkExists = neo4jSession.run("MATCH (p1:Person)-[r:LINK]->(p1:Person) return r limit 1"); + if (checkExists.hasNext()) { + neo4jSession.run("MATCH (p1:Person)-[r:LINK]->(p1:Person) delete r,p1,p2"); + } + + String createNodeCql = "create (p:Person) set p.id = '%s'"; + Configuration configuration = Configuration.from(new File("src/test/resources/relationship.json")); + + Neo4jClient neo4jClient = Neo4jClient.build(configuration, null); + neo4jClient.init(); + //创建节点为后续写关系做准备 + //Create nodes to prepare for subsequent write relationships + for (int i = 0; i < MOCK_NUM; i++) { + neo4jSession.run(String.format(createNodeCql, i + "start")); + neo4jSession.run(String.format(createNodeCql, i + "end")); + Record record = new MockRecord(); + record.addColumn(new StringColumn(i + "start")); + record.addColumn(new StringColumn(i + "end")); + neo4jClient.tryWrite(record); + + } + neo4jClient.destroy(); + + Result result = neo4jSession.run("MATCH (start:Person)-[r:LINK]->(end:Person) return r,start,end"); + // relationships + assertTrue(result.hasNext()); + int cnt = 0; + while (result.hasNext()) { + org.neo4j.driver.Record record = result.next(); + + Node startNode = record.get("start").asNode(); + assertTrue(startNode.hasLabel("Person")); + assertTrue(startNode.asMap().containsKey("id")); + + Node endNode = record.get("end").asNode(); + assertTrue(startNode.hasLabel("Person")); + assertTrue(endNode.asMap().containsKey("id")); + + + String name = record.get("r").type().name(); + assertEquals("RELATIONSHIP", name); + cnt++; + } + assertEquals(cnt, MOCK_NUM); + } + + /** + * neo4j中,Label和关系类型,想动态的写,需要借助于apoc函数 + */ + @Test + public void testUseApocCreateDynamicLabel() { + List dynamicLabel = new ArrayList<>(); + for (int i = 0; i < MOCK_NUM; i++) { + dynamicLabel.add("Label" + i); + } + //删除原有数据 + //remove test data if exist + //这种占位符的方式不支持批量动态写,当然可以使用union拼接,但是性能不好 + String query = "match (p:%s) return p"; + String delete = "match (p:%s) delete p"; + for (String label : dynamicLabel) { + Result result = neo4jSession.run(String.format(query, label)); + if (result.hasNext()) { + neo4jSession.run(String.format(delete, label)); + } + } + + Configuration configuration = Configuration.from(new File("src/test/resources/dynamicLabel.json")); + Neo4jClient neo4jClient = Neo4jClient.build(configuration, null); + + neo4jClient.init(); + for (int i = 0; i < dynamicLabel.size(); i++) { + Record record = new MockRecord(); + record.addColumn(new StringColumn(dynamicLabel.get(i))); + record.addColumn(new StringColumn(String.valueOf(i))); + neo4jClient.tryWrite(record); + } + neo4jClient.destroy(); + + //校验脚本的批量写入是否正确 + int cnt = 0; + for (int i = 0; i < dynamicLabel.size(); i++) { + String label = dynamicLabel.get(i); + Result result = neo4jSession.run(String.format(query, label)); + while (result.hasNext()) { + org.neo4j.driver.Record record = result.next(); + Node node = record.get("p").asNode(); + assertTrue(node.hasLabel(label)); + assertEquals(node.asMap().get("id"), i + ""); + cnt++; + } + } + assertEquals(cnt, MOCK_NUM); + + } + + + private Record mockAllTypeFieldTestNode(List neo4JProperties) { + Record mock = new MockRecord(); + for (Neo4jProperty field : neo4JProperties) { + mock.addColumn(MockUtil.mockColumnByType(PropertyType.fromStrIgnoreCase(field.getType()))); + } + return mock; + } + + @After + public void destroy() { + if (neo4jSession != null) { + neo4jSession.close(); + } + if (neo4jDriver != null) { + neo4jDriver.close(); + } + if (container != null) { + container.close(); + } + } + + private void initConnection() { + neo4jDriver = + GraphDatabase.driver( + CONTAINER_URI, + AuthTokens.basic(CONTAINER_NEO4J_USERNAME, CONTAINER_NEO4J_PASSWORD)); + neo4jSession = neo4jDriver.session(SessionConfig.forDatabase("neo4j")); + } +} diff --git a/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/mock/MockRecord.java b/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/mock/MockRecord.java new file mode 100644 index 00000000..77d3f500 --- /dev/null +++ b/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/mock/MockRecord.java @@ -0,0 +1,104 @@ +package com.alibaba.datax.plugin.writer.mock; + + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.common.element.Record; +import com.alibaba.fastjson2.JSON; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MockRecord implements Record { + private static final int RECORD_AVERGAE_COLUMN_NUMBER = 16; + + private List columns; + + private int byteSize; + + + private Map meta; + + public MockRecord() { + this.columns = new ArrayList<>(RECORD_AVERGAE_COLUMN_NUMBER); + } + + @Override + public void addColumn(Column column) { + columns.add(column); + incrByteSize(column); + } + + @Override + public Column getColumn(int i) { + if (i < 0 || i >= columns.size()) { + return null; + } + return columns.get(i); + } + + @Override + public void setColumn(int i, final Column column) { + if (i < 0) { + throw new IllegalArgumentException("不能给index小于0的column设置值"); + } + + if (i >= columns.size()) { + expandCapacity(i + 1); + } + + decrByteSize(getColumn(i)); + this.columns.set(i, column); + incrByteSize(getColumn(i)); + } + + @Override + public String toString() { + Map json = new HashMap(); + json.put("size", this.getColumnNumber()); + json.put("data", this.columns); + return JSON.toJSONString(json); + } + + @Override + public int getColumnNumber() { + return this.columns.size(); + } + + @Override + public int getByteSize() { + return byteSize; + } + + public int getMemorySize() { + throw new UnsupportedOperationException(); + } + + @Override + public void setMeta(Map meta) { + + } + + @Override + public Map getMeta() { + return null; + } + + private void decrByteSize(final Column column) { + } + + private void incrByteSize(final Column column) { + } + + private void expandCapacity(int totalSize) { + if (totalSize <= 0) { + return; + } + + int needToExpand = totalSize - columns.size(); + while (needToExpand-- > 0) { + this.columns.add(null); + } + } +} diff --git a/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/mock/MockUtil.java b/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/mock/MockUtil.java new file mode 100644 index 00000000..8f05f1e8 --- /dev/null +++ b/neo4jwriter/src/test/java/com/alibaba/datax/plugin/writer/mock/MockUtil.java @@ -0,0 +1,50 @@ +package com.alibaba.datax.plugin.writer.mock; + + +import com.alibaba.datax.common.element.*; +import com.alibaba.datax.plugin.writer.neo4jwriter.element.PropertyType; +import com.alibaba.fastjson2.JSON; + +import java.time.LocalDate; +import java.time.format.DateTimeFormatter; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; + +public class MockUtil { + + public static Column mockColumnByType(PropertyType type) { + Random random = new Random(); + switch (type) { + case SHORT: + return new StringColumn("1"); + case BOOLEAN: + return new BoolColumn(random.nextInt() % 2 == 0); + case INTEGER: + case LONG: + return new LongColumn(random.nextInt(Integer.MAX_VALUE)); + case FLOAT: + case DOUBLE: + return new DoubleColumn(random.nextDouble()); + case NULL: + return null; + case BYTE_ARRAY: + return new BytesColumn(new byte[]{(byte) (random.nextInt() % 2)}); + case LOCAL_DATE: + return new StringColumn(LocalDate.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd"))); + case MAP: + return new StringColumn(JSON.toJSONString(propmap())); + case STRING_ARRAY: + return new StringColumn("[1,1,1,1,1,1,1]"); + default: + return new StringColumn("randomStr" + random.nextInt(Integer.MAX_VALUE)); + } + } + + public static Map propmap() { + Map prop = new HashMap<>(); + prop.put("name", "neo4jWriter"); + prop.put("age", "1"); + return prop; + } +} diff --git a/neo4jwriter/src/test/resources/allTypeFieldNode.json b/neo4jwriter/src/test/resources/allTypeFieldNode.json new file mode 100644 index 00000000..6d504d79 --- /dev/null +++ b/neo4jwriter/src/test/resources/allTypeFieldNode.json @@ -0,0 +1,41 @@ +{ + "uri": "neo4j://localhost:7687", + "username":"neo4j", + "password":"Test@12343", + "database":"neo4j", + "cypher": "unwind $batch as row create(p:Person) set p.pbool = row.pbool,p.pstring = row.pstring,p.plong = row.plong,p.pshort = row.pshort,p.pdouble=row.pdouble,p.pstringarr=row.pstringarr,p.plocaldate=row.plocaldate", + "batchDataVariableName": "batch", + "batchSize": "33", + "properties": [ + { + "name": "pbool", + "type": "BOOLEAN" + }, + { + "name": "pstring", + "type": "STRING" + }, + { + "name": "plong", + "type": "LONG" + }, + { + "name": "pshort", + "type": "SHORT" + }, + { + "name": "pdouble", + "type": "DOUBLE" + }, + { + "name": "pstringarr", + "type": "STRING_ARRAY", + "split": "," + }, + { + "name": "plocaldate", + "type": "LOCAL_DATE", + "dateFormat": "yyyy-MM-dd" + } + ] +} \ No newline at end of file diff --git a/neo4jwriter/src/test/resources/dynamicLabel.json b/neo4jwriter/src/test/resources/dynamicLabel.json new file mode 100644 index 00000000..05ed3e76 --- /dev/null +++ b/neo4jwriter/src/test/resources/dynamicLabel.json @@ -0,0 +1,19 @@ +{ + "uri": "bolt://localhost:7687", + "username":"neo4j", + "password":"Test@12343", + "database":"neo4j", + "cypher": "unwind $batch as row CALL apoc.cypher.doIt( 'create (n:`' + row.Label + '`{id:$id})' ,{id: row.id} ) YIELD value RETURN 1 ", + "batchDataVariableName": "batch", + "batchSize": "33", + "properties": [ + { + "name": "Label", + "type": "string" + }, + { + "name": "id", + "type": "STRING" + } + ] +} \ No newline at end of file diff --git a/neo4jwriter/src/test/resources/relationship.json b/neo4jwriter/src/test/resources/relationship.json new file mode 100644 index 00000000..cb9bbdf4 --- /dev/null +++ b/neo4jwriter/src/test/resources/relationship.json @@ -0,0 +1,19 @@ +{ + "uri": "neo4j://localhost:7687", + "username":"neo4j", + "password":"Test@12343", + "database":"neo4j", + "cypher": "unwind $batch as row match(p1:Person) where p1.id = row.startNodeId match(p2:Person) where p2.id = row.endNodeId create (p1)-[:LINK]->(p2)", + "batchDataVariableName": "batch", + "batchSize": "33", + "properties": [ + { + "name": "startNodeId", + "type": "STRING" + }, + { + "name": "endNodeId", + "type": "STRING" + } + ] +} \ No newline at end of file diff --git a/neo4jwriter/src/test/resources/streamreader2neo4j.json b/neo4jwriter/src/test/resources/streamreader2neo4j.json new file mode 100644 index 00000000..3d543ce3 --- /dev/null +++ b/neo4jwriter/src/test/resources/streamreader2neo4j.json @@ -0,0 +1,51 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "sliceRecordCount": 10, + "column": [ + { + "type": "string", + "value": "StreamReader" + }, + { + "type": "string", + "value": "1997" + } + ] + } + }, + "writer": { + "name": "neo4jWriter", + "parameter": { + "uri": "bolt://localhost:7687", + "username":"neo4j", + "password":"Test@12343", + "database":"neo4j", + "cypher": "unwind $batch as row CALL apoc.cypher.doIt( 'create (n:`' + row.Label + '`{id:$id})' ,{id: row.id} ) YIELD value RETURN 1 ", + "batchDataVariableName": "batch", + "batchSize": "3", + "properties": [ + { + "name": "Label", + "type": "string" + }, + { + "name": "id", + "type": "STRING" + } + ] + } + } + } + ], + "setting": { + "speed": { + "channel": 5 + } + } + } +} \ No newline at end of file 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 06d53108..d7b8f2ed 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 @@ -1,6 +1,7 @@ package com.alibaba.datax.plugin.reader.oceanbasev10reader.util; import com.alibaba.datax.common.element.*; +import com.alibaba.datax.plugin.rdbms.reader.util.ObVersion; import com.alibaba.datax.plugin.rdbms.reader.util.SingleTableSplitUtil; import com.alibaba.datax.plugin.rdbms.util.DBUtil; import com.alibaba.datax.plugin.rdbms.util.DataBaseType; 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 2929658a..ad165d99 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,6 +3,7 @@ package com.alibaba.datax.plugin.reader.oceanbasev10reader.util; 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.ObVersion; 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; diff --git a/oceanbasev10writer/pom.xml b/oceanbasev10writer/pom.xml index cbe19732..11997a1e 100644 --- a/oceanbasev10writer/pom.xml +++ b/oceanbasev10writer/pom.xml @@ -64,8 +64,16 @@ + + com.oceanbase + shade-ob-partition-calculator + 1.0-SNAPSHOT + system + ${pom.basedir}/src/main/libs/shade-ob-partition-calculator-1.0-SNAPSHOT.jar + - + + log4j log4j 1.2.16 diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/Config.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/Config.java index 9fa3cd9a..6776196b 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/Config.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/Config.java @@ -6,6 +6,7 @@ public interface Config { double DEFAULT_MEMSTORE_THRESHOLD = 0.9d; + double DEFAULT_SLOW_MEMSTORE_THRESHOLD = 0.75d; String MEMSTORE_CHECK_INTERVAL_SECOND = "memstoreCheckIntervalSecond"; long DEFAULT_MEMSTORE_CHECK_INTERVAL_SECOND = 30; 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 3bcc1019..06292db5 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 @@ -86,6 +86,7 @@ public class OceanBaseV10Writer extends Writer { if (tableNumber == 1) { this.commonJob.prepare(this.originalConfig); final String version = fetchServerVersion(originalConfig); + ObWriterUtils.setObVersion(version); originalConfig.set(Config.OB_VERSION, version); } @@ -187,8 +188,9 @@ public class OceanBaseV10Writer extends Writer { } private String fetchServerVersion(Configuration config) { - final String fetchVersionSql = "show variables like 'version'"; - return DbUtils.fetchSingleValueWithRetry(config, fetchVersionSql); + final String fetchVersionSql = "show variables like 'version_comment'"; + String versionComment = DbUtils.fetchSingleValueWithRetry(config, fetchVersionSql); + return versionComment.split(" ")[1]; } private void checkCompatibleMode(Configuration configure) { diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/AbstractConnHolder.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/AbstractConnHolder.java new file mode 100644 index 00000000..c8630cd0 --- /dev/null +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/AbstractConnHolder.java @@ -0,0 +1,48 @@ +package com.alibaba.datax.plugin.writer.oceanbasev10writer.ext; + +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.util.DBUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; + +public abstract class AbstractConnHolder { + private static final Logger LOG = LoggerFactory.getLogger(AbstractConnHolder.class); + + protected final Configuration config; + protected Connection conn; + + public AbstractConnHolder(Configuration config) { + this.config = config; + } + + public abstract Connection initConnection(); + + public Configuration getConfig() { + return config; + } + + public Connection getConn() { + try { + if (conn != null && !conn.isClosed()) { + return conn; + } + } catch (Exception e) { + LOG.warn("judge connection is closed or not failed. try to reconnect.", e); + } + return reconnect(); + } + + public Connection reconnect() { + DBUtil.closeDBResources(null, conn); + return initConnection(); + } + + public abstract String getJdbcUrl(); + + public abstract String getUserName(); + + public abstract void destroy(); +} diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/DataBaseWriterBuffer.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/DataBaseWriterBuffer.java index 53172495..b8ae259a 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/DataBaseWriterBuffer.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/DataBaseWriterBuffer.java @@ -23,7 +23,7 @@ import org.slf4j.LoggerFactory; public class DataBaseWriterBuffer { private static final Logger LOG = LoggerFactory.getLogger(DataBaseWriterBuffer.class); - private final ConnHolder connHolder; + private final AbstractConnHolder connHolder; private final String dbName; private Map> tableBuffer = new HashMap>(); private long lastCheckMemstoreTime; @@ -33,7 +33,7 @@ public class DataBaseWriterBuffer { this.dbName=dbName; } - public ConnHolder getConnHolder(){ + public AbstractConnHolder getConnHolder(){ return connHolder; } diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/OCJConnHolder.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/OCJConnHolder.java index 10de5615..262fb1cb 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/OCJConnHolder.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/OCJConnHolder.java @@ -3,15 +3,13 @@ package com.alibaba.datax.plugin.writer.oceanbasev10writer.ext; import java.sql.Connection; import com.alibaba.datax.common.util.Configuration; -import com.alibaba.datax.plugin.rdbms.util.DBUtil; -import com.alibaba.datax.plugin.rdbms.util.DataBaseType; /** * wrap oceanbase java client * @author oceanbase */ -public class OCJConnHolder extends ConnHolder { +public class OCJConnHolder extends AbstractConnHolder { private ServerConnectInfo connectInfo; private String dataSourceKey; @@ -28,17 +26,6 @@ public class OCJConnHolder extends ConnHolder { return conn; } - @Override - public Connection reconnect() { - DBUtil.closeDBResources(null, conn); - return initConnection(); - } - - @Override - public Connection getConn() { - return conn; - } - @Override public String getJdbcUrl() { return connectInfo.jdbcUrl; diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/ObClientConnHolder.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/ObClientConnHolder.java index 8ff53039..ac75d359 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/ObClientConnHolder.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/ObClientConnHolder.java @@ -16,7 +16,7 @@ import com.alibaba.datax.plugin.writer.oceanbasev10writer.util.ObWriterUtils; * @author oceanbase * */ -public class ObClientConnHolder extends ConnHolder { +public class ObClientConnHolder extends AbstractConnHolder { private final String jdbcUrl; private final String userName; private final String password; diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/ServerConnectInfo.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/ServerConnectInfo.java index b0611642..fe8889e1 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/ServerConnectInfo.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/ext/ServerConnectInfo.java @@ -1,5 +1,7 @@ package com.alibaba.datax.plugin.writer.oceanbasev10writer.ext; +import static org.apache.commons.lang3.StringUtils.EMPTY; + import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -12,40 +14,19 @@ public class ServerConnectInfo { public String databaseName; public String ipPort; public String jdbcUrl; + public boolean publicCloud; + /** + * + * @param jdbcUrl format is jdbc:oceanbase//ip:port + * @param username format is cluster:tenant:username or username@tenant#cluster or user@tenant or user + * @param password + */ public ServerConnectInfo(final String jdbcUrl, final String username, final String password) { - if (jdbcUrl.startsWith(com.alibaba.datax.plugin.rdbms.writer.Constant.OB10_SPLIT_STRING)) { - String[] ss = jdbcUrl.split(com.alibaba.datax.plugin.rdbms.writer.Constant.OB10_SPLIT_STRING_PATTERN); - if (ss.length != 3) { - throw new RuntimeException("jdbc url format is not correct: " + jdbcUrl); - } - this.userName = username; - this.clusterName = ss[1].trim().split(":")[0]; - this.tenantName = ss[1].trim().split(":")[1]; - this.jdbcUrl = ss[2].replace("jdbc:mysql:", "jdbc:oceanbase:"); - } else { - this.jdbcUrl = jdbcUrl.replace("jdbc:mysql:", "jdbc:oceanbase:"); - if (username.contains("@") && username.contains("#")) { - this.userName = username.substring(0, username.indexOf("@")); - this.tenantName = username.substring(username.indexOf("@") + 1, username.indexOf("#")); - this.clusterName = username.substring(username.indexOf("#") + 1); - } else if (username.contains(":")) { - String[] config = username.split(":"); - if (config.length != 3) { - throw new RuntimeException ("username format is not correct: " + username); - } - this.clusterName = config[0]; - this.tenantName = config[1]; - this.userName = config[2]; - } else { - this.clusterName = null; - this.tenantName = null; - this.userName = username; - } - } - + this.jdbcUrl = jdbcUrl; this.password = password; parseJdbcUrl(jdbcUrl); + parseFullUserName(username); } private void parseJdbcUrl(final String jdbcUrl) { @@ -56,11 +37,42 @@ public class ServerConnectInfo { String dbName = matcher.group(2); this.ipPort = ipPort; this.databaseName = dbName; + this.publicCloud = ipPort.split(":")[0].endsWith("aliyuncs.com"); } else { throw new RuntimeException("Invalid argument:" + jdbcUrl); } } + private void parseFullUserName(final String fullUserName) { + int tenantIndex = fullUserName.indexOf("@"); + int clusterIndex = fullUserName.indexOf("#"); + if (fullUserName.contains(":") && tenantIndex < 0) { + String[] names = fullUserName.split(":"); + if (names.length != 3) { + throw new RuntimeException("invalid argument: " + fullUserName); + } else { + this.clusterName = names[0]; + this.tenantName = names[1]; + this.userName = names[2]; + } + } else if (!publicCloud || tenantIndex < 0) { + this.userName = tenantIndex < 0 ? fullUserName : fullUserName.substring(0, tenantIndex); + this.clusterName = clusterIndex < 0 ? EMPTY : fullUserName.substring(clusterIndex + 1); + this.tenantName = tenantIndex < 0 ? EMPTY : fullUserName.substring(tenantIndex + 1, clusterIndex); + } else { + // If in public cloud, the username with format user@tenant#cluster should be parsed, otherwise, connection can't be created. + this.userName = fullUserName.substring(0, tenantIndex); + if (clusterIndex > tenantIndex) { + this.tenantName = fullUserName.substring(tenantIndex + 1, clusterIndex); + this.clusterName = fullUserName.substring(clusterIndex + 1); + } else { + this.tenantName = fullUserName.substring(tenantIndex + 1); + this.clusterName = EMPTY; + } + } + } + + @Override public String toString() { StringBuffer strBuffer = new StringBuffer(); return strBuffer.append("clusterName:").append(clusterName).append(", tenantName:").append(tenantName) @@ -69,11 +81,18 @@ public class ServerConnectInfo { } public String getFullUserName() { - StringBuilder builder = new StringBuilder(userName); - if (tenantName != null && clusterName != null) { - builder.append("@").append(tenantName).append("#").append(clusterName); + StringBuilder builder = new StringBuilder(); + builder.append(userName); + if (!EMPTY.equals(tenantName)) { + builder.append("@").append(tenantName); } + if (!EMPTY.equals(clusterName)) { + builder.append("#").append(clusterName); + } + if (EMPTY.equals(this.clusterName) && EMPTY.equals(this.tenantName)) { + return this.userName; + } return builder.toString(); } } diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/IObPartCalculator.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/IObPartCalculator.java new file mode 100644 index 00000000..b49ade02 --- /dev/null +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/IObPartCalculator.java @@ -0,0 +1,19 @@ +package com.alibaba.datax.plugin.writer.oceanbasev10writer.part; + +import com.alibaba.datax.common.element.Record; + +/** + * @author cjyyz + * @date 2023/02/07 + * @since + */ +public interface IObPartCalculator { + + /** + * 计算 Partition Id + * + * @param record + * @return Long + */ + Long calculate(Record record); +} \ No newline at end of file diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/ObPartitionCalculatorV1.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/ObPartitionCalculatorV1.java new file mode 100644 index 00000000..96985588 --- /dev/null +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/ObPartitionCalculatorV1.java @@ -0,0 +1,109 @@ +package com.alibaba.datax.plugin.writer.oceanbasev10writer.part; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ServerConnectInfo; +import com.alipay.oceanbase.obproxy.data.TableEntryKey; +import com.alipay.oceanbase.obproxy.util.ObPartitionIdCalculator; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * OceanBase 1.x和2.x的分区计算 + * + * @author cjyyz + * @date 2023/02/07 + * @since + */ +public class ObPartitionCalculatorV1 implements IObPartCalculator { + + private static final Logger LOG = LoggerFactory.getLogger(ObPartitionCalculatorV1.class); + + /** + * 分区键的位置 + */ + private List partIndexes; + + /** + * 表的全部字段名 + */ + private List columnNames; + + /** + * ocj partition calculator + */ + private ObPartitionIdCalculator calculator; + + /** + * @param connectInfo + * @param table + * @param columns + */ + public ObPartitionCalculatorV1(ServerConnectInfo connectInfo, String table, List columns) { + + initCalculator(connectInfo, table); + + if (Objects.isNull(calculator)) { + LOG.warn("partCalculator is null"); + return; + } + + this.partIndexes = new ArrayList<>(columns.size()); + this.columnNames = new ArrayList<>(columns); + + for (int i = 0; i < columns.size(); ++i) { + String columnName = columns.get(i); + if (calculator.isPartitionKeyColumn(columnName)) { + LOG.info(columnName + " is partition key."); + partIndexes.add(i); + } + } + } + + /** + * @param record + * @return Long + */ + @Override + public Long calculate(Record record) { + if (Objects.isNull(calculator)) { + return null; + } + + for (Integer i : partIndexes) { + calculator.addColumn(columnNames.get(i), record.getColumn(i).asString()); + } + return calculator.calculate(); + } + + /** + * @param connectInfo + * @param table + */ + private void initCalculator(ServerConnectInfo connectInfo, String table) { + + LOG.info(String.format("create tableEntryKey with clusterName %s, tenantName %s, databaseName %s, tableName %s", + connectInfo.clusterName, connectInfo.tenantName, connectInfo.databaseName, table)); + TableEntryKey tableEntryKey = new TableEntryKey(connectInfo.clusterName, connectInfo.tenantName, + connectInfo.databaseName, table); + + int retry = 0; + + do { + try { + if (retry > 0) { + TimeUnit.SECONDS.sleep(1); + LOG.info("retry create new part calculator {} times", retry); + } + LOG.info("create partCalculator with address: " + connectInfo.ipPort); + calculator = new ObPartitionIdCalculator(connectInfo.ipPort, tableEntryKey); + } catch (Exception ex) { + ++retry; + LOG.warn("create new part calculator failed, retry: {}", ex.getMessage()); + } + } while (calculator == null && retry < 3); + } +} \ No newline at end of file diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/ObPartitionCalculatorV2.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/ObPartitionCalculatorV2.java new file mode 100644 index 00000000..11b7b25c --- /dev/null +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/part/ObPartitionCalculatorV2.java @@ -0,0 +1,169 @@ +package com.alibaba.datax.plugin.writer.oceanbasev10writer.part; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.plugin.rdbms.util.DBUtil; +import com.alibaba.datax.plugin.rdbms.util.DataBaseType; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ServerConnectInfo; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.util.DbUtils; +import com.oceanbase.partition.calculator.ObPartIdCalculator; +import com.oceanbase.partition.calculator.enums.ObPartLevel; +import com.oceanbase.partition.calculator.enums.ObServerMode; +import com.oceanbase.partition.calculator.helper.TableEntryExtractor; +import com.oceanbase.partition.calculator.model.TableEntry; +import com.oceanbase.partition.calculator.model.TableEntryKey; +import com.oceanbase.partition.calculator.model.Version; +import com.oceanbase.partition.metadata.desc.ObPartColumn; +import com.oceanbase.partition.metadata.desc.ObTablePart; +import java.sql.Connection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * OceanBase 3.x和4.x的分区计算 + * + * @author cjyyz + * @date 2023/02/07 + * @since + */ +public class ObPartitionCalculatorV2 implements IObPartCalculator { + + private static final Logger LOG = LoggerFactory.getLogger(ObPartitionCalculatorV2.class); + + /** + * OB的模式以及版本信息 + */ + private ObServerMode mode; + + /** + * ob-partition-calculator 分区计算组件 + */ + private ObPartIdCalculator calculator; + + /** + * 记录columns的字段名和在record中的位置。 + * 当目标表结构的分区键是生成列时,calculator 需要从改结构中获取到生成列所依赖的字段的值 + * e.g. + * create table t1 ( + * c1 varchar(20), + * c2 varchar(20) generated always as (substr(`c1`,1,8)) + * )partition by key(c2) partitions 5 + * + * 此时,columnNameIndexMap包含的元素是 c1:0 + * 需要将c1字段的值从columnNameIndexMap中添加到{@link com.oceanbase.partition.calculator.ObPartIdCalculator#getRefColumnValues()} + */ + private Map columnNameIndexMap; + + /** + * @param connectInfo + * @param table + * @param mode + */ + public ObPartitionCalculatorV2(ServerConnectInfo connectInfo, String table, ObServerMode mode, List columns) { + this.mode = mode; + this.columnNameIndexMap = new HashMap<>(); + for (int i = 0; i < columns.size(); i++) { + columnNameIndexMap.put(columns.get(i).toLowerCase(), i); + } + initCalculator(connectInfo, table); + } + + /** + * @param record + * @return Long + */ + @Override + public Long calculate(Record record) { + if (Objects.isNull(calculator)) { + return null; + } + if (!calculator.getTableEntry().isPartitionTable()) { + return 0L; + } + return calculator.calculatePartId(filterNullableColumns(record)); + } + + /** + * 初始化分区计算组件 + * + * @param connectInfo + * @param table + */ + private void initCalculator(ServerConnectInfo connectInfo, String table) { + TableEntryKey tableEntryKey = new TableEntryKey(connectInfo.clusterName, connectInfo.tenantName, connectInfo.databaseName, table, mode); + boolean subsequentFromV4 = !mode.getVersion().isOlderThan(new Version("4.0.0.0")); + try { + TableEntry tableEntry; + try (Connection conn = getConnection(connectInfo, subsequentFromV4)){ + TableEntryExtractor extractor = new TableEntryExtractor(); + tableEntry = extractor.queryTableEntry(conn, tableEntryKey,subsequentFromV4); + } + this.calculator = new ObPartIdCalculator(false, tableEntry, subsequentFromV4); + } catch (Exception e) { + LOG.warn("create new part calculator failed. reason: {}", e.getMessage()); + } + } + + private Connection getConnection(ServerConnectInfo connectInfo, boolean subsequentFromV4) throws Exception { + // OceanBase 4.0.0.0及之后版本均使用业务租户连接计算分区 + if (subsequentFromV4) { + return DBUtil.getConnection(DataBaseType.OceanBase, connectInfo.jdbcUrl, connectInfo.getFullUserName(), connectInfo.password); + } + // OceanBase 4.0.0.0之前版本使用sys租户连接计算分区 + return DbUtils.buildSysConn(connectInfo.jdbcUrl, connectInfo.clusterName); + } + + /** + * 只选择分区字段值传入分区计算组件 + * + * @param record + * @return Object[] + */ + private Object[] filterNullableColumns(Record record) { + final ObTablePart tablePart = calculator.getTableEntry().getTablePart(); + + final Object[] filteredRecords = new Object[record.getColumnNumber()]; + + if (tablePart.getLevel().getIndex() > ObPartLevel.LEVEL_ZERO.getIndex()) { + // 从record中添加非生成列的一级分区值到filteredRecords数组中 + for (ObPartColumn partColumn : tablePart.getPartColumns()) { + if (partColumn.getColumnExpr() == null) { + int metaIndex = partColumn.getColumnIndex(); + String columnName = partColumn.getColumnName().toLowerCase(); + int idxInRecord = columnNameIndexMap.get(columnName); + filteredRecords[metaIndex] = record.getColumn(idxInRecord).asString(); + } + + } + // 从record中添加生成列的一级分区值到calculator的redColumnMap中,ObTablePart.getRefPartColumns中的字段名均为小写 + for (ObPartColumn partColumn : tablePart.getRefPartColumns()) { + String columnName = partColumn.getColumnName(); + int index = columnNameIndexMap.get(columnName); + calculator.addRefColumn(columnName, record.getColumn(index).asString()); + } + } + + if (tablePart.getLevel().getIndex() >= ObPartLevel.LEVEL_TWO.getIndex()) { + // 从record中添加非生成列的二级分区值到filteredRecords数组中 + for (ObPartColumn partColumn : tablePart.getSubPartColumns()) { + if (partColumn.getColumnExpr() == null) { + int metaIndex = partColumn.getColumnIndex(); + String columnName = partColumn.getColumnName().toLowerCase(); + int idxInRecord = columnNameIndexMap.get(columnName); + filteredRecords[metaIndex] = record.getColumn(idxInRecord).asString(); + } + + } + // 从record中添加生成列的二级分区值到calculator的redColumnMap中,ObTablePart.getRefSubPartColumns中的字段名均为小写 + for (ObPartColumn partColumn : tablePart.getRefSubPartColumns()) { + String columnName = partColumn.getColumnName(); + int index = columnNameIndexMap.get(columnName); + calculator.addRefColumn(columnName, record.getColumn(index).asString()); + } + } + return filteredRecords; + } +} \ No newline at end of file diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/ConcurrentTableWriterTask.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/ConcurrentTableWriterTask.java index 82b16923..0ad3a1ed 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/ConcurrentTableWriterTask.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/ConcurrentTableWriterTask.java @@ -1,6 +1,5 @@ package com.alibaba.datax.plugin.writer.oceanbasev10writer.task; -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; @@ -11,16 +10,14 @@ import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; import com.alibaba.datax.plugin.rdbms.util.DataBaseType; import com.alibaba.datax.plugin.rdbms.writer.CommonRdbmsWriter; import com.alibaba.datax.plugin.writer.oceanbasev10writer.Config; -import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ConnHolder; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.AbstractConnHolder; import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ObClientConnHolder; import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ServerConnectInfo; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.part.IObPartCalculator; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.part.ObPartitionCalculatorV1; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.part.ObPartitionCalculatorV2; import com.alibaba.datax.plugin.writer.oceanbasev10writer.util.ObWriterUtils; -import com.alipay.oceanbase.obproxy.data.TableEntryKey; -import com.alipay.oceanbase.obproxy.util.ObPartitionIdCalculator; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +import com.oceanbase.partition.calculator.enums.ObServerMode; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; @@ -35,8 +32,12 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; - -//import java.sql.PreparedStatement; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static com.alibaba.datax.plugin.writer.oceanbasev10writer.Config.DEFAULT_SLOW_MEMSTORE_THRESHOLD; +import static com.alibaba.datax.plugin.writer.oceanbasev10writer.util.ObWriterUtils.LoadMode.FAST; +import static com.alibaba.datax.plugin.writer.oceanbasev10writer.util.ObWriterUtils.LoadMode.PAUSE; +import static com.alibaba.datax.plugin.writer.oceanbasev10writer.util.ObWriterUtils.LoadMode.SLOW; public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { private static final Logger LOG = LoggerFactory.getLogger(ConcurrentTableWriterTask.class); @@ -47,41 +48,31 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { private long memstoreCheckIntervalSecond = Config.DEFAULT_MEMSTORE_CHECK_INTERVAL_SECOND; // 最后一次检查 private long lastCheckMemstoreTime; + + private volatile ObWriterUtils.LoadMode loadMode = FAST; private static AtomicLong totalTask = new AtomicLong(0); private long taskId = -1; - private AtomicBoolean isMemStoreFull = new AtomicBoolean(false); - private ConnHolder checkConnHolder; + private HashMap> groupInsertValues; + private IObPartCalculator obPartCalculator; + private ConcurrentTableWriter concurrentWriter = null; + private AbstractConnHolder connHolder; + private boolean allTaskInQueue = false; + private Lock lock = new ReentrantLock(); + private Condition condition = lock.newCondition(); + private long startTime; + private String obWriteMode = "update"; + private boolean isOracleCompatibleMode = false; + private String obUpdateColumns = null; + private String dbName; + private int calPartFailedCount = 0; - public ConcurrentTableWriterTask(DataBaseType dataBaseType) { + public ConcurrentTableWriterTask(DataBaseType dataBaseType) { super(dataBaseType); taskId = totalTask.getAndIncrement(); } - private ObPartitionIdCalculator partCalculator = null; - - private HashMap> groupInsertValues; - List unknownPartRecords = new ArrayList(); -// private List unknownPartRecords; - private List partitionKeyIndexes; - - private ConcurrentTableWriter concurrentWriter = null; - - private ConnHolder connHolder; - - private boolean allTaskInQueue = false; - - private Lock lock = new ReentrantLock(); - private Condition condition = lock.newCondition(); - - private long startTime; - private String obWriteMode = "update"; - private boolean isOracleCompatibleMode = false; - private String obUpdateColumns = null; - private List> deleteColPos; - private String dbName; - @Override public void init(Configuration config) { super.init(config); @@ -95,15 +86,11 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { this.memstoreThreshold = config.getDouble(Config.MEMSTORE_THRESHOLD, Config.DEFAULT_MEMSTORE_THRESHOLD); this.memstoreCheckIntervalSecond = config.getLong(Config.MEMSTORE_CHECK_INTERVAL_SECOND, Config.DEFAULT_MEMSTORE_CHECK_INTERVAL_SECOND); - this.isOracleCompatibleMode = ObWriterUtils.isOracleMode(); - LOG.info("configure url is unavailable, use obclient for connections."); - this.checkConnHolder = new ObClientConnHolder(config, connectInfo.jdbcUrl, + this.connHolder = new ObClientConnHolder(config, connectInfo.jdbcUrl, connectInfo.getFullUserName(), connectInfo.password); - this.connHolder = new ObClientConnHolder(config, connectInfo.jdbcUrl, - connectInfo.getFullUserName(), connectInfo.password); - checkConnHolder.initConnection(); - if (isOracleCompatibleMode) { + this.isOracleCompatibleMode = ObWriterUtils.isOracleMode(); + if (isOracleCompatibleMode) { connectInfo.databaseName = connectInfo.databaseName.toUpperCase(); //在转义的情况下不翻译 if (!(table.startsWith("\"") && table.endsWith("\""))) { @@ -115,43 +102,36 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { } if (config.getBool(Config.USE_PART_CALCULATOR, Config.DEFAULT_USE_PART_CALCULATOR)) { - initPartCalculator(connectInfo); + this.obPartCalculator = createPartitionCalculator(connectInfo, ObServerMode.from(config.getString(Config.OB_COMPATIBLE_MODE), config.getString(Config.OB_VERSION))); } else { LOG.info("Disable partition calculation feature."); } - obUpdateColumns = config.getString(Config.OB_UPDATE_COLUMNS, null); - groupInsertValues = new HashMap>(); - partitionKeyIndexes = new ArrayList(); - rewriteSql(); + obUpdateColumns = config.getString(Config.OB_UPDATE_COLUMNS, null); + groupInsertValues = new HashMap>(); + rewriteSql(); - if (null == concurrentWriter) { - concurrentWriter = new ConcurrentTableWriter(config, connectInfo, writeRecordSql); - allTaskInQueue = false; - } - } + if (null == concurrentWriter) { + concurrentWriter = new ConcurrentTableWriter(config, connectInfo, writeRecordSql); + allTaskInQueue = false; + } + } - private void initPartCalculator(ServerConnectInfo connectInfo) { - int retry = 0; - LOG.info(String.format("create tableEntryKey with clusterName %s, tenantName %s, databaseName %s, tableName %s", - connectInfo.clusterName, connectInfo.tenantName, connectInfo.databaseName, table)); - TableEntryKey tableEntryKey = new TableEntryKey(connectInfo.clusterName, connectInfo.tenantName, - connectInfo.databaseName, table); - do { - try { - if (retry > 0) { - int sleep = retry > 8 ? 500 : (1 << retry); - TimeUnit.SECONDS.sleep(sleep); - LOG.info("retry create new part calculator, the {} times", retry); - } - LOG.info("create partCalculator with address: " + connectInfo.ipPort); - partCalculator = new ObPartitionIdCalculator(connectInfo.ipPort, tableEntryKey); - } catch (Exception ex) { - ++retry; - LOG.warn("create new part calculator failed, retry {}: {}", retry, ex.getMessage()); - } - } while (partCalculator == null && retry < 3); // try 3 times - } + /** + * 创建需要的分区计算组件 + * + * @param connectInfo + * @return + */ + private IObPartCalculator createPartitionCalculator(ServerConnectInfo connectInfo, ObServerMode obServerMode) { + if (obServerMode.isSubsequentFrom("3.0.0.0")) { + LOG.info("oceanbase version is {}, use ob-partition-calculator to calculate partition Id.", obServerMode.getVersion()); + return new ObPartitionCalculatorV2(connectInfo, table, obServerMode, columns); + } + + LOG.info("oceanbase version is {}, use ocj to calculate partition Id.", obServerMode.getVersion()); + return new ObPartitionCalculatorV1(connectInfo, table, columns); + } public boolean isFinished() { return allTaskInQueue && concurrentWriter.checkFinish(); @@ -174,43 +154,18 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { if (isOracleCompatibleMode && obWriteMode.equalsIgnoreCase("update")) { // change obWriteMode to insert so the insert statement will be generated. obWriteMode = "insert"; - deleteColPos = ObWriterUtils.buildDeleteSql(conn, dbName, table, columns); } this.writeRecordSql = ObWriterUtils.buildWriteSql(table, columns, conn, obWriteMode, obUpdateColumns); LOG.info("writeRecordSql :{}", this.writeRecordSql); } - + + @Override public void prepare(Configuration writerSliceConfig) { super.prepare(writerSliceConfig); - calPartitionKeyIndex(partitionKeyIndexes); concurrentWriter.start(); } - private void calPartitionKeyIndex(List partKeyIndexes) { - partKeyIndexes.clear(); - if (null == partCalculator) { - LOG.error("partCalculator is null"); - return; - } - for (int i = 0; i < columns.size(); ++i) { - if (partCalculator.isPartitionKeyColumn(columns.get(i))) { - LOG.info(columns.get(i) + " is partition key."); - partKeyIndexes.add(i); - } - } - } - - private Long calPartitionId(List partKeyIndexes, Record record) { - if (partCalculator == null) { - return null; - } - for (Integer i : partKeyIndexes) { - partCalculator.addColumn(columns.get(i), record.getColumn(i).asString()); - } - return partCalculator.calculate(); - } - - @Override + @Override public void startWriteWithConnection(RecordReceiver recordReceiver, TaskPluginCollector taskPluginCollector, Connection connection) { this.taskPluginCollector = taskPluginCollector; @@ -271,21 +226,6 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { return fillPreparedStatement(preparedStatement, record); } - public PreparedStatement fillStatementIndex(PreparedStatement preparedStatement, - int prepIdx, int columnIndex, Column column) throws SQLException { - int columnSqltype = this.resultSetMetaData.getMiddle().get(columnIndex); - String typeName = this.resultSetMetaData.getRight().get(columnIndex); - return fillPreparedStatementColumnType(preparedStatement, prepIdx, columnSqltype, typeName, column); - } - - public void collectDirtyRecord(Record record, SQLException e) { - taskPluginCollector.collectDirtyRecord(record, e); - } - - public void insertOneRecord(Connection connection, List buffer) { - doOneInsert(connection, buffer); - } - private void addLeftRecords() { //不需要刷新Cache,已经是最后一批数据了 for (List groupValues : groupInsertValues.values()) { @@ -293,17 +233,16 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { addRecordsToWriteQueue(groupValues); } } - if (unknownPartRecords.size() > 0) { - addRecordsToWriteQueue(unknownPartRecords); - } } private void addRecordToCache(final Record record) { Long partId =null; try { - partId = calPartitionId(partitionKeyIndexes, record); + partId = obPartCalculator == null ? Long.MAX_VALUE : obPartCalculator.calculate(record); } catch (Exception e1) { - LOG.warn("fail to get partition id: " + e1.getMessage() + ", record: " + record); + if (calPartFailedCount++ < 10) { + LOG.warn("fail to get partition id: " + e1.getMessage() + ", record: " + record); + } } if (partId == null) { @@ -311,24 +250,11 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { partId = Long.MAX_VALUE; } - if (partId != null) { - List groupValues = groupInsertValues.get(partId); - if (groupValues == null) { - groupValues = new ArrayList(batchSize); - groupInsertValues.put(partId, groupValues); - } - groupValues.add(record); - if (groupValues.size() >= batchSize) { - groupValues = addRecordsToWriteQueue(groupValues); - groupInsertValues.put(partId, groupValues); - } - } else { - LOG.debug("add unknown part record {}", record); - unknownPartRecords.add(record); - if (unknownPartRecords.size() >= batchSize) { - unknownPartRecords = addRecordsToWriteQueue(unknownPartRecords); - } - + List groupValues = groupInsertValues.computeIfAbsent(partId, k -> new ArrayList(batchSize)); + groupValues.add(record); + if (groupValues.size() >= batchSize) { + groupValues = addRecordsToWriteQueue(groupValues); + groupInsertValues.put(partId, groupValues); } } @@ -354,15 +280,25 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { return new ArrayList(batchSize); } private void checkMemStore() { - Connection checkConn = checkConnHolder.reconnect(); + Connection checkConn = connHolder.getConn(); + try { + if (checkConn == null || checkConn.isClosed()) { + checkConn = connHolder.reconnect(); + } + }catch (Exception e) { + LOG.warn("Check connection is unusable"); + } + long now = System.currentTimeMillis(); if (now - lastCheckMemstoreTime < 1000 * memstoreCheckIntervalSecond) { return; } - boolean isFull = ObWriterUtils.isMemstoreFull(checkConn, memstoreThreshold); - this.isMemStoreFull.set(isFull); - if (isFull) { - LOG.warn("OB memstore is full,sleep 30 seconds, threshold=" + memstoreThreshold); + double memUsedRatio = ObWriterUtils.queryMemUsedRatio(checkConn); + if (memUsedRatio >= DEFAULT_SLOW_MEMSTORE_THRESHOLD) { + this.loadMode = memUsedRatio >= memstoreThreshold ? PAUSE : SLOW; + LOG.info("Memstore used ration is {}. Load data {}", memUsedRatio, loadMode.name()); + }else { + this.loadMode = FAST; } lastCheckMemstoreTime = now; } @@ -370,21 +306,23 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { public boolean isMemStoreFull() { return isMemStoreFull.get(); } - - public void printEveryTime() { - long cost = System.currentTimeMillis() - startTime; - if (cost > 10000) { //10s - print(); - startTime = System.currentTimeMillis(); - } + + public boolean isShouldPause() { + return this.loadMode.equals(PAUSE); + } + + public boolean isShouldSlow() { + return this.loadMode.equals(SLOW); } public void print() { - LOG.debug("Statistic total task {}, finished {}, queue Size {}", - concurrentWriter.getTotalTaskCount(), - concurrentWriter.getFinishTaskCount(), - concurrentWriter.getTaskQueueSize()); - concurrentWriter.printStatistics(); + if (LOG.isDebugEnabled()) { + LOG.debug("Statistic total task {}, finished {}, queue Size {}", + concurrentWriter.getTotalTaskCount(), + concurrentWriter.getFinishTaskCount(), + concurrentWriter.getTaskQueueSize()); + concurrentWriter.printStatistics(); + } } public void waitTaskFinish() { @@ -417,8 +355,6 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { } // 把本级持有的conn关闭掉 DBUtil.closeDBResources(null, connHolder.getConn()); - DBUtil.closeDBResources(null, checkConnHolder.getConn()); - checkConnHolder.destroy(); super.destroy(writerSliceConfig); } @@ -469,7 +405,7 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { public synchronized void start() { for (int i = 0; i < threadCount; ++i) { LOG.info("start {} insert task.", (i+1)); - InsertTask insertTask = new InsertTask(taskId, queue, config, connectInfo, rewriteRecordSql, deleteColPos); + InsertTask insertTask = new InsertTask(taskId, queue, config, connectInfo, rewriteRecordSql); insertTask.setWriterTask(ConcurrentTableWriterTask.this); insertTask.setWriter(this); insertTasks.add(insertTask); @@ -495,7 +431,7 @@ public class ConcurrentTableWriterTask extends CommonRdbmsWriter.Task { public void addBatchRecords(final List records) throws InterruptedException { boolean isSucc = false; while (!isSucc) { - isSucc = queue.offer(records, 5, TimeUnit.SECONDS); + isSucc = queue.offer(records, 5, TimeUnit.MILLISECONDS); checkMemStore(); } totalTaskCount.incrementAndGet(); diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/InsertTask.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/InsertTask.java index 968908ca..df80cf7f 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/InsertTask.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/InsertTask.java @@ -1,286 +1,204 @@ package com.alibaba.datax.plugin.writer.oceanbasev10writer.task; -import java.sql.Connection; -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.TimeUnit; - -import com.alibaba.datax.common.exception.DataXException; -import com.alibaba.datax.plugin.rdbms.util.DBUtil; -import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; -import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ObClientConnHolder; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import com.alibaba.datax.common.element.Record; import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.util.DBUtil; import com.alibaba.datax.plugin.writer.oceanbasev10writer.Config; -import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ConnHolder; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.AbstractConnHolder; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ObClientConnHolder; import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ServerConnectInfo; import com.alibaba.datax.plugin.writer.oceanbasev10writer.task.ConcurrentTableWriterTask.ConcurrentTableWriter; import com.alibaba.datax.plugin.writer.oceanbasev10writer.util.ObWriterUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; + public class InsertTask implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(InsertTask.class); - private ConcurrentTableWriterTask writerTask; - private ConcurrentTableWriter writer; + private ConcurrentTableWriterTask writerTask; + private ConcurrentTableWriter writer; - private String writeRecordSql; - private long totalCost = 0; - private long insertCount = 0; + private String writeRecordSql; + private long totalCost = 0; + private long insertCount = 0; - private Queue> queue; - private boolean isStop; - private ConnHolder connHolder; + private BlockingQueue> queue; + private boolean isStop; + private AbstractConnHolder connHolder; - private final long taskId; - private ServerConnectInfo connInfo; + private final long taskId; + private ServerConnectInfo connInfo; - // 失败重试次数 - private int failTryCount = Config.DEFAULT_FAIL_TRY_COUNT; - private boolean printCost = Config.DEFAULT_PRINT_COST; - private long costBound = Config.DEFAULT_COST_BOUND; - private List> deleteMeta; + // 失败重试次数 + private int failTryCount = Config.DEFAULT_FAIL_TRY_COUNT; + private boolean printCost = Config.DEFAULT_PRINT_COST; + private long costBound = Config.DEFAULT_COST_BOUND; - public InsertTask( - final long taskId, - Queue> recordsQueue, - Configuration config, - ServerConnectInfo connectInfo, - String writeRecordSql, - List> deleteMeta) { - this.taskId = taskId; - this.queue = recordsQueue; - this.connInfo = connectInfo; - failTryCount = config.getInt(Config.FAIL_TRY_COUNT, Config.DEFAULT_FAIL_TRY_COUNT); - printCost = config.getBool(Config.PRINT_COST, Config.DEFAULT_PRINT_COST); - costBound = config.getLong(Config.COST_BOUND, Config.DEFAULT_COST_BOUND); - this.connHolder = new ObClientConnHolder(config, connInfo.jdbcUrl, - connInfo.getFullUserName(), connInfo.password); - this.writeRecordSql = writeRecordSql; - this.isStop = false; - this.deleteMeta = deleteMeta; - connHolder.initConnection(); - } - - void setWriterTask(ConcurrentTableWriterTask writerTask) { - this.writerTask = writerTask; - } - - void setWriter(ConcurrentTableWriter writer) { - this.writer = writer; - } + public InsertTask( + final long taskId, + BlockingQueue> recordsQueue, + Configuration config, + ServerConnectInfo connectInfo, + String writeRecordSql) { + this.taskId = taskId; + this.queue = recordsQueue; + this.connInfo = connectInfo; + failTryCount = config.getInt(Config.FAIL_TRY_COUNT, Config.DEFAULT_FAIL_TRY_COUNT); + printCost = config.getBool(Config.PRINT_COST, Config.DEFAULT_PRINT_COST); + costBound = config.getLong(Config.COST_BOUND, Config.DEFAULT_COST_BOUND); + this.connHolder = new ObClientConnHolder(config, connInfo.jdbcUrl, + connInfo.getFullUserName(), connInfo.password); + this.writeRecordSql = writeRecordSql; + this.isStop = false; + connHolder.initConnection(); + } - private boolean isStop() { return isStop; } - public void setStop() { isStop = true; } - public long getTotalCost() { return totalCost; } - public long getInsertCount() { return insertCount; } - - @Override - public void run() { - Thread.currentThread().setName(String.format("%d-insertTask-%d", taskId, Thread.currentThread().getId())); - LOG.debug("Task {} start to execute...", taskId); - while (!isStop()) { - try { - List records = queue.poll(); - if (null != records) { - doMultiInsert(records, this.printCost, this.costBound); + void setWriterTask(ConcurrentTableWriterTask writerTask) { + this.writerTask = writerTask; + } - } else if (writerTask.isFinished()) { - writerTask.singalTaskFinish(); - LOG.debug("not more task, thread exist ..."); - break; - } else { - TimeUnit.MILLISECONDS.sleep(5); - } - } catch (InterruptedException e) { - LOG.debug("TableWriter is interrupt"); - } catch (Exception e) { - LOG.warn("ERROR UNEXPECTED {}", e); - } - } - LOG.debug("Thread exist..."); - } - - public void destroy() { - connHolder.destroy(); - }; - - public void calStatistic(final long cost) { - writer.increFinishCount(); - ++insertCount; - totalCost += cost; - if (this.printCost && cost > this.costBound) { - LOG.info("slow multi insert cost {}ms", cost); - } - } + void setWriter(ConcurrentTableWriter writer) { + this.writer = writer; + } - private void doDelete(Connection conn, final List buffer) throws SQLException { - if(deleteMeta == null || deleteMeta.size() == 0) { - return; - } - for (int i = 0; i < deleteMeta.size(); i++) { - String deleteSql = deleteMeta.get(i).getKey(); - int[] valueIdx = deleteMeta.get(i).getValue(); - PreparedStatement ps = null; - try { - ps = conn.prepareStatement(deleteSql); - StringBuilder builder = new StringBuilder(); - for (Record record : buffer) { - int bindIndex = 0; - for (int idx : valueIdx) { - writerTask.fillStatementIndex(ps, bindIndex++, idx, record.getColumn(idx)); - builder.append(record.getColumn(idx).asString()).append(","); - } - ps.addBatch(); - } - LOG.debug("delete values: " + builder.toString()); - ps.executeBatch(); - } catch (SQLException ex) { - LOG.error("SQL Exception when delete records with {}", deleteSql, ex); - throw ex; - } finally { - DBUtil.closeDBResources(ps, null); - } - } - } + private boolean isStop() { + return isStop; + } - public void doMultiInsert(final List buffer, final boolean printCost, final long restrict) { - checkMemstore(); - Connection conn = connHolder.getConn(); - boolean success = false; - long cost = 0; - long startTime = 0; - try { - for (int i = 0; i < failTryCount; ++i) { - if (i > 0) { - try { - int sleep = i >= 9 ? 500 : 1 << i;//不明白为什么要sleep 500s - TimeUnit.SECONDS.sleep(sleep); - } catch (InterruptedException e) { - LOG.info("thread interrupted ..., ignore"); - } - conn = connHolder.getConn(); - LOG.info("retry {}, start do batch insert, size={}", i, buffer.size()); - checkMemstore(); - } - startTime = System.currentTimeMillis(); - PreparedStatement ps = null; - try { - conn.setAutoCommit(false); + public void setStop() { + isStop = true; + } - // do delete if necessary - doDelete(conn, buffer); + public long getTotalCost() { + return totalCost; + } - ps = conn.prepareStatement(writeRecordSql); - for (Record record : buffer) { - ps = writerTask.fillStatement(ps, record); - ps.addBatch(); - } - ps.executeBatch(); - conn.commit(); - success = true; - cost = System.currentTimeMillis() - startTime; - calStatistic(cost); - break; - } catch (SQLException e) { - LOG.warn("Insert fatal error SqlState ={}, errorCode = {}, {}", e.getSQLState(), e.getErrorCode(), e); - if (i == 0 || i > 10 ) { - for (Record record : buffer) { - LOG.warn("ERROR : record {}", record); - } - } - // 按照错误码分类,分情况处理 - // 如果是OB系统级异常,则需要重建连接 - boolean fatalFail = ObWriterUtils.isFatalError(e); - if (fatalFail) { - ObWriterUtils.sleep(300000); - connHolder.reconnect(); - // 如果是可恢复的异常,则重试 - } else if (ObWriterUtils.isRecoverableError(e)) { - conn.rollback(); - ObWriterUtils.sleep(60000); - } else {// 其它异常直接退出,采用逐条写入方式 - conn.rollback(); - ObWriterUtils.sleep(1000); - break; - } - } catch (Exception e) { - e.printStackTrace(); - LOG.warn("Insert error unexpected {}", e); - } finally { - DBUtil.closeDBResources(ps, null); - } - } - } catch (SQLException e) { - LOG.warn("ERROR:retry failSql State ={}, errorCode = {}, {}", e.getSQLState(), e.getErrorCode(), e); - } + public long getInsertCount() { + return insertCount; + } - if (!success) { - try { - LOG.info("do one insert"); - conn = connHolder.reconnect(); - doOneInsert(conn, buffer); - cost = System.currentTimeMillis() - startTime; - calStatistic(cost); - } finally { - } - } - } + @Override + public void run() { + Thread.currentThread().setName(String.format("%d-insertTask-%d", taskId, Thread.currentThread().getId())); + LOG.debug("Task {} start to execute...", taskId); + while (!isStop()) { + try { + List records = queue.poll(5, TimeUnit.MILLISECONDS); + if (null != records) { + doMultiInsert(records, this.printCost, this.costBound); + } else if (writerTask.isFinished()) { + writerTask.singalTaskFinish(); + LOG.debug("not more task, thread exist ..."); + break; + } + } catch (InterruptedException e) { + LOG.debug("TableWriter is interrupt"); + } catch (Exception e) { + LOG.warn("ERROR UNEXPECTED ", e); + } + } + LOG.debug("Thread exist..."); + } - // process one row, delete before insert - private void doOneInsert(Connection connection, List buffer) { - List deletePstmtList = new ArrayList(); - PreparedStatement preparedStatement = null; - try { - connection.setAutoCommit(false); - if (deleteMeta != null && deleteMeta.size() > 0) { - for (int i = 0; i < deleteMeta.size(); i++) { - String deleteSql = deleteMeta.get(i).getKey(); - deletePstmtList.add(connection.prepareStatement(deleteSql)); - } - } + public void destroy() { + connHolder.destroy(); + } - preparedStatement = connection.prepareStatement(this.writeRecordSql); - for (Record record : buffer) { - try { - for (int i = 0; i < deletePstmtList.size(); i++) { - PreparedStatement deleteStmt = deletePstmtList.get(i); - int[] valueIdx = deleteMeta.get(i).getValue(); - int bindIndex = 0; - for (int idx : valueIdx) { - writerTask.fillStatementIndex(deleteStmt, bindIndex++, idx, record.getColumn(idx)); - } - deleteStmt.execute(); - } - preparedStatement = writerTask.fillStatement(preparedStatement, record); - preparedStatement.execute(); - connection.commit(); - } catch (SQLException e) { - writerTask.collectDirtyRecord(record, e); - } finally { - // 此处不应该关闭statement,后续的数据还需要用到 - } - } - } catch (Exception e) { - throw DataXException.asDataXException( - DBUtilErrorCode.WRITE_DATA_ERROR, e); - } finally { - DBUtil.closeDBResources(preparedStatement, null); - for (PreparedStatement pstmt : deletePstmtList) { - DBUtil.closeDBResources(pstmt, null); - } - } - } + public void calStatistic(final long cost) { + writer.increFinishCount(); + ++insertCount; + totalCost += cost; + if (this.printCost && cost > this.costBound) { + LOG.info("slow multi insert cost {}ms", cost); + } + } - private void checkMemstore() { - while (writerTask.isMemStoreFull()) { - ObWriterUtils.sleep(30000); - } - } + public void doMultiInsert(final List buffer, final boolean printCost, final long restrict) { + checkMemstore(); + Connection conn = connHolder.getConn(); + boolean success = false; + long cost = 0; + long startTime = 0; + try { + for (int i = 0; i < failTryCount; ++i) { + if (i > 0) { + conn = connHolder.getConn(); + LOG.info("retry {}, start do batch insert, size={}", i, buffer.size()); + checkMemstore(); + } + startTime = System.currentTimeMillis(); + PreparedStatement ps = null; + try { + conn.setAutoCommit(false); + ps = conn.prepareStatement(writeRecordSql); + for (Record record : buffer) { + ps = writerTask.fillStatement(ps, record); + ps.addBatch(); + } + ps.executeBatch(); + conn.commit(); + success = true; + cost = System.currentTimeMillis() - startTime; + calStatistic(cost); + break; + } catch (SQLException e) { + LOG.warn("Insert fatal error SqlState ={}, errorCode = {}, {}", e.getSQLState(), e.getErrorCode(), e); + if (LOG.isDebugEnabled() && (i == 0 || i > 10)) { + for (Record record : buffer) { + LOG.warn("ERROR : record {}", record); + } + } + // 按照错误码分类,分情况处理 + // 如果是OB系统级异常,则需要重建连接 + boolean fatalFail = ObWriterUtils.isFatalError(e); + if (fatalFail) { + ObWriterUtils.sleep(300000); + connHolder.reconnect(); + // 如果是可恢复的异常,则重试 + } else if (ObWriterUtils.isRecoverableError(e)) { + conn.rollback(); + ObWriterUtils.sleep(60000); + } else {// 其它异常直接退出,采用逐条写入方式 + conn.rollback(); + ObWriterUtils.sleep(1000); + break; + } + } catch (Exception e) { + e.printStackTrace(); + LOG.warn("Insert error unexpected {}", e); + } finally { + DBUtil.closeDBResources(ps, null); + } + } + } catch (SQLException e) { + LOG.warn("ERROR:retry failSql State ={}, errorCode = {}, {}", e.getSQLState(), e.getErrorCode(), e); + } + + if (!success) { + LOG.info("do one insert"); + conn = connHolder.reconnect(); + writerTask.doOneInsert(conn, buffer); + cost = System.currentTimeMillis() - startTime; + calStatistic(cost); + } + } + + private void checkMemstore() { + if (writerTask.isShouldSlow()) { + ObWriterUtils.sleep(100); + } else { + while (writerTask.isShouldPause()) { + ObWriterUtils.sleep(100); + } + } + } } diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/SingleTableWriterTask.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/SingleTableWriterTask.java index 637a3be4..d2f42de5 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/SingleTableWriterTask.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/task/SingleTableWriterTask.java @@ -12,7 +12,7 @@ import com.alibaba.datax.plugin.rdbms.util.DataBaseType; import com.alibaba.datax.plugin.rdbms.writer.CommonRdbmsWriter; import com.alibaba.datax.plugin.rdbms.writer.Key; import com.alibaba.datax.plugin.writer.oceanbasev10writer.Config; -import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ConnHolder; +import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.AbstractConnHolder; import com.alibaba.datax.plugin.writer.oceanbasev10writer.ext.ObClientConnHolder; import com.alibaba.datax.plugin.writer.oceanbasev10writer.util.ObWriterUtils; @@ -30,7 +30,7 @@ public class SingleTableWriterTask extends CommonRdbmsWriter.Task { // 失败重试次数 private int failTryCount = Config.DEFAULT_FAIL_TRY_COUNT; - private ConnHolder connHolder; + private AbstractConnHolder connHolder; private String obWriteMode = "update"; private boolean isOracleCompatibleMode = false; private String obUpdateColumns = null; diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/util/DbUtils.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/util/DbUtils.java index e590fe6b..adffc6f7 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/util/DbUtils.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/util/DbUtils.java @@ -3,18 +3,17 @@ package com.alibaba.datax.plugin.writer.oceanbasev10writer.util; import com.alibaba.datax.common.util.Configuration; import com.alibaba.datax.plugin.rdbms.util.DBUtil; import com.alibaba.datax.plugin.rdbms.util.DataBaseType; -import com.alibaba.datax.plugin.rdbms.writer.CommonRdbmsWriter; import com.alibaba.datax.plugin.rdbms.writer.Constant; import com.alibaba.datax.plugin.rdbms.writer.Key; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +import com.alibaba.datax.plugin.writer.oceanbasev10writer.Config; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.util.List; import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DbUtils { @@ -25,7 +24,7 @@ public class DbUtils { final String password = config.getString(Key.PASSWORD); String jdbcUrl = config.getString(Key.JDBC_URL); - if(jdbcUrl == null) { + if (jdbcUrl == null) { List conns = config.getList(Constant.CONN_MARK, Object.class); Configuration connConf = Configuration.from(conns.get(0).toString()); jdbcUrl = connConf.getString(Key.JDBC_URL); @@ -34,9 +33,9 @@ public class DbUtils { Connection conn = null; PreparedStatement stmt = null; ResultSet result = null; - boolean need_retry = false; String value = null; int retry = 0; + int failTryCount = config.getInt(Config.FAIL_TRY_COUNT, Config.DEFAULT_FAIL_TRY_COUNT); do { try { if (retry > 0) { @@ -58,14 +57,57 @@ public class DbUtils { LOG.info("value for query [{}] is [{}]", query, value); break; } catch (SQLException e) { - need_retry = true; ++retry; LOG.warn("fetch value with {} error {}", query, e); } finally { DBUtil.closeDBResources(result, stmt, conn); } - } while (need_retry); + } while (retry < failTryCount); return value; } + + /** + * build sys connection from ordinary jdbc url + * + * @param jdbcUrl + * @param clusterName + * @return + * @throws Exception + */ + public static Connection buildSysConn(String jdbcUrl, String clusterName) throws Exception { + jdbcUrl = jdbcUrl.replace("jdbc:mysql://", "jdbc:oceanbase://"); + int startIdx = jdbcUrl.indexOf('/', "jdbc:oceanbase://".length()); + int endIdx = jdbcUrl.lastIndexOf('?'); + String prefix = jdbcUrl.substring(0, startIdx + 1); + final String postfix = jdbcUrl.substring(endIdx); + String sysJDBCUrl = prefix + "oceanbase" + postfix; + + String tenantName = "sys"; + String[][] userConfigs = { + {"monitor", "monitor"} + }; + + Connection conn = null; + for (String[] userConfig : userConfigs) { + try { + conn = DBUtil.getConnectionWithoutRetry(DataBaseType.OceanBase, sysJDBCUrl, String.format("%s@%s#%s", userConfig[0], + tenantName, clusterName), userConfig[1]); + } catch (Exception e) { + LOG.warn("fail connecting to ob: " + e.getMessage()); + + } + if (conn == null) { + LOG.warn("fail to get connection with user " + userConfig[0] + ", try alternative user."); + } else { + break; + } + } + + if (conn == null) { + throw new Exception("fail to get connection with sys tenant."); + } + + return conn; + } } 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 edc4b236..a5d6b0ea 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 @@ -1,8 +1,10 @@ package com.alibaba.datax.plugin.writer.oceanbasev10writer.util; +import com.alibaba.datax.plugin.rdbms.reader.util.ObVersion; import com.alibaba.datax.plugin.rdbms.util.DBUtil; import com.alibaba.datax.plugin.rdbms.writer.CommonRdbmsWriter.Task; import com.alibaba.datax.plugin.writer.oceanbasev10writer.Config; +import org.apache.commons.lang3.RandomUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -11,6 +13,7 @@ import org.slf4j.LoggerFactory; import java.sql.*; import java.util.*; +import static com.alibaba.datax.plugin.writer.oceanbasev10writer.Config.DEFAULT_SLOW_MEMSTORE_THRESHOLD; public class ObWriterUtils { @@ -18,8 +21,14 @@ public class ObWriterUtils { private static final String ORACLE_KEYWORDS = "ACCESS,ADD,ALL,ALTER,AND,ANY,ARRAYLEN,AS,ASC,AUDIT,BETWEEN,BY,CHAR,CHECK,CLUSTER,COLUMN,COMMENT,COMPRESS,CONNECT,CREATE,CURRENT,DATE,DECIMAL,DEFAULT,DELETE,DESC,DISTINCT,DROP,ELSE,EXCLUSIVE,EXISTS,FILE,FLOAT,FOR,FROM,GRANT,GROUP,HAVING,IDENTIFIED,IMMEDIATE,IN,INCREMENT,INDEX,INITIAL,INSERT,INTEGER,INTERSECT,INTO,IS,LEVEL,LIKE,LOCK,LONG,MAXEXTENTS,MINUS,MODE,MODIFY,NOAUDIT,NOCOMPRESS,NOT,NOTFOUND,NOWAIT,NULL,NUMBER,OF,OFFLINE,ON,ONLINE,OPTION,OR,ORDER,PCTFREE,PRIOR,PRIVILEGES,PUBLIC,RAW,RENAME,RESOURCE,REVOKE,ROW,ROWID,ROWLABEL,ROWNUM,ROWS,SELECT,SESSION,SET,SHARE,SIZE,SMALLINT,SQLBUF,START,SUCCESSFUL,SYNONYM,TABLE,THEN,TO,TRIGGER,UID,UNION,UNIQUE,UPDATE,USER,VALIDATE,VALUES,VARCHAR,VARCHAR2,VIEW,WHENEVER,WHERE,WITH"; private static String CHECK_MEMSTORE = "select 1 from %s.gv$memstore t where t.total>t.mem_limit * ?"; + private static final String CHECK_MEMSTORE_4_0 = "select 1 from %s.gv$ob_memstore t where t.MEMSTORE_USED>t.MEMSTORE_LIMIT * ?"; + + private static String CHECK_MEMSTORE_RATIO = "select min(t.total/t.mem_limit) from %s.gv$memstore t"; + private static final String CHECK_MEMSTORE_RATIO_4_0 = "select min(t.MEMSTORE_USED/t.MEMSTORE_LIMIT) from %s.gv$ob_memstore t"; + private static Set databaseKeywords; private static String compatibleMode = null; + private static String obVersion = null; protected static final Logger LOG = LoggerFactory.getLogger(Task.class); private static Set keywordsFromString2HashSet(final String keywords) { return new HashSet(Arrays.asList(keywords.split(","))); @@ -61,7 +70,7 @@ public class ObWriterUtils { if (isOracleMode()) { sysDbName = "sys"; } - ps = conn.prepareStatement(String.format(CHECK_MEMSTORE, sysDbName)); + ps = conn.prepareStatement(String.format(getMemStoreSql(), sysDbName)); ps.setDouble(1, memstoreThreshold); rs = ps.executeQuery(); // 只要有满足条件的,则表示当前租户 有个机器的memstore即将满 @@ -77,10 +86,50 @@ public class ObWriterUtils { return result; } + public static double queryMemUsedRatio (Connection conn) { + PreparedStatement ps = null; + ResultSet rs = null; + double result = 0; + try { + String sysDbName = "oceanbase"; + if (isOracleMode()) { + sysDbName = "sys"; + } + ps = conn.prepareStatement(String.format(getMemStoreRatioSql(), sysDbName)); + rs = ps.executeQuery(); + // 只要有满足条件的,则表示当前租户 有个机器的memstore即将满 + if (rs.next()) { + result = rs.getDouble(1); + } + } catch (Throwable e) { + LOG.warn("Check memstore fail, reason: {}. Use a random value instead.", e.getMessage()); + result = RandomUtils.nextDouble(0.3D, DEFAULT_SLOW_MEMSTORE_THRESHOLD + 0.2D); + } finally { + //do not need to close the statment in ob1.0 + } + return result; + } + public static boolean isOracleMode(){ return (compatibleMode.equals(Config.OB_COMPATIBLE_MODE_ORACLE)); } + private static String getMemStoreSql() { + if (ObVersion.valueOf(obVersion).compareTo(ObVersion.V4000) >= 0) { + return CHECK_MEMSTORE_4_0; + } else { + return CHECK_MEMSTORE; + } + } + + private static String getMemStoreRatioSql() { + if (ObVersion.valueOf(obVersion).compareTo(ObVersion.V4000) >= 0) { + return CHECK_MEMSTORE_RATIO_4_0; + } else { + return CHECK_MEMSTORE_RATIO; + } + } + public static String getCompatibleMode() { return compatibleMode; } @@ -89,6 +138,10 @@ public class ObWriterUtils { compatibleMode = mode; } + public static void setObVersion(String version) { + obVersion = version; + } + private static String buildDeleteSql (String tableName, List columns) { StringBuilder builder = new StringBuilder("DELETE FROM "); builder.append(tableName).append(" WHERE "); @@ -165,7 +218,7 @@ public class ObWriterUtils { } List s = uniqueKeys.get(keyName); if (s == null) { - s = new ArrayList(); + s = new ArrayList<>(); uniqueKeys.put(keyName, s); } s.add(columnName); @@ -237,7 +290,7 @@ public class ObWriterUtils { String columnName = StringUtils.upperCase(rs.getString("Column_name")); Set s = uniqueKeys.get(keyName); if (s == null) { - s = new HashSet(); + s = new HashSet<>(); uniqueKeys.put(keyName, s); } s.add(columnName); @@ -399,7 +452,7 @@ public class ObWriterUtils { private static Set white = new HashSet(); static { - int[] errList = { 1213, 1047, 1041, 1094, 4000, 4012 }; + int[] errList = { 1213, 1047, 1041, 1094, 4000, 4012, 4013 }; for (int err : errList) { white.add(err); } @@ -429,4 +482,26 @@ public class ObWriterUtils { t.setDaemon(true); t.start(); } + + /** + * + */ + public static enum LoadMode { + + /** + * Fast insert + */ + FAST, + + /** + * Insert slowly + */ + SLOW, + + /** + * Pause to insert + */ + PAUSE + } + } diff --git a/oceanbasev10writer/src/main/libs/shade-ob-partition-calculator-1.0-SNAPSHOT.jar b/oceanbasev10writer/src/main/libs/shade-ob-partition-calculator-1.0-SNAPSHOT.jar new file mode 100644 index 00000000..34453ce6 Binary files /dev/null and b/oceanbasev10writer/src/main/libs/shade-ob-partition-calculator-1.0-SNAPSHOT.jar differ diff --git a/ossreader/doc/ossreader.md b/ossreader/doc/ossreader.md index e0259a2a..51d757bc 100644 --- a/ossreader/doc/ossreader.md +++ b/ossreader/doc/ossreader.md @@ -26,6 +26,8 @@ OSSReader实现了从OSS读取数据并转为DataX协议的功能,OSS本身是 6. 多个object可以支持并发读取。 +7. 支持读取 parquet orc 文件 + 我们暂时不能做到: 1. 单个Object(File)支持多线程并发读取,这里涉及到单个Object内部切分算法。二期考虑支持。 @@ -37,7 +39,7 @@ OSSReader实现了从OSS读取数据并转为DataX协议的功能,OSS本身是 ### 3.1 配置样例 - +读取 txt, csv 格式样例 ```json { "job": { @@ -80,6 +82,63 @@ OSSReader实现了从OSS读取数据并转为DataX协议的功能,OSS本身是 } } ``` +读取 orc 格式样例 +```json +{ + "stepType": "oss", + "parameter": { + "endpoint": "http://oss.aliyuncs.com", + "accessId": "", + "accessKey": "", + "bucket": "myBucket", + "fileFormat": "orc", + "path": "/tests/case61/orc__691b6815_9260_4037_9899_****", + "column": [ + { + "index": 0, + "type": "long" + }, + { + "index": "1", + "type": "string" + }, + { + "index": "2", + "type": "string" + } + ] + } +} +``` +读取 parquet 格式样例 +```json +{ + "stepType": "oss", + "parameter": { + "endpoint": "http://oss.aliyuncs.com", + "accessId": "", + "accessKey": "", + "bucket": "myBucket", + "fileFormat": "parquet", + "path": "/parquet", + "parquetSchema":"message m { optional BINARY registration_dttm (UTF8); optional Int64 id; optional BINARY first_name (UTF8); optional BINARY last_name (UTF8); optional BINARY email (UTF8); optional BINARY gender (UTF8); optional BINARY ip_address (UTF8); optional BINARY cc (UTF8); optional BINARY country (UTF8); optional BINARY birthdate (UTF8); optional DOUBLE salary; optional BINARY title (UTF8); optional BINARY comments (UTF8); }", + "column": [ + { + "index": 0, + "type": "long" + }, + { + "index": "1", + "type": "string" + }, + { + "index": "2", + "type": "string" + } + ] + } +} +``` ### 3.2 参数说明 diff --git a/osswriter/doc/osswriter.md b/osswriter/doc/osswriter.md index 1a3d3e47..0c23e698 100644 --- a/osswriter/doc/osswriter.md +++ b/osswriter/doc/osswriter.md @@ -18,7 +18,7 @@ OSSWriter提供了向OSS写入类CSV格式的一个或者多个表文件。 OSSWriter实现了从DataX协议转为OSS中的TXT文件功能,OSS本身是无结构化数据存储,OSSWriter需要在如下几个方面增加: -1. 支持且仅支持写入 TXT的文件,且要求TXT中shema为一张二维表。 +1. 支持写入 TXT的文件,且要求TXT中shema为一张二维表。 2. 支持类CSV格式文件,自定义分隔符。 @@ -28,6 +28,8 @@ OSSWriter实现了从DataX协议转为OSS中的TXT文件功能,OSS本身是无 7. 文件支持滚动,当文件大于某个size值或者行数值,文件需要切换。 [暂不支持] +8. 支持写 PARQUET、ORC 文件 + 我们不能做到: 1. 单个文件不能支持并发写入。 @@ -37,7 +39,7 @@ OSSWriter实现了从DataX协议转为OSS中的TXT文件功能,OSS本身是无 ### 3.1 配置样例 - +写 txt文件样例 ```json { "job": { @@ -65,7 +67,90 @@ OSSWriter实现了从DataX协议转为OSS中的TXT文件功能,OSS本身是无 } } ``` +写 orc 文件样例 +```json +{ + "job": { + "setting": {}, + "content": [ + { + "reader": {}, + "writer": { + "name": "osswriter", + "parameter": { + "endpoint": "http://oss.aliyuncs.com", + "accessId": "", + "accessKey": "", + "bucket": "myBucket", + "fileName": "test", + "encoding": "UTF-8", + "column": [ + { + "name": "col1", + "type": "BIGINT" + }, + { + "name": "col2", + "type": "DOUBLE" + }, + { + "name": "col3", + "type": "STRING" + } + ], + "fileFormat": "orc", + "path": "/tests/case61", + "writeMode": "append" + } + } + } + ] + } +} +``` +写 parquet 文件样例 +```json +{ + "job": { + "setting": {}, + "content": [ + { + "reader": {}, + "writer": { + "name": "osswriter", + "parameter": { + "endpoint": "http://oss.aliyuncs.com", + "accessId": "", + "accessKey": "", + "bucket": "myBucket", + "fileName": "test", + "encoding": "UTF-8", + "column": [ + { + "name": "col1", + "type": "BIGINT" + }, + { + "name": "col2", + "type": "DOUBLE" + }, + { + "name": "col3", + "type": "STRING" + } + ], + "parquetSchema": "message test { required int64 int64_col;\n required binary str_col (UTF8);\nrequired group params (MAP) {\nrepeated group key_value {\nrequired binary key (UTF8);\nrequired binary value (UTF8);\n}\n}\nrequired group params_arr (LIST) {\n repeated group list {\n required binary element (UTF8);\n }\n}\nrequired group params_struct {\n required int64 id;\n required binary name (UTF8);\n }\nrequired group params_arr_complex (LIST) {\n repeated group list {\n required group element {\n required int64 id;\n required binary name (UTF8);\n}\n }\n}\nrequired group params_complex (MAP) {\nrepeated group key_value {\nrequired binary key (UTF8);\nrequired group value {\n required int64 id;\n required binary name (UTF8);\n }\n}\n}\nrequired group params_struct_complex {\n required int64 id;\n required group detail {\n required int64 id;\n required binary name (UTF8);\n }\n }\n}", + "fileFormat": "parquet", + "path": "/tests/case61", + "writeMode": "append" + } + } + } + ] + } +} +``` ### 3.2 参数说明 * **endpoint** diff --git a/otsreader/doc/otsreader.md b/otsreader/doc/otsreader.md index 1297dbd6..77b4edfe 100644 --- a/otsreader/doc/otsreader.md +++ b/otsreader/doc/otsreader.md @@ -13,7 +13,7 @@ OTSReader插件实现了从OTS读取数据,并可以通过用户指定抽取 * 范围抽取 * 指定分片抽取 -OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务,提供海量结构化数据的存储和实时访问。OTS 以实例和表的形式组织数据,通过数据分片和负载均衡技术,实现规模上的无缝扩展。 +本版本的OTSReader新增了支持多版本数据的读取功能,同时兼容旧版本的配置文件 ## 2 实现原理 @@ -25,201 +25,425 @@ OTSReader会根据OTS的表范围,按照Datax并发的数目N,将范围等 ### 3.1 配置样例 -* 配置一个从OTS全表同步抽取数据到本地的作业: +#### 3.1.1 +* 配置一个从OTS表读取单版本数据的reader: ``` { - "job": { - "setting": { - }, - "content": [ - { - "reader": { - "name": "otsreader", - "parameter": { - /* ----------- 必填 --------------*/ - "endpoint":"", - "accessId":"", - "accessKey":"", - "instanceName":"", - - // 导出数据表的表名 - "table":"", - - // 需要导出的列名,支持重复列和常量列,区分大小写 - // 常量列:类型支持STRING,INT,DOUBLE,BOOL和BINARY - // 备注:BINARY需要通过Base64转换为对应的字符串传入插件 - "column":[ - {"name":"col1"}, // 普通列 - {"name":"col2"}, // 普通列 - {"name":"col3"}, // 普通列 - {"type":"STRING", "value" : "bazhen"}, // 常量列(字符串) - {"type":"INT", "value" : ""}, // 常量列(整形) - {"type":"DOUBLE", "value" : ""}, // 常量列(浮点) - {"type":"BOOL", "value" : ""}, // 常量列(布尔) - {"type":"BINARY", "value" : "Base64(bin)"} // 常量列(二进制),使用Base64编码完成 - ], - "range":{ - // 导出数据的起始范围 - // 支持INF_MIN, INF_MAX, STRING, INT - "begin":[ - {"type":"INF_MIN"}, - ], - // 导出数据的结束范围 - // 支持INF_MIN, INF_MAX, STRING, INT - "end":[ - {"type":"INF_MAX"}, - ] - } - } - }, - "writer": {} - } - ] - } -} -``` - -* 配置一个定义抽取范围的OTSReader: - -``` -{ - "job": { - "setting": { - "speed": { - "byte":10485760 + "job": { + "setting": { + "speed": { + //设置传输速度,单位为byte/s,DataX运行会尽可能达到该速度但是不超过它. + "byte": 1048576 + } + //出错限制 + "errorLimit": { + //出错的record条数上限,当大于该值即报错。 + "record": 0, + //出错的record百分比上限 1.0表示100%,0.02表示2% + "percentage": 0.02 + } + }, + "content": [ + { + "reader": { + "name": "otsreader-internal", + "parameter": { + "endpoint":"", + "accessId":"", + "accessKey":"", + "instanceName":"", + "table": "", + //version定义了是否使用新版本插件 可选值:false || true + "newVersion":"false", + //mode定义了读取数据的格式(普通数据/多版本数据),可选值:normal || multiversion + "mode": "normal", + + // 导出的范围,读取的范围是[begin,end),左闭右开的区间 + // begin小于end,表示正序读取数据 + // begin大于end,表示反序读取数据 + // begin和end不能相等 + // type支持的类型有如下几类: + // string、int、binary + // binary输入的方式采用二进制的Base64字符串形式传入 + // INF_MIN 表示无限小 + // INF_MAX 表示无限大 + "range":{ + // 可选,默认表示从无限小开始读取 + // 这个值的输入可以填写空数组,或者PK前缀,亦或者完整的PK,在正序读取数据时,默认填充PK后缀为INF_MIN,反序为INF_MAX + // 例子: + // 如果用户的表有2个PK,类型分别为string、int,那么如下3种输入都是合法,如: + // 1. [] --> 表示从表的开始位置读取 + // 2. [{"type":"string", "value":"a"}] --> 表示从[{"type":"string", "value":"a"},{"type":"INF_MIN"}] + // 3. [{"type":"string", "value":"a"},{"type":"INF_MIN"}] + // + // binary类型的PK列比较特殊,因为Json不支持直接输入二进制数,所以系统定义:用户如果要传入 + // 二进制,必须使用(Java)Base64.encodeBase64String方法,将二进制转换为一个可视化的字符串,然后将这个字符串填入value中 + // 例子(Java): + // byte[] bytes = "hello".getBytes(); # 构造一个二进制数据,这里使用字符串hello的byte值 + // String inputValue = Base64.encodeBase64String(bytes) # 调用Base64方法,将二进制转换为可视化的字符串 + // 上面的代码执行之后,可以获得inputValue为"aGVsbG8=" + // 最终写入配置:{"type":"binary","value" : "aGVsbG8="} + + "begin":[{"type":"string", "value":"a"},{"type":"INF_MIN"}], + + // 默认表示读取到无限大结束 + // 这个值得输入可以填写空数组,或者PK前缀,亦或者完整的PK,在正序读取数据时,默认填充PK后缀为INF_MAX,反序为INF_MIN + // 可选 + "end":[{"type":"string", "value":"a"},{"type":"INF_MAX"}], + + // 当前用户数据较多时,需要开启并发导出,Split可以将当前范围的的数据按照切分点切分为多个并发任务 + // 可选 + // 1. split中的输入值只能PK的第一列(分片建),且值的类型必须和PartitionKey一致 + // 2. 值的范围必须在begin和end之间 + // 3. split内部的值必须根据begin和end的正反序关系而递增或者递减 + "split":[{"type":"string", "value":"b"}, {"type":"string", "value":"c"}] }, - "errorLimit":0.0 + + + // 指定要导出的列,支持普通列和常量列 + // 格式 + // 普通列格式:{"name":"{your column name}"} + // 常量列格式:{"type":"", "value":""} , type支持string、int、binary、bool、double + // binary类型需要使用base64转换成对应的字符串传入 + // 注意: + // 1. PK列也是需要用户在下面单独指定 + "column": [ + {"name":"pk1"}, // 普通列,下同 + {"name":"pk2"}, + {"name":"attr1"}, + {"type":"string","value" : ""} // 指定常量列,下同 + {"type":"int","value" : ""} + {"type":"double","value" : ""} + // binary类型的常量列比较特殊,因为Json不支持直接输入二进制数,所以系统定义:用户如果要传入 + // 二进制,必须使用(Java)Base64.encodeBase64String方法,将二进制转换为一个可视化的字符串,然后将这个字符串填入value中 + // 例子(Java): + // byte[] bytes = "hello".getBytes(); # 构造一个二进制数据,这里使用字符串hello的byte值 + // String inputValue = Base64.encodeBase64String(bytes) # 调用Base64方法,将二进制转换为可视化的字符串 + // 上面的代码执行之后,可以获得inputValue为"aGVsbG8=" + // 最终写入配置:{"type":"binary","value" : "aGVsbG8="} + + {"type":"binary","value" : "aGVsbG8="} + ], + } }, - "content": [ - { - "reader": { - "name": "otsreader", - "parameter": { - "endpoint":"", - "accessId":"", - "accessKey":"", - "instanceName":"", - - // 导出数据表的表名 - "table":"", - - // 需要导出的列名,支持重复类和常量列,区分大小写 - // 常量列:类型支持STRING,INT,DOUBLE,BOOL和BINARY - // 备注:BINARY需要通过Base64转换为对应的字符串传入插件 - "column":[ - {"name":"col1"}, // 普通列 - {"name":"col2"}, // 普通列 - {"name":"col3"}, // 普通列 - {"type":"STRING","value" : ""}, // 常量列(字符串) - {"type":"INT","value" : ""}, // 常量列(整形) - {"type":"DOUBLE","value" : ""}, // 常量列(浮点) - {"type":"BOOL","value" : ""}, // 常量列(布尔) - {"type":"BINARY","value" : "Base64(bin)"} // 常量列(二进制) - ], - "range":{ - // 导出数据的起始范围 - // 支持INF_MIN, INF_MAX, STRING, INT - "begin":[ - {"type":"INF_MIN"}, - {"type":"INF_MAX"}, - {"type":"STRING", "value":"hello"}, - {"type":"INT", "value":"2999"}, - ], - // 导出数据的结束范围 - // 支持INF_MIN, INF_MAX, STRING, INT - "end":[ - {"type":"INF_MAX"}, - {"type":"INF_MIN"}, - {"type":"STRING", "value":"hello"}, - {"type":"INT", "value":"2999"}, - ] - } - } - }, - "writer": {} - } - ] - } + "writer": { + //writer类型 + "name": "streamwriter", + //是否打印内容 + "parameter": { + "print": true + } + } + } + ] + } } ``` +#### 3.1.2 +* 配置一个从OTS表读取多版本数据的reader(仅在newVersion == true时支持): + +``` +{ + "job": { + "setting": { + "speed": { + //设置传输速度,单位为byte/s,DataX运行会尽可能达到该速度但是不超过它. + "byte": 1048576 + } + //出错限制 + "errorLimit": { + //出错的record条数上限,当大于该值即报错。 + "record": 0, + //出错的record百分比上限 1.0表示100%,0.02表示2% + "percentage": 0.02 + } + }, + "content": [ + { + "reader": { + "name": "otsreader-internal", + "parameter": { + "endpoint":"", + "accessId":"", + "accessKey":"", + "instanceName":"", + "table": "", + //version定义了是否使用新版本插件 可选值:false || true + "newVersion":"true", + //mode定义了读取数据的格式(普通数据/多版本数据),可选值:normal || multiversion + "mode": "multiversion", + + // 导出的范围,,读取的范围是[begin,end),左闭右开的区间 + // begin小于end,表示正序读取数据 + // begin大于end,表示反序读取数据 + // begin和end不能相等 + // type支持的类型有如下几类: + // string、int、binary + // binary输入的方式采用二进制的Base64字符串形式传入 + // INF_MIN 表示无限小 + // INF_MAX 表示无限大 + "range":{ + // 可选,默认表示从无限小开始读取 + // 这个值的输入可以填写空数组,或者PK前缀,亦或者完整的PK,在正序读取数据时,默认填充PK后缀为INF_MIN,反序为INF_MAX + // 例子: + // 如果用户的表有2个PK,类型分别为string、int,那么如下3种输入都是合法,如: + // 1. [] --> 表示从表的开始位置读取 + // 2. [{"type":"string", "value":"a"}] --> 表示从[{"type":"string", "value":"a"},{"type":"INF_MIN"}] + // 3. [{"type":"string", "value":"a"},{"type":"INF_MIN"}] + // + // binary类型的PK列比较特殊,因为Json不支持直接输入二进制数,所以系统定义:用户如果要传入 + // 二进制,必须使用(Java)Base64.encodeBase64String方法,将二进制转换为一个可视化的字符串,然后将这个字符串填入value中 + // 例子(Java): + // byte[] bytes = "hello".getBytes(); # 构造一个二进制数据,这里使用字符串hello的byte值 + // String inputValue = Base64.encodeBase64String(bytes) # 调用Base64方法,将二进制转换为可视化的字符串 + // 上面的代码执行之后,可以获得inputValue为"aGVsbG8=" + // 最终写入配置:{"type":"binary","value" : "aGVsbG8="} + + "begin":[{"type":"string", "value":"a"},{"type":"INF_MIN"}], + + // 默认表示读取到无限大结束 + // 这个值得输入可以填写空数组,或者PK前缀,亦或者完整的PK,在正序读取数据时,默认填充PK后缀为INF_MAX,反序为INF_MIN + // 可选 + "end":[{"type":"string", "value":"g"},{"type":"INF_MAX"}], + + // 当前用户数据较多时,需要开启并发导出,Split可以将当前范围的的数据按照切分点切分为多个并发任务 + // 可选 + // 1. split中的输入值只能PK的第一列(分片建),且值的类型必须和PartitionKey一致 + // 2. 值的范围必须在begin和end之间 + // 3. split内部的值必须根据begin和end的正反序关系而递增或者递减 + "split":[{"type":"string", "value":"b"}, {"type":"string", "value":"c"}] + }, + + // 指定要导出的列,在多版本模式下只支持普通列 + // 格式: + // 普通列格式:{"name":"{your column name}"} + // 可选,默认导出所有列的所有版本 + // 注意: + // 1.在多版本模式下,不支持常量列 + // 2.PK列不能指定,导出4元组中默认包括完整的PK + // 3.不能重复指定列 + "column": [ + {"name":"attr1"} + ], + + // 请求数据的Time Range,读取的范围是[begin,end),左闭右开的区间 + // 可选,默认读取全部版本 + // 注意:begin必须小于end + "timeRange":{ + // 可选,默认为0 + // 取值范围是0~LONG_MAX + "begin":1400000000, + // 可选,默认为Long Max(9223372036854775807L) + // 取值范围是0~LONG_MAX + "end" :1600000000 + }, + + // 请求的指定Version + // 可选,默认读取所有版本 + // 取值范围是1~INT32_MAX + "maxVersion":10, + } + }, + "writer": { + //writer类型 + "name": "streamwriter", + //是否打印内容 + "parameter": { + "print": true + } + } + } + ] + } +} +``` +#### 3.1.3 +* 配置一个从OTS **时序表**读取数据的reader(仅在newVersion == true时支持): +```json +{ + "job": { + "setting": { + "speed": { + // 读取时序数据的通道数 + "channel": 5 + } + }, + "content": [ + { + "reader": { + "name": "otsreader", + "parameter": { + "endpoint": "", + "accessId": "", + "accessKey": "", + "instanceName": "", + "table": "", + // 读时序数据mode必须为normal + "mode": "normal", + // 读时序数据newVersion必须为true + "newVersion": "true", + // 配置该表为时序表 + "isTimeseriesTable":"true", + // 配置需要读取时间线的measurementName字段,非必需 + // 为空则读取全表数据 + "measurementName":"measurement_5", + // column是一个数组,每个元素表示一列 + // 对于常量列,需要配置以下字段: + // 1. type : 字段值类型,必需 + // 支持类型 : string, int, double, bool, binary + // 2. value : 字段值,必需 + // + // 对于普通列,需要配置以下字段: + // 1. name : 列名,必需 + // 时间线的'度量名称'使用_m_name标识,数据类型为String + // 时间线的'数据源'使用_data_source标识,数据类型为String + // 时间线的'标签'使用_tags标识,数据类型为String + // 时间线的'时间戳'使用_time标识,数据类型为Long + // 2. is_timeseries_tag : 是否为tags字段内部的键值,非必需,默认为false。 + // 3. type : 字段值类型,非必需,默认为string。 + // 支持类型 : string, int, double, bool, binary + "column": [ + { + "name": "_m_name" + }, + { + "name": "tagA", + "is_timeseries_tag":"true" + }, + { + "name": "double_0", + "type":"DOUBLE" + }, + { + "name": "string_0", + "type":"STRING" + }, + { + "name": "long_0", + "type":"int" + }, + { + "name": "binary_0", + "type":"BINARY" + }, + { + "name": "bool_0", + "type":"BOOL" + }, + { + "type":"STRING", + "value":"testString" + } + ] + } + }, + "writer": { + + } + } + ] + } +} + +``` ### 3.2 参数说明 * **endpoint** - * 描述:OTS Server的EndPoint地址,例如http://bazhen.cn−hangzhou.ots.aliyuncs.com。 + * 描述:OTS Server的EndPoint地址,例如http://bazhen.cn−hangzhou.ots.aliyuncs.com。 - * 必选:是
+ * 必选:是
- * 默认值:无
+ * 默认值:无
* **accessId** - * 描述:OTS的accessId
+ * 描述:OTS的accessId
- * 必选:是
+ * 必选:是
- * 默认值:无
+ * 默认值:无
* **accessKey** - * 描述:OTS的accessKey
+ * 描述:OTS的accessKey
- * 必选:是
+ * 必选:是
- * 默认值:无
+ * 默认值:无
* **instanceName** - * 描述:OTS的实例名称,实例是用户使用和管理 OTS 服务的实体,用户在开通 OTS 服务之后,需要通过管理控制台来创建实例,然后在实例内进行表的创建和管理。实例是 OTS 资源管理的基础单元,OTS 对应用程序的访问控制和资源计量都在实例级别完成。
+ * 描述:OTS的实例名称,实例是用户使用和管理 OTS 服务的实体,用户在开通 OTS 服务之后,需要通过管理控制台来创建实例,然后在实例内进行表的创建和管理。实例是 OTS 资源管理的基础单元,OTS 对应用程序的访问控制和资源计量都在实例级别完成。
- * 必选:是
+ * 必选:是
- * 默认值:无
+ * 默认值:无
* **table** - * 描述:所选取的需要抽取的表名称,这里有且只能填写一张表。在OTS不存在多表同步的需求。
+ * 描述:所选取的需要抽取的表名称,这里有且只能填写一张表。在OTS不存在多表同步的需求。
- * 必选:是
+ * 必选:是
- * 默认值:无
+ * 默认值:无
+ +* **newVersion** + + * 描述:version定义了使用的ots SDK版本。
+ * true,新版本插件,使用com.alicloud.openservices.tablestore的依赖(推荐) + * false,旧版本插件,使用com.aliyun.openservices.ots的依赖,**不支持多版本数据的读取** + + * 必选:否
+ + * 默认值:false
+ +* **mode** + + * 描述:读取为多版本格式的数据,目前有两种模式。
+ * normal,对应普通的数据 + * multiVersion,写入数据为多版本格式的数据,多版本模式下,配置参数有所不同,详见3.1.2 + + * 必选:否
+ + * 默认值:normal
* **column** - * 描述:所配置的表中需要同步的列名集合,使用JSON的数组描述字段信息。由于OTS本身是NoSQL系统,在OTSReader抽取数据过程中,必须指定相应地字段名称。 + * 描述:所配置的表中需要同步的列名集合,使用JSON的数组描述字段信息。由于OTS本身是NoSQL系统,在OTSReader抽取数据过程中,必须指定相应地字段名称。 - 支持普通的列读取,例如: {"name":"col1"} + 支持普通的列读取,例如: {"name":"col1"} - 支持部分列读取,如用户不配置该列,则OTSReader不予读取。 + 支持部分列读取,如用户不配置该列,则OTSReader不予读取。 - 支持常量列读取,例如: {"type":"STRING", "value" : "DataX"}。使用type描述常量类型,目前支持STRING、INT、DOUBLE、BOOL、BINARY(用户使用Base64编码填写)、INF_MIN(OTS的系统限定最小值,使用该值用户不能填写value属性,否则报错)、INF_MAX(OTS的系统限定最大值,使用该值用户不能填写value属性,否则报错)。 + 支持常量列读取,例如: {"type":"STRING", "value" : "DataX"}。使用type描述常量类型,目前支持STRING、INT、DOUBLE、BOOL、BINARY(用户使用Base64编码填写)、INF_MIN(OTS的系统限定最小值,使用该值用户不能填写value属性,否则报错)、INF_MAX(OTS的系统限定最大值,使用该值用户不能填写value属性,否则报错)。 - 不支持函数或者自定义表达式,由于OTS本身不提供类似SQL的函数或者表达式功能,OTSReader也不能提供函数或表达式列功能。 + 不支持函数或者自定义表达式,由于OTS本身不提供类似SQL的函数或者表达式功能,OTSReader也不能提供函数或表达式列功能。 - * 必选:是
+ * 必选:是
- * 默认值:无
+ * 默认值:无
* **begin/end** - * 描述:该配置项必须配对使用,用于支持OTS表范围抽取。begin/end中描述的是OTS **PrimaryKey**的区间分布状态,而且必须保证区间覆盖到所有的PrimaryKey,**需要指定该表下所有的PrimaryKey范围,不能遗漏任意一个PrimaryKey**,对于无限大小的区间,可以使用{"type":"INF_MIN"},{"type":"INF_MAX"}指代。例如对一张主键为 [DeviceID, SellerID]的OTS进行抽取任务,begin/end可以配置为: + * 描述:该配置项必须配对使用,用于支持OTS表范围抽取。begin/end中描述的是OTS **PrimaryKey**的区间分布状态,而且必须保证区间覆盖到所有的PrimaryKey,**需要指定该表下所有的PrimaryKey范围,不能遗漏任意一个PrimaryKey**,对于无限大小的区间,可以使用{"type":"INF_MIN"},{"type":"INF_MAX"}指代。例如对一张主键为 [DeviceID, SellerID]的OTS进行抽取任务,begin/end可以配置为: - ```json - "range": { - "begin": { - {"type":"INF_MIN"}, //指定deviceID最小值 - {"type":"INT", "value":"0"} //指定deviceID最小值 - }, - "end": { - {"type":"INF_MAX"}, //指定deviceID抽取最大值 - {"type":"INT", "value":"9999"} //指定deviceID抽取最大值 - } - } - ``` + ```json + "range": { + "begin": { + {"type":"INF_MIN"}, //指定deviceID最小值 + {"type":"INT", "value":"0"} //指定deviceID最小值 + }, + "end": { + {"type":"INF_MAX"}, //指定deviceID抽取最大值 + {"type":"INT", "value":"9999"} //指定deviceID抽取最大值 + } + } + ``` 如果要对上述表抽取全表,可以使用如下配置: @@ -237,42 +461,42 @@ OTSReader会根据OTS的表范围,按照Datax并发的数目N,将范围等 } ``` - * 必选:是
+ * 必选:否
- * 默认值:空
+ * 默认值:读取全部值
* **split** - * 描述:该配置项属于高级配置项,是用户自己定义切分配置信息,普通情况下不建议用户使用。适用场景通常在OTS数据存储发生热点,使用OTSReader自动切分的策略不能生效情况下,使用用户自定义的切分规则。split指定是的在Begin、End区间内的切分点,且只能是partitionKey的切分点信息,即在split仅配置partitionKey,而不需要指定全部的PrimaryKey。 + * 描述:该配置项属于高级配置项,是用户自己定义切分配置信息,普通情况下不建议用户使用。适用场景通常在OTS数据存储发生热点,使用OTSReader自动切分的策略不能生效情况下,使用用户自定义的切分规则。split指定是的在Begin、End区间内的切分点,且只能是partitionKey的切分点信息,即在split仅配置partitionKey,而不需要指定全部的PrimaryKey。 - 例如对一张主键为 [DeviceID, SellerID]的OTS进行抽取任务,可以配置为: + 例如对一张主键为 [DeviceID, SellerID]的OTS进行抽取任务,可以配置为: - ```json - "range": { - "begin": { - {"type":"INF_MIN"}, //指定deviceID最小值 - {"type":"INF_MIN"} //指定deviceID最小值 - }, - "end": { - {"type":"INF_MAX"}, //指定deviceID抽取最大值 - {"type":"INF_MAX"} //指定deviceID抽取最大值 - }, - // 用户指定的切分点,如果指定了切分点,Job将按照begin、end和split进行Task的切分, - // 切分的列只能是Partition Key(ParimaryKey的第一列) - // 支持INF_MIN, INF_MAX, STRING, INT - "split":[ - {"type":"STRING", "value":"1"}, - {"type":"STRING", "value":"2"}, - {"type":"STRING", "value":"3"}, - {"type":"STRING", "value":"4"}, - {"type":"STRING", "value":"5"} - ] - } - ``` + ```json + "range": { + "begin": { + {"type":"INF_MIN"}, //指定deviceID最小值 + {"type":"INF_MIN"} //指定deviceID最小值 + }, + "end": { + {"type":"INF_MAX"}, //指定deviceID抽取最大值 + {"type":"INF_MAX"} //指定deviceID抽取最大值 + }, + // 用户指定的切分点,如果指定了切分点,Job将按照begin、end和split进行Task的切分, + // 切分的列只能是Partition Key(ParimaryKey的第一列) + // 支持INF_MIN, INF_MAX, STRING, INT + "split":[ + {"type":"STRING", "value":"1"}, + {"type":"STRING", "value":"2"}, + {"type":"STRING", "value":"3"}, + {"type":"STRING", "value":"4"}, + {"type":"STRING", "value":"5"} + ] + } + ``` - * 必选:否
+ * 必选:否
- * 默认值:无
+ * 默认值:无
### 3.3 类型转换 @@ -291,44 +515,14 @@ OTSReader会根据OTS的表范围,按照Datax并发的数目N,将范围等 * 注意,OTS本身不支持日期型类型。应用层一般使用Long报错时间的Unix TimeStamp。 -## 4 性能报告 -### 4.1 环境准备 +## 4 约束限制 -#### 4.1.1 数据特征 - -15列String(10 Byte), 2两列Integer(8 Byte),总计168Byte/r。 - -#### 4.1.2 机器参数 - -OTS端:3台前端机,5台后端机 - -DataX运行端: 24核CPU, 98GB内存 - -#### 4.1.3 DataX jvm 参数 - - -Xms1024m -Xmx1024m -XX:+HeapDumpOnOutOfMemoryError - -### 4.2 测试报告 - -#### 4.2.1 测试报告 - -|并发数|DataX CPU|OTS 流量|DATAX流量 | 前端QPS| 前端延时| -|--------|--------| --------|--------|--------|------| -|2| 36% |6.3M/s |12739 rec/s | 4.7 | 308ms | -|11| 155% | 32M/s |60732 rec/s | 23.9 | 412ms | -|50| 377% | 73M/s |145139 rec/s | 54 | 874ms | -|100| 448% | 82M/s | 156262 rec/s |60 | 1570ms | - - - -## 5 约束限制 - -### 5.1 一致性约束 +### 4.1 一致性约束 OTS是类BigTable的存储系统,OTS本身能够保证单行写事务性,无法提供跨行级别的事务。对于OTSReader而言也无法提供全表的一致性视图。例如对于OTSReader在0点启动的数据同步任务,在整个表数据同步过程中,OTSReader同样会抽取到后续更新的数据,无法提供准确的0点时刻该表一致性视图。 -### 5.2 增量数据同步 +### 4.2 增量数据同步 OTS本质上KV存储,目前只能针对PK进行范围查询,暂不支持按照字段范围抽取数据。因此只能对于增量查询,如果PK能够表示范围信息,例如自增ID,或者时间戳。 @@ -336,5 +530,4 @@ OTS本质上KV存储,目前只能针对PK进行范围查询,暂不支持按 时间戳, OTSReader可以通过PK过滤时间戳,通过制定Range范围进行增量抽取。这样使用的前提是OTS中的PrimaryKey必须包含主键时间列(时间主键需要使用OTS应用方生成。) -## 6 FAQ - +## 5 FAQ diff --git a/otsreader/pom.xml b/otsreader/pom.xml index eaac8804..dad538bf 100644 --- a/otsreader/pom.xml +++ b/otsreader/pom.xml @@ -1,5 +1,5 @@ + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 com.alibaba.datax @@ -10,17 +10,6 @@ otsreader - - org.apache.logging.log4j - log4j-api - 2.17.1 - - - - org.apache.logging.log4j - log4j-core - 2.17.1 - com.alibaba.datax datax-common @@ -47,22 +36,43 @@ 2.2.4 - log4j-api + log4j-core org.apache.logging.log4j + + + + com.aliyun.openservices + tablestore + 5.13.13 + log4j-core org.apache.logging.log4j - + com.google.code.gson gson 2.2.4 + + com.alibaba + fastjson + 1.2.83_noneautotype + compile + + + + src/main/java + + **/*.properties + + + @@ -98,10 +108,6 @@ maven-surefire-plugin 2.5 - all - 10 - true - -Xmx1024m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=. **/unittest/*.java **/functiontest/*.java @@ -111,4 +117,3 @@ - diff --git a/otsreader/src/main/assembly/package.xml b/otsreader/src/main/assembly/package.xml index 7ee305d1..cb90f3e8 100644 --- a/otsreader/src/main/assembly/package.xml +++ b/otsreader/src/main/assembly/package.xml @@ -12,8 +12,8 @@ src/main/resources plugin.json - plugin_job_template.json - + plugin_job_template.json + plugin/reader/otsreader diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/IOtsReaderMasterProxy.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/IOtsReaderMasterProxy.java new file mode 100644 index 00000000..ee622e16 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/IOtsReaderMasterProxy.java @@ -0,0 +1,15 @@ +package com.alibaba.datax.plugin.reader.otsreader; + +import java.util.List; + +import com.alibaba.datax.common.util.Configuration; + +public interface IOtsReaderMasterProxy { + + public void init(Configuration param) throws Exception; + + public List split(int num) throws Exception; + + public void close(); + +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/IOtsReaderSlaveProxy.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/IOtsReaderSlaveProxy.java new file mode 100644 index 00000000..d1100a2a --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/IOtsReaderSlaveProxy.java @@ -0,0 +1,26 @@ +package com.alibaba.datax.plugin.reader.otsreader; + +import com.alibaba.datax.common.plugin.RecordSender; +import com.alibaba.datax.common.util.Configuration; + +/** + * OTS Reader工作进程接口 + */ +public interface IOtsReaderSlaveProxy { + /** + * 初始化函数,解析配置、初始化相关资源 + */ + public void init(Configuration configuration); + + /** + * 关闭函数,释放资源 + */ + public void close(); + + /** + * 数据导出函数 + * @param recordSender + * @throws Exception + */ + public void startRead(RecordSender recordSender) throws Exception; +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReader.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReader.java index 8880c07e..c6bc44b8 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReader.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReader.java @@ -1,45 +1,48 @@ package com.alibaba.datax.plugin.reader.otsreader; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - 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.plugin.reader.otsreader.utils.Common; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSMode; +import com.alibaba.datax.plugin.reader.otsreader.utils.Constant; +import com.alibaba.datax.plugin.reader.otsreader.utils.GsonParser; +import com.alibaba.datax.plugin.reader.otsreader.utils.OtsReaderError; +import com.alicloud.openservices.tablestore.TableStoreException; import com.aliyun.openservices.ots.ClientException; -import com.aliyun.openservices.ots.OTSException; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + public class OtsReader extends Reader { public static class Job extends Reader.Job { private static final Logger LOG = LoggerFactory.getLogger(Job.class); - private OtsReaderMasterProxy proxy = new OtsReaderMasterProxy(); + //private static final MessageSource MESSAGE_SOURCE = MessageSource.loadResourceBundle(OtsReader.class); + private IOtsReaderMasterProxy proxy = null; + @Override - public void init() { + public void init() { LOG.info("init() begin ..."); + + proxy = new OtsReaderMasterProxy(); try { this.proxy.init(getPluginJobConf()); - } catch (OTSException e) { - LOG.error("OTSException. ErrorCode:{}, ErrorMsg:{}, RequestId:{}", - new Object[]{e.getErrorCode(), e.getMessage(), e.getRequestId()}); - LOG.error("Stack", e); - throw DataXException.asDataXException(new OtsReaderError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); + } catch (TableStoreException e) { + LOG.error("OTSException: {}", e.toString(), e); + throw DataXException.asDataXException(new OtsReaderError(e.getErrorCode(), "OTS ERROR"), e.toString(), e); } catch (ClientException e) { - LOG.error("ClientException. ErrorCode:{}, ErrorMsg:{}", - new Object[]{e.getErrorCode(), e.getMessage()}); - LOG.error("Stack", e); - throw DataXException.asDataXException(new OtsReaderError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); - } catch (IllegalArgumentException e) { - LOG.error("IllegalArgumentException. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsReaderError.INVALID_PARAM, Common.getDetailMessage(e), e); + LOG.error("ClientException: {}", e.toString(), e); + throw DataXException.asDataXException(OtsReaderError.ERROR, e.toString(), e); } catch (Exception e) { - LOG.error("Exception. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsReaderError.ERROR, Common.getDetailMessage(e), e); + LOG.error("Exception. ErrorMsg:{}", e.toString(), e); + throw DataXException.asDataXException(OtsReaderError.ERROR, e.toString(), e); } + LOG.info("init() end ..."); } @@ -60,22 +63,9 @@ public class OtsReader extends Reader { try { confs = this.proxy.split(adviceNumber); - } catch (OTSException e) { - LOG.error("OTSException. ErrorCode:{}, ErrorMsg:{}, RequestId:{}", - new Object[]{e.getErrorCode(), e.getMessage(), e.getRequestId()}); - LOG.error("Stack", e); - throw DataXException.asDataXException(new OtsReaderError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); - } catch (ClientException e) { - LOG.error("ClientException. ErrorCode:{}, ErrorMsg:{}", - new Object[]{e.getErrorCode(), e.getMessage()}); - LOG.error("Stack", e); - throw DataXException.asDataXException(new OtsReaderError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); - } catch (IllegalArgumentException e) { - LOG.error("IllegalArgumentException. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsReaderError.INVALID_PARAM, Common.getDetailMessage(e), e); } catch (Exception e) { LOG.error("Exception. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsReaderError.ERROR, Common.getDetailMessage(e), e); + throw DataXException.asDataXException(OtsReaderError.ERROR, e.toString(), e); } LOG.info("split() end ..."); @@ -85,39 +75,60 @@ public class OtsReader extends Reader { public static class Task extends Reader.Task { private static final Logger LOG = LoggerFactory.getLogger(Task.class); - private OtsReaderSlaveProxy proxy = new OtsReaderSlaveProxy(); + //private static final MessageSource MESSAGE_SOURCE = MessageSource.loadResourceBundle(OtsReader.class); + private IOtsReaderSlaveProxy proxy = null; @Override public void init() { + + OTSConf conf = GsonParser.jsonToConf((String) this.getPluginJobConf().get(Constant.ConfigKey.CONF)); + // 是否使用新接口 + if(conf.isNewVersion()) { + if (conf.getMode() == OTSMode.MULTI_VERSION) { + LOG.info("init OtsReaderSlaveProxyMultiVersion"); + proxy = new OtsReaderSlaveMultiVersionProxy(); + } else { + LOG.info("init OtsReaderSlaveProxyNormal"); + proxy = new OtsReaderSlaveNormalProxy(); + } + + } + else{ + String metaMode = conf.getMetaMode(); + if (StringUtils.isNotBlank(metaMode) && !metaMode.equalsIgnoreCase("false")) { + LOG.info("init OtsMetaReaderSlaveProxy"); + proxy = new OtsReaderSlaveMetaProxy(); + } else { + LOG.info("init OtsReaderSlaveProxyOld"); + proxy = new OtsReaderSlaveProxyOld(); + } + } + + proxy.init(this.getPluginJobConf()); } @Override public void destroy() { + try { + proxy.close(); + } catch (Exception e) { + LOG.error("Exception. ErrorMsg:{}", e.toString(), e); + throw DataXException.asDataXException(OtsReaderError.ERROR, e.toString(), e); + } } @Override public void startRead(RecordSender recordSender) { - LOG.info("startRead() begin ..."); + try { - this.proxy.read(recordSender,getPluginJobConf()); - } catch (OTSException e) { - LOG.error("OTSException. ErrorCode:{}, ErrorMsg:{}, RequestId:{}", - new Object[]{e.getErrorCode(), e.getMessage(), e.getRequestId()}); - LOG.error("Stack", e); - throw DataXException.asDataXException(new OtsReaderError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); - } catch (ClientException e) { - LOG.error("ClientException. ErrorCode:{}, ErrorMsg:{}", - new Object[]{e.getErrorCode(), e.getMessage()}); - LOG.error("Stack", e); - throw DataXException.asDataXException(new OtsReaderError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); - } catch (IllegalArgumentException e) { - LOG.error("IllegalArgumentException. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsReaderError.INVALID_PARAM, Common.getDetailMessage(e), e); + proxy.startRead(recordSender); } catch (Exception e) { - LOG.error("Exception. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsReaderError.ERROR, Common.getDetailMessage(e), e); + LOG.error("Exception. ErrorMsg:{}", e.toString(), e); + throw DataXException.asDataXException(OtsReaderError.ERROR, e.toString(), e); } - LOG.info("startRead() end ..."); + + + } } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderMasterProxy.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderMasterProxy.java index 2b758f06..4ecdd8c1 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderMasterProxy.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderMasterProxy.java @@ -1,221 +1,243 @@ package com.alibaba.datax.plugin.reader.otsreader; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.alibaba.datax.common.util.Configuration; import com.alibaba.datax.plugin.reader.otsreader.callable.GetFirstRowPrimaryKeyCallable; -import com.alibaba.datax.plugin.reader.otsreader.callable.GetTableMetaCallable; import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSConst; import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; -import com.alibaba.datax.plugin.reader.otsreader.utils.ParamChecker; -import com.alibaba.datax.plugin.reader.otsreader.utils.Common; -import com.alibaba.datax.plugin.reader.otsreader.utils.GsonParser; -import com.alibaba.datax.plugin.reader.otsreader.utils.ReaderModelParser; -import com.alibaba.datax.plugin.reader.otsreader.utils.RangeSplit; -import com.alibaba.datax.plugin.reader.otsreader.utils.RetryHelper; -import com.aliyun.openservices.ots.OTSClient; -import com.aliyun.openservices.ots.model.Direction; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; -import com.aliyun.openservices.ots.model.RangeRowQueryCriteria; -import com.aliyun.openservices.ots.model.RowPrimaryKey; -import com.aliyun.openservices.ots.model.TableMeta; +import com.alibaba.datax.plugin.reader.otsreader.utils.*; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.*; +import com.alicloud.openservices.tablestore.model.timeseries.ScanTimeseriesDataResponse; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesScanSplitInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class OtsReaderMasterProxy { +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.List; - private OTSConf conf = new OTSConf(); - - private OTSRange range = null; - - private OTSClient ots = null; - - private TableMeta meta = null; - - private Direction direction = null; +public class OtsReaderMasterProxy implements IOtsReaderMasterProxy { private static final Logger LOG = LoggerFactory.getLogger(OtsReaderMasterProxy.class); + private OTSConf conf = null; + private TableMeta meta = null; + private SyncClientInterface ots = null; + private Direction direction = null; - /** - * 1.检查参数是否为 - * null,endpoint,accessid,accesskey,instance-name,table,column,range-begin,range-end,range-split - * 2.检查参数是否为空字符串 - * endpoint,accessid,accesskey,instance-name,table - * 3.检查是否为空数组 - * column - * 4.检查Range的类型个个数是否和PrimaryKey匹配 - * column,range-begin,range-end - * 5.检查Range Split 顺序和类型是否Range一致,类型是否于PartitionKey一致 - * column-split - * @param param - * @throws Exception - */ - public void init(Configuration param) throws Exception { - // 默认参数 - // 每次重试的时间都是上一次的一倍,当sleep时间大于30秒时,Sleep重试时间不在增长。18次能覆盖OTS的Failover时间5分钟 - conf.setRetry(param.getInt(OTSConst.RETRY, 18)); - conf.setSleepInMilliSecond(param.getInt(OTSConst.SLEEP_IN_MILLI_SECOND, 100)); - - // 必选参数 - conf.setEndpoint(ParamChecker.checkStringAndGet(param, Key.OTS_ENDPOINT)); - conf.setAccessId(ParamChecker.checkStringAndGet(param, Key.OTS_ACCESSID)); - conf.setAccesskey(ParamChecker.checkStringAndGet(param, Key.OTS_ACCESSKEY)); - conf.setInstanceName(ParamChecker.checkStringAndGet(param, Key.OTS_INSTANCE_NAME)); - conf.setTableName(ParamChecker.checkStringAndGet(param, Key.TABLE_NAME)); - - ots = new OTSClient( - this.conf.getEndpoint(), - this.conf.getAccessId(), - this.conf.getAccesskey(), - this.conf.getInstanceName()); - - meta = getTableMeta(ots, conf.getTableName()); - LOG.info("Table Meta : {}", GsonParser.metaToJson(meta)); - - conf.setColumns(ReaderModelParser.parseOTSColumnList(ParamChecker.checkListAndGet(param, Key.COLUMN, true))); - - Map rangeMap = ParamChecker.checkMapAndGet(param, Key.RANGE, true); - conf.setRangeBegin(ReaderModelParser.parsePrimaryKey(ParamChecker.checkListAndGet(rangeMap, Key.RANGE_BEGIN, false))); - conf.setRangeEnd(ReaderModelParser.parsePrimaryKey(ParamChecker.checkListAndGet(rangeMap, Key.RANGE_END, false))); - - range = ParamChecker.checkRangeAndGet(meta, this.conf.getRangeBegin(), this.conf.getRangeEnd()); - - direction = ParamChecker.checkDirectionAndEnd(meta, range.getBegin(), range.getEnd()); - LOG.info("Direction : {}", direction); - - List points = ReaderModelParser.parsePrimaryKey(ParamChecker.checkListAndGet(rangeMap, Key.RANGE_SPLIT)); - ParamChecker.checkInputSplitPoints(meta, range, direction, points); - conf.setRangeSplit(points); - } - - public List split(int num) throws Exception { - LOG.info("Expect split num : " + num); - - List configurations = new ArrayList(); - - List ranges = null; - - if (this.conf.getRangeSplit() != null) { // 用户显示指定了拆分范围 - LOG.info("Begin userDefinedRangeSplit"); - ranges = userDefinedRangeSplit(meta, range, this.conf.getRangeSplit()); - LOG.info("End userDefinedRangeSplit"); - } else { // 采用默认的切分算法 - LOG.info("Begin defaultRangeSplit"); - ranges = defaultRangeSplit(ots, meta, range, num); - LOG.info("End defaultRangeSplit"); - } - - // 解决大量的Split Point序列化消耗内存的问题 - // 因为slave中不会使用这个配置,所以置为空 - this.conf.setRangeSplit(null); - - for (OTSRange item : ranges) { - Configuration configuration = Configuration.newDefault(); - configuration.set(OTSConst.OTS_CONF, GsonParser.confToJson(this.conf)); - configuration.set(OTSConst.OTS_RANGE, GsonParser.rangeToJson(item)); - configuration.set(OTSConst.OTS_DIRECTION, GsonParser.directionToJson(direction)); - configurations.add(configuration); - } - - LOG.info("Configuration list count : " + configurations.size()); - - return configurations; - } public OTSConf getConf() { return conf; } + public TableMeta getMeta() { + return meta; + } + + public SyncClientInterface getOts() { + return ots; + } + + public void setOts(SyncClientInterface ots) { + this.ots = ots; + } + + /** + * 基于配置传入的配置文件,解析为对应的参数 + * + * @param param + * @throws Exception + */ + public void init(Configuration param) throws Exception { + // 基于预定义的Json格式,检查传入参数是否符合Conf定义规范 + conf = OTSConf.load(param); + + // Init ots + ots = OtsHelper.getOTSInstance(conf); + + // 宽行表init + if (!conf.isTimeseriesTable()) { + // 获取TableMeta + meta = OtsHelper.getTableMeta( + ots, + conf.getTableName(), + conf.getRetry(), + conf.getRetryPauseInMillisecond()); + + // 基于Meta检查Conf是否正确 + ParamChecker.checkAndSetOTSConf(conf, meta); + direction = ParamChecker.checkDirectionAndEnd(meta, conf.getRange().getBegin(), conf.getRange().getEnd()); + } + // 时序表 检查tablestore SDK version + if (conf.isTimeseriesTable()){ + Common.checkTableStoreSDKVersion(); + } + + + } + + public List split(int mandatoryNumber) throws Exception { + LOG.info("Expect split num : " + mandatoryNumber); + + List configurations = new ArrayList(); + + if (conf.isTimeseriesTable()) {{ // 时序表全部采用默认切分策略 + LOG.info("Begin timeseries table defaultRangeSplit"); + configurations = getTimeseriesConfigurationBySplit(mandatoryNumber); + LOG.info("End timeseries table defaultRangeSplit"); + }} + else if (this.conf.getRange().getSplit().size() != 0) { // 用户显示指定了拆分范围 + LOG.info("Begin userDefinedRangeSplit"); + configurations = getNormalConfigurationBySplit(); + LOG.info("End userDefinedRangeSplit"); + } else { // 采用默认的切分算法 + LOG.info("Begin defaultRangeSplit"); + configurations = getDefaultConfiguration(mandatoryNumber); + LOG.info("End defaultRangeSplit"); + } + + LOG.info("Expect split num: "+ mandatoryNumber +", and final configuration list count : " + configurations.size()); + return configurations; + } + public void close() { ots.shutdown(); } - // private function - - private TableMeta getTableMeta(OTSClient ots, String tableName) throws Exception { - return RetryHelper.executeWithRetry( - new GetTableMetaCallable(ots, tableName), + /** + * timeseries split信息,根据切分数配置多个Task + */ + private List getTimeseriesConfigurationBySplit(int mandatoryNumber) throws Exception { + List timeseriesScanSplitInfoList = OtsHelper.splitTimeseriesScan( + ots, + conf.getTableName(), + conf.getMeasurementName(), + mandatoryNumber, conf.getRetry(), - conf.getSleepInMilliSecond() - ); + conf.getRetryPauseInMillisecond()); + List configurations = new ArrayList<>(); + + for (int i = 0; i < timeseriesScanSplitInfoList.size(); i++) { + Configuration configuration = Configuration.newDefault(); + configuration.set(Constant.ConfigKey.CONF, GsonParser.confToJson(conf)); + configuration.set(Constant.ConfigKey.SPLIT_INFO, GsonParser.timeseriesScanSplitInfoToString(timeseriesScanSplitInfoList.get(i))); + configurations.add(configuration); + } + return configurations; } - private RowPrimaryKey getPKOfFirstRow( - OTSRange range , Direction direction) throws Exception { + /** + * 根据用户配置的split信息,将配置文件基于Range范围转换为多个Task的配置 + */ + private List getNormalConfigurationBySplit() { + List> primaryKeys = new ArrayList>(); + primaryKeys.add(conf.getRange().getBegin()); + for (PrimaryKeyColumn column : conf.getRange().getSplit()) { + List point = new ArrayList(); + point.add(column); + ParamChecker.fillPrimaryKey(this.meta.getPrimaryKeyList(), point, PrimaryKeyValue.INF_MIN); + primaryKeys.add(point); + } + primaryKeys.add(conf.getRange().getEnd()); - RangeRowQueryCriteria cur = new RangeRowQueryCriteria(this.conf.getTableName()); - cur.setInclusiveStartPrimaryKey(range.getBegin()); - cur.setExclusiveEndPrimaryKey(range.getEnd()); - cur.setLimit(1); - cur.setColumnsToGet(Common.getPrimaryKeyNameList(meta)); - cur.setDirection(direction); + List configurations = new ArrayList(primaryKeys.size() - 1); - return RetryHelper.executeWithRetry( - new GetFirstRowPrimaryKeyCallable(ots, meta, cur), - conf.getRetry(), - conf.getSleepInMilliSecond() - ); + for (int i = 0; i < primaryKeys.size() - 1; i++) { + OTSRange range = new OTSRange(); + range.setBegin(primaryKeys.get(i)); + range.setEnd(primaryKeys.get(i + 1)); + + Configuration configuration = Configuration.newDefault(); + configuration.set(Constant.ConfigKey.CONF, GsonParser.confToJson(conf)); + configuration.set(Constant.ConfigKey.RANGE, GsonParser.rangeToJson(range)); + configuration.set(Constant.ConfigKey.META, GsonParser.metaToJson(meta)); + configurations.add(configuration); + } + return configurations; } - private List defaultRangeSplit(OTSClient ots, TableMeta meta, OTSRange range, int num) throws Exception { + private List getDefaultConfiguration(int num) throws Exception { if (num == 1) { List ranges = new ArrayList(); + OTSRange range = new OTSRange(); + range.setBegin(conf.getRange().getBegin()); + range.setEnd(conf.getRange().getEnd()); ranges.add(range); - return ranges; + + return getConfigurationsFromRanges(ranges); } - + OTSRange reverseRange = new OTSRange(); - reverseRange.setBegin(range.getEnd()); - reverseRange.setEnd(range.getBegin()); + reverseRange.setBegin(conf.getRange().getEnd()); + reverseRange.setEnd(conf.getRange().getBegin()); Direction reverseDirection = (direction == Direction.FORWARD ? Direction.BACKWARD : Direction.FORWARD); - RowPrimaryKey realBegin = getPKOfFirstRow(range, direction); - RowPrimaryKey realEnd = getPKOfFirstRow(reverseRange, reverseDirection); - + List realBegin = getPKOfFirstRow(conf.getRange(), direction); + List realEnd = getPKOfFirstRow(reverseRange, reverseDirection); + // 因为如果其中一行为空,表示这个范围内至多有一行数据 // 所以不再细分,直接使用用户定义的范围 if (realBegin == null || realEnd == null) { List ranges = new ArrayList(); - ranges.add(range); - return ranges; + ranges.add(conf.getRange()); + return getConfigurationsFromRanges(ranges); } - + // 如果出现realBegin,realEnd的方向和direction不一致的情况,直接返回range int cmp = Common.compareRangeBeginAndEnd(meta, realBegin, realEnd); Direction realDirection = cmp > 0 ? Direction.BACKWARD : Direction.FORWARD; if (realDirection != direction) { LOG.warn("Expect '" + direction + "', but direction of realBegin and readlEnd is '" + realDirection + "'"); List ranges = new ArrayList(); - ranges.add(range); - return ranges; + ranges.add(conf.getRange()); + return getConfigurationsFromRanges(ranges); } List ranges = RangeSplit.rangeSplitByCount(meta, realBegin, realEnd, num); if (ranges.isEmpty()) { // 当PartitionKey相等时,工具内部不会切分Range - ranges.add(range); + ranges.add(conf.getRange()); } else { // replace first and last OTSRange first = ranges.get(0); OTSRange last = ranges.get(ranges.size() - 1); - first.setBegin(range.getBegin()); - last.setEnd(range.getEnd()); + first.setBegin(conf.getRange().getBegin()); + last.setEnd(conf.getRange().getEnd()); } - - return ranges; + + return getConfigurationsFromRanges(ranges); } - private List userDefinedRangeSplit(TableMeta meta, OTSRange range, List points) { - List ranges = RangeSplit.rangeSplitByPoint(meta, range.getBegin(), range.getEnd(), points); - if (ranges.isEmpty()) { // 当PartitionKey相等时,工具内部不会切分Range - ranges.add(range); + private List getConfigurationsFromRanges(List ranges){ + List configurationList = new ArrayList<>(); + for (OTSRange range:ranges + ) { + Configuration configuration = Configuration.newDefault(); + configuration.set(Constant.ConfigKey.CONF, GsonParser.confToJson(conf)); + configuration.set(Constant.ConfigKey.RANGE, GsonParser.rangeToJson(range)); + configuration.set(Constant.ConfigKey.META, GsonParser.metaToJson(meta)); + configurationList.add(configuration); } - return ranges; + return configurationList; } + + private List getPKOfFirstRow( + OTSRange range , Direction direction) throws Exception { + + RangeRowQueryCriteria cur = new RangeRowQueryCriteria(this.conf.getTableName()); + cur.setInclusiveStartPrimaryKey(new PrimaryKey(range.getBegin())); + cur.setExclusiveEndPrimaryKey(new PrimaryKey(range.getEnd())); + cur.setLimit(1); + cur.addColumnsToGet(Common.getPrimaryKeyNameList(meta)); + cur.setDirection(direction); + cur.setMaxVersions(1); + + return RetryHelper.executeWithRetry( + new GetFirstRowPrimaryKeyCallable(ots, meta, cur), + conf.getRetry(), + conf.getRetryPauseInMillisecond() + ); + } + } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveMetaProxy.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveMetaProxy.java new file mode 100644 index 00000000..f9860194 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveMetaProxy.java @@ -0,0 +1,160 @@ +package com.alibaba.datax.plugin.reader.otsreader; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; +import com.alibaba.datax.plugin.reader.otsreader.utils.Constant; +import com.alibaba.datax.plugin.reader.otsreader.utils.Key; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.element.StringColumn; +import com.alibaba.datax.common.plugin.RecordSender; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.reader.otsreader.utils.ParamCheckerOld; +import com.alibaba.datax.plugin.reader.otsreader.utils.ReaderModelParser; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; +import com.alibaba.datax.plugin.reader.otsreader.utils.DefaultNoRetry; +import com.alibaba.datax.plugin.reader.otsreader.utils.GsonParser; +import com.alibaba.fastjson.JSON; +import com.aliyun.openservices.ots.OTSClient; +import com.aliyun.openservices.ots.OTSServiceConfiguration; +import com.aliyun.openservices.ots.model.DescribeTableRequest; +import com.aliyun.openservices.ots.model.DescribeTableResult; +import com.aliyun.openservices.ots.model.ListTableResult; +import com.aliyun.openservices.ots.model.PrimaryKeyType; +import com.aliyun.openservices.ots.model.ReservedThroughputDetails; +import com.aliyun.openservices.ots.model.TableMeta; + +public class OtsReaderSlaveMetaProxy implements IOtsReaderSlaveProxy { + + private OTSClient ots = null; + private OTSConf conf = null; + private OTSRange range = null; + private com.alicloud.openservices.tablestore.model.TableMeta meta = null; + private Configuration configuration = null; + private static final Logger LOG = LoggerFactory.getLogger(OtsReaderSlaveMetaProxy.class); + + + @Override + public void init(Configuration configuration) { + OTSServiceConfiguration configure = new OTSServiceConfiguration(); + configure.setRetryStrategy(new DefaultNoRetry()); + + this.configuration = configuration; + conf = GsonParser.jsonToConf((String) configuration.get(Constant.ConfigKey.CONF)); + range = GsonParser.jsonToRange((String) configuration.get(Constant.ConfigKey.RANGE)); + meta = GsonParser.jsonToMeta((String) configuration.get(Constant.ConfigKey.META)); + + String endpoint = conf.getEndpoint(); + String accessId = conf.getAccessId(); + String accessKey = conf.getAccessKey(); + String instanceName = conf.getInstanceName(); + + ots = new OTSClient(endpoint, accessId, accessKey, instanceName, null, configure, null); + } + + @Override + public void close() { + ots.shutdown(); + } + + @Override + public void startRead(RecordSender recordSender) throws Exception { + List columns = ReaderModelParser + .parseOTSColumnList(ParamCheckerOld.checkListAndGet(configuration, Key.COLUMN, true)); + String metaMode = conf.getMetaMode(); // column + + + ListTableResult listTableResult = null; + try { + listTableResult = ots.listTable(); + LOG.info(String.format("ots listTable requestId:%s, traceId:%s", listTableResult.getRequestID(), + listTableResult.getTraceId())); + List allTables = listTableResult.getTableNames(); + for (String eachTable : allTables) { + DescribeTableRequest describeTableRequest = new DescribeTableRequest(); + describeTableRequest.setTableName(eachTable); + DescribeTableResult describeTableResult = ots.describeTable(describeTableRequest); + LOG.info(String.format("ots describeTable requestId:%s, traceId:%s", describeTableResult.getRequestID(), + describeTableResult.getTraceId())); + + TableMeta tableMeta = describeTableResult.getTableMeta(); + // table_name: first_table + // table primary key: type, data type: STRING + // table primary key: db_name, data type: STRING + // table primary key: table_name, data type: STRING + // Reserved throughput: read(0), write(0) + // last increase time: 1502881295 + // last decrease time: None + // number of decreases today: 0 + + String tableName = tableMeta.getTableName(); + Map primaryKey = tableMeta.getPrimaryKey(); + ReservedThroughputDetails reservedThroughputDetails = describeTableResult + .getReservedThroughputDetails(); + int reservedThroughputRead = reservedThroughputDetails.getCapacityUnit().getReadCapacityUnit(); + int reservedThroughputWrite = reservedThroughputDetails.getCapacityUnit().getWriteCapacityUnit(); + long lastIncreaseTime = reservedThroughputDetails.getLastIncreaseTime(); + long lastDecreaseTime = reservedThroughputDetails.getLastDecreaseTime(); + int numberOfDecreasesToday = reservedThroughputDetails.getNumberOfDecreasesToday(); + + Map allData = new HashMap(); + allData.put("endpoint", conf.getEndpoint()); + allData.put("instanceName", conf.getInstanceName()); + allData.put("table", tableName); + // allData.put("primaryKey", JSON.toJSONString(primaryKey)); + allData.put("reservedThroughputRead", reservedThroughputRead + ""); + allData.put("reservedThroughputWrite", reservedThroughputWrite + ""); + allData.put("lastIncreaseTime", lastIncreaseTime + ""); + allData.put("lastDecreaseTime", lastDecreaseTime + ""); + allData.put("numberOfDecreasesToday", numberOfDecreasesToday + ""); + + // 可扩展的可配置的形式 + if ("column".equalsIgnoreCase(metaMode)) { + // 如果是列元数据模式并且column中配置的name是primaryKey,映射成多行DataX Record + List primaryKeyRecords = new ArrayList(); + for (Entry eachPk : primaryKey.entrySet()) { + Record line = recordSender.createRecord(); + for (OTSColumn col : columns) { + if (col.getColumnType() == OTSColumn.OTSColumnType.CONST) { + line.addColumn(col.getValue()); + } else if ("primaryKey.name".equalsIgnoreCase(col.getName())) { + line.addColumn(new StringColumn(eachPk.getKey())); + } else if ("primaryKey.type".equalsIgnoreCase(col.getName())) { + line.addColumn(new StringColumn(eachPk.getValue().name())); + } else { + String v = allData.get(col.getName()); + line.addColumn(new StringColumn(v)); + } + } + LOG.debug("Reader send record : {}", line.toString()); + recordSender.sendToWriter(line); + primaryKeyRecords.add(line); + } + } else { + Record line = recordSender.createRecord(); + for (OTSColumn col : columns) { + if (col.getColumnType() == OTSColumn.OTSColumnType.CONST) { + line.addColumn(col.getValue()); + } else { + String v = allData.get(col.getName()); + line.addColumn(new StringColumn(v)); + } + } + LOG.debug("Reader send record : {}", line.toString()); + recordSender.sendToWriter(line); + } + } + } catch (Exception e) { + LOG.warn(JSON.toJSONString(listTableResult), e); + } + + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveMultiVersionProxy.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveMultiVersionProxy.java new file mode 100644 index 00000000..818a507e --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveMultiVersionProxy.java @@ -0,0 +1,102 @@ +package com.alibaba.datax.plugin.reader.otsreader; + +import com.alibaba.datax.common.element.LongColumn; +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.element.StringColumn; +import com.alibaba.datax.common.plugin.RecordSender; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; +import com.alibaba.datax.plugin.reader.otsreader.utils.*; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class OtsReaderSlaveMultiVersionProxy implements IOtsReaderSlaveProxy { + private OTSConf conf = null; + private OTSRange range = null; + private TableMeta meta = null; + private SyncClientInterface ots = null; + + private static final Logger LOG = LoggerFactory.getLogger(OtsReaderSlaveMultiVersionProxy.class); + + @Override + public void init(Configuration configuration) { + conf = GsonParser.jsonToConf((String) configuration.get(Constant.ConfigKey.CONF)); + range = GsonParser.jsonToRange((String) configuration.get(Constant.ConfigKey.RANGE)); + meta = GsonParser.jsonToMeta((String) configuration.get(Constant.ConfigKey.META)); + + this.ots = OtsHelper.getOTSInstance(conf); + } + + @Override + public void close() { + ots.shutdown(); + } + + private void sendToDatax(RecordSender recordSender, PrimaryKey pk, Column c) { + Record line = recordSender.createRecord(); + //------------------------- + // 四元组 pk, column name, timestamp, value + //------------------------- + + // pk + for( PrimaryKeyColumn pkc : pk.getPrimaryKeyColumns()) { + line.addColumn(TranformHelper.otsPrimaryKeyColumnToDataxColumn(pkc)); + } + // column name + line.addColumn(new StringColumn(c.getName())); + // Timestamp + line.addColumn(new LongColumn(c.getTimestamp())); + // Value + line.addColumn(TranformHelper.otsColumnToDataxColumn(c)); + + recordSender.sendToWriter(line); + } + + private void sendToDatax(RecordSender recordSender, Row row) { + PrimaryKey pk = row.getPrimaryKey(); + for (Column c : row.getColumns()) { + sendToDatax(recordSender, pk, c); + } + } + + /** + * 将获取到的数据采用4元组的方式传递给datax + * @param recordSender + * @param result + */ + private void sendToDatax(RecordSender recordSender, GetRangeResponse result) { + LOG.debug("Per request get row count : " + result.getRows().size()); + for (Row row : result.getRows()) { + sendToDatax(recordSender, row); + } + } + + @Override + public void startRead(RecordSender recordSender) throws Exception { + + PrimaryKey inclusiveStartPrimaryKey = new PrimaryKey(range.getBegin()); + PrimaryKey exclusiveEndPrimaryKey = new PrimaryKey(range.getEnd()); + PrimaryKey next = inclusiveStartPrimaryKey; + + RangeRowQueryCriteria rangeRowQueryCriteria = new RangeRowQueryCriteria(conf.getTableName()); + rangeRowQueryCriteria.setExclusiveEndPrimaryKey(exclusiveEndPrimaryKey); + rangeRowQueryCriteria.setDirection(Common.getDirection(range.getBegin(), range.getEnd())); + rangeRowQueryCriteria.setTimeRange(conf.getMulti().getTimeRange()); + rangeRowQueryCriteria.setMaxVersions(conf.getMulti().getMaxVersion()); + rangeRowQueryCriteria.addColumnsToGet(Common.toColumnToGet(conf.getColumn(), meta)); + + do{ + rangeRowQueryCriteria.setInclusiveStartPrimaryKey(next); + GetRangeResponse result = OtsHelper.getRange( + ots, + rangeRowQueryCriteria, + conf.getRetry(), + conf.getRetryPauseInMillisecond()); + sendToDatax(recordSender, result); + next = result.getNextStartPrimaryKey(); + } while(next != null); + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveNormalProxy.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveNormalProxy.java new file mode 100644 index 00000000..f7d89b15 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveNormalProxy.java @@ -0,0 +1,256 @@ +package com.alibaba.datax.plugin.reader.otsreader; + +import com.alibaba.datax.common.element.LongColumn; +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.util.Configuration; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSCriticalException; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; +import com.alibaba.datax.plugin.reader.otsreader.utils.*; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.core.utils.Pair; +import com.alicloud.openservices.tablestore.model.*; +import com.alicloud.openservices.tablestore.model.timeseries.ScanTimeseriesDataRequest; +import com.alicloud.openservices.tablestore.model.timeseries.ScanTimeseriesDataResponse; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesRow; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesScanSplitInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class OtsReaderSlaveNormalProxy implements IOtsReaderSlaveProxy { + private static final Logger LOG = LoggerFactory.getLogger(OtsReaderSlaveNormalProxy.class); + private OTSConf conf = null; + private OTSRange range = null; + private TableMeta meta = null; + private SyncClientInterface ots = null; + private TimeseriesScanSplitInfo splitInfo = null; + + @Override + public void init(Configuration configuration) { + conf = GsonParser.jsonToConf((String) configuration.get(Constant.ConfigKey.CONF)); + if (!conf.isTimeseriesTable()) { + range = GsonParser.jsonToRange((String) configuration.get(Constant.ConfigKey.RANGE)); + meta = GsonParser.jsonToMeta((String) configuration.get(Constant.ConfigKey.META)); + } else { + splitInfo = GsonParser.stringToTimeseriesScanSplitInfo((String) configuration.get(Constant.ConfigKey.SPLIT_INFO)); + // 时序表 检查tablestore SDK version + try{ + Common.checkTableStoreSDKVersion(); + } + catch (Exception e){ + LOG.error("Exception. ErrorMsg:{}", e.getMessage(), e); + throw DataXException.asDataXException(OtsReaderError.ERROR, e.toString(), e); + } + } + + + this.ots = OtsHelper.getOTSInstance(conf); + } + + @Override + public void close() { + ots.shutdown(); + } + + private void sendToDatax(RecordSender recordSender, Row row) { + Record line = recordSender.createRecord(); + + PrimaryKey pk = row.getPrimaryKey(); + for (OTSColumn column : conf.getColumn()) { + if (column.getColumnType() == OTSColumn.OTSColumnType.NORMAL) { + // 获取指定的列 + PrimaryKeyColumn value = pk.getPrimaryKeyColumn(column.getName()); + if (value != null) { + line.addColumn(TranformHelper.otsPrimaryKeyColumnToDataxColumn(value)); + } else { + Column c = row.getLatestColumn(column.getName()); + if (c != null) { + line.addColumn(TranformHelper.otsColumnToDataxColumn(c)); + } else { + // 这里使用StringColumn的无参构造函数构造对象,而不是用null,下 + // 游(writer)应该通过获取Column,然后通过Column的数据接口的返回值 + // 是否是null来判断改Column是否为null + // Datax其他插件的也是使用这种方式,约定俗成,并没有使用直接向record中注入null方式代表空 + line.addColumn(new StringColumn()); + } + } + } else { + line.addColumn(column.getValue()); + } + } + recordSender.sendToWriter(line); + } + + private void sendToDatax(RecordSender recordSender, TimeseriesRow row) { + + + Record line = recordSender.createRecord(); + // 对于配置项中的每一列 + for (int i = 0; i < conf.getColumn().size(); i++) { + OTSColumn column = conf.getColumn().get(i); + // 如果不是常数列 + if (column.getColumnType() == OTSColumn.OTSColumnType.NORMAL) { + // 如果是tags内字段 + if (conf.getColumn().get(i).getTimeseriesTag()) { + String s = row.getTimeseriesKey().getTags().get(column.getName()); + line.addColumn(new StringColumn(s)); + } + // 如果为measurement字段 + else if (column.getName().equals(Constant.ConfigKey.TimeseriesPKColumn.MEASUREMENT_NAME)) { + String s = row.getTimeseriesKey().getMeasurementName(); + line.addColumn(new StringColumn(s)); + } + // 如果为dataSource字段 + else if (column.getName().equals(Constant.ConfigKey.TimeseriesPKColumn.DATA_SOURCE)) { + String s = row.getTimeseriesKey().getDataSource(); + line.addColumn(new StringColumn(s)); + } + // 如果为tags字段 + else if (column.getName().equals(Constant.ConfigKey.TimeseriesPKColumn.TAGS)) { + line.addColumn(new StringColumn(row.getTimeseriesKey().buildTagsString())); + } + else if (column.getName().equals(Constant.ConfigKey.TimeseriesPKColumn.TIME)) { + Long l = row.getTimeInUs(); + line.addColumn(new LongColumn(l)); + } + // 否则为field内字段 + else { + ColumnValue c = row.getFields().get(column.getName()); + if (c == null) { + LOG.warn("Get column {} : type {} failed, use empty string instead", column.getName(), conf.getColumn().get(i).getValueType()); + line.addColumn(new StringColumn()); + } else if (c.getType() != conf.getColumn().get(i).getValueType()) { + LOG.warn("Get column {} failed, expected type: {}, actual type: {}. Sending actual type to writer.", column.getName(), conf.getColumn().get(i).getValueType(), c.getType()); + line.addColumn(TranformHelper.otsColumnToDataxColumn(c)); + } else { + line.addColumn(TranformHelper.otsColumnToDataxColumn(c)); + } + } + } + // 如果是常数列 + else { + line.addColumn(column.getValue()); + } + } + recordSender.sendToWriter(line); + } + + /** + * 将获取到的数据根据用户配置Column的方式传递给datax + * + * @param recordSender + * @param result + */ + private void sendToDatax(RecordSender recordSender, GetRangeResponse result) { + for (Row row : result.getRows()) { + sendToDatax(recordSender, row); + } + } + + private void sendToDatax(RecordSender recordSender, ScanTimeseriesDataResponse result) { + for (TimeseriesRow row : result.getRows()) { + sendToDatax(recordSender, row); + } + } + + @Override + public void startRead(RecordSender recordSender) throws Exception { + if (conf.isTimeseriesTable()) { + readTimeseriesTable(recordSender); + } else { + readNormalTable(recordSender); + } + } + + public void readTimeseriesTable(RecordSender recordSender) throws Exception { + + List timeseriesPkName = new ArrayList<>(); + timeseriesPkName.add(Constant.ConfigKey.TimeseriesPKColumn.MEASUREMENT_NAME); + timeseriesPkName.add(Constant.ConfigKey.TimeseriesPKColumn.DATA_SOURCE); + timeseriesPkName.add(Constant.ConfigKey.TimeseriesPKColumn.TAGS); + timeseriesPkName.add(Constant.ConfigKey.TimeseriesPKColumn.TIME); + + ScanTimeseriesDataRequest scanTimeseriesDataRequest = new ScanTimeseriesDataRequest(conf.getTableName()); + List> fieldsToGet = new ArrayList<>(); + for (int i = 0; i < conf.getColumn().size(); i++) { + /** + * 如果所配置列 + * 1. 不是常量列(即列名不为null) + * 2. 列名不在["measurementName","dataSource","tags"]中 + * 3. 不是tags内的字段 + * 则为需要获取的field字段。 + */ + String fieldName = conf.getColumn().get(i).getName(); + if (fieldName != null && !timeseriesPkName.contains(fieldName) && !conf.getColumn().get(i).getTimeseriesTag()) { + Pair pair = new Pair<>(fieldName, conf.getColumn().get(i).getValueType()); + fieldsToGet.add(pair); + } + } + scanTimeseriesDataRequest.setFieldsToGet(fieldsToGet); + scanTimeseriesDataRequest.setSplitInfo(splitInfo); + + while (true) { + ScanTimeseriesDataResponse response = OtsHelper.scanTimeseriesData( + ots, + scanTimeseriesDataRequest, + conf.getRetry(), + conf.getRetryPauseInMillisecond()); + sendToDatax(recordSender, response); + if (response.getNextToken() == null) { + break; + } + scanTimeseriesDataRequest.setNextToken(response.getNextToken()); + } + } + + public void readNormalTable(RecordSender recordSender) throws Exception { + PrimaryKey inclusiveStartPrimaryKey = new PrimaryKey(range.getBegin()); + PrimaryKey exclusiveEndPrimaryKey = new PrimaryKey(range.getEnd()); + PrimaryKey next = inclusiveStartPrimaryKey; + + RangeRowQueryCriteria rangeRowQueryCriteria = new RangeRowQueryCriteria(conf.getTableName()); + rangeRowQueryCriteria.setExclusiveEndPrimaryKey(exclusiveEndPrimaryKey); + rangeRowQueryCriteria.setDirection(Common.getDirection(range.getBegin(), range.getEnd())); + rangeRowQueryCriteria.setMaxVersions(1); + rangeRowQueryCriteria.addColumnsToGet(Common.toColumnToGet(conf.getColumn(), meta)); + + do { + rangeRowQueryCriteria.setInclusiveStartPrimaryKey(next); + GetRangeResponse result = OtsHelper.getRange( + ots, + rangeRowQueryCriteria, + conf.getRetry(), + conf.getRetryPauseInMillisecond()); + sendToDatax(recordSender, result); + next = result.getNextStartPrimaryKey(); + } while (next != null); + } + + + public void setConf(OTSConf conf) { + this.conf = conf; + } + + + public void setRange(OTSRange range) { + this.range = range; + } + + + public void setMeta(TableMeta meta) { + this.meta = meta; + } + + + public void setOts(SyncClientInterface ots) { + this.ots = ots; + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveProxy.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveProxy.java deleted file mode 100644 index e64b4e7e..00000000 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveProxy.java +++ /dev/null @@ -1,135 +0,0 @@ -package com.alibaba.datax.plugin.reader.otsreader; - -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.alibaba.datax.common.element.Record; -import com.alibaba.datax.common.plugin.RecordSender; -import com.alibaba.datax.common.util.Configuration; -import com.alibaba.datax.plugin.reader.otsreader.callable.GetRangeCallable; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSConst; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; -import com.alibaba.datax.plugin.reader.otsreader.utils.Common; -import com.alibaba.datax.plugin.reader.otsreader.utils.GsonParser; -import com.alibaba.datax.plugin.reader.otsreader.utils.DefaultNoRetry; -import com.alibaba.datax.plugin.reader.otsreader.utils.RetryHelper; -import com.aliyun.openservices.ots.OTSClientAsync; -import com.aliyun.openservices.ots.OTSServiceConfiguration; -import com.aliyun.openservices.ots.model.Direction; -import com.aliyun.openservices.ots.model.GetRangeRequest; -import com.aliyun.openservices.ots.model.GetRangeResult; -import com.aliyun.openservices.ots.model.OTSFuture; -import com.aliyun.openservices.ots.model.RangeRowQueryCriteria; -import com.aliyun.openservices.ots.model.Row; -import com.aliyun.openservices.ots.model.RowPrimaryKey; - -public class OtsReaderSlaveProxy { - - class RequestItem { - private RangeRowQueryCriteria criteria; - private OTSFuture future; - - RequestItem(RangeRowQueryCriteria criteria, OTSFuture future) { - this.criteria = criteria; - this.future = future; - } - - public RangeRowQueryCriteria getCriteria() { - return criteria; - } - - public OTSFuture getFuture() { - return future; - } - } - - private static final Logger LOG = LoggerFactory.getLogger(OtsReaderSlaveProxy.class); - - private void rowsToSender(List rows, RecordSender sender, List columns) { - for (Row row : rows) { - Record line = sender.createRecord(); - line = Common.parseRowToLine(row, columns, line); - - LOG.debug("Reader send record : {}", line.toString()); - - sender.sendToWriter(line); - } - } - - private RangeRowQueryCriteria generateRangeRowQueryCriteria(String tableName, RowPrimaryKey begin, RowPrimaryKey end, Direction direction, List columns) { - RangeRowQueryCriteria criteria = new RangeRowQueryCriteria(tableName); - criteria.setInclusiveStartPrimaryKey(begin); - criteria.setDirection(direction); - criteria.setColumnsToGet(columns); - criteria.setLimit(-1); - criteria.setExclusiveEndPrimaryKey(end); - return criteria; - } - - private RequestItem generateRequestItem( - OTSClientAsync ots, - OTSConf conf, - RowPrimaryKey begin, - RowPrimaryKey end, - Direction direction, - List columns) throws Exception { - RangeRowQueryCriteria criteria = generateRangeRowQueryCriteria(conf.getTableName(), begin, end, direction, columns); - - GetRangeRequest request = new GetRangeRequest(); - request.setRangeRowQueryCriteria(criteria); - OTSFuture future = ots.getRange(request); - - return new RequestItem(criteria, future); - } - - public void read(RecordSender sender, Configuration configuration) throws Exception { - LOG.info("read begin."); - - OTSConf conf = GsonParser.jsonToConf(configuration.getString(OTSConst.OTS_CONF)); - OTSRange range = GsonParser.jsonToRange(configuration.getString(OTSConst.OTS_RANGE)); - Direction direction = GsonParser.jsonToDirection(configuration.getString(OTSConst.OTS_DIRECTION)); - - OTSServiceConfiguration configure = new OTSServiceConfiguration(); - configure.setRetryStrategy(new DefaultNoRetry()); - - OTSClientAsync ots = new OTSClientAsync( - conf.getEndpoint(), - conf.getAccessId(), - conf.getAccesskey(), - conf.getInstanceName(), - null, - configure, - null); - - RowPrimaryKey token = range.getBegin(); - List columns = Common.getNormalColumnNameList(conf.getColumns()); - - RequestItem request = null; - - do { - LOG.debug("Next token : {}", GsonParser.rowPrimaryKeyToJson(token)); - if (request == null) { - request = generateRequestItem(ots, conf, token, range.getEnd(), direction, columns); - } else { - RequestItem req = request; - - GetRangeResult result = RetryHelper.executeWithRetry( - new GetRangeCallable(ots, req.getCriteria(), req.getFuture()), - conf.getRetry(), - conf.getSleepInMilliSecond() - ); - if ((token = result.getNextStartPrimaryKey()) != null) { - request = generateRequestItem(ots, conf, token, range.getEnd(), direction, columns); - } - - rowsToSender(result.getRows(), sender, conf.getColumns()); - } - } while (token != null); - ots.shutdown(); - LOG.info("read end."); - } -} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveProxyOld.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveProxyOld.java new file mode 100644 index 00000000..72eb885e --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderSlaveProxyOld.java @@ -0,0 +1,181 @@ +package com.alibaba.datax.plugin.reader.otsreader; + +import java.util.List; + +import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; +import com.alibaba.datax.plugin.reader.otsreader.utils.*; +import com.alicloud.openservices.tablestore.model.PrimaryKeyColumn; +import com.aliyun.openservices.ots.model.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.plugin.RecordSender; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.reader.otsreader.callable.GetRangeCallableOld; +import com.aliyun.openservices.ots.OTSClientAsync; +import com.aliyun.openservices.ots.OTSServiceConfiguration; + +public class OtsReaderSlaveProxyOld implements IOtsReaderSlaveProxy { + + + private OTSClientAsync ots = null; + private OTSConf conf = null; + private OTSRange range = null; + + class RequestItem { + private RangeRowQueryCriteria criteria; + private OTSFuture future; + + RequestItem(RangeRowQueryCriteria criteria, OTSFuture future) { + this.criteria = criteria; + this.future = future; + } + + public RangeRowQueryCriteria getCriteria() { + return criteria; + } + + public OTSFuture getFuture() { + return future; + } + } + + private static final Logger LOG = LoggerFactory.getLogger(OtsReaderSlaveProxyOld.class); + + private void rowsToSender(List rows, RecordSender sender, List columns) { + for (Row row : rows) { + Record line = sender.createRecord(); + line = CommonOld.parseRowToLine(row, columns, line); + + LOG.debug("Reader send record : {}", line.toString()); + + sender.sendToWriter(line); + } + } + + private RangeRowQueryCriteria generateRangeRowQueryCriteria(String tableName, RowPrimaryKey begin, RowPrimaryKey end, Direction direction, List columns) { + RangeRowQueryCriteria criteria = new RangeRowQueryCriteria(tableName); + criteria.setInclusiveStartPrimaryKey(begin); + criteria.setDirection(direction); + criteria.setColumnsToGet(columns); + criteria.setLimit(-1); + criteria.setExclusiveEndPrimaryKey(end); + return criteria; + } + + private RequestItem generateRequestItem( + OTSClientAsync ots, + OTSConf conf, + RowPrimaryKey begin, + RowPrimaryKey end, + Direction direction, + List columns) throws Exception { + RangeRowQueryCriteria criteria = generateRangeRowQueryCriteria(conf.getTableName(), begin, end, direction, columns); + + GetRangeRequest request = new GetRangeRequest(); + request.setRangeRowQueryCriteria(criteria); + OTSFuture future = ots.getRange(request); + + return new RequestItem(criteria, future); + } + + @Override + public void init(Configuration configuration) { + conf = GsonParser.jsonToConf(configuration.getString(Constant.ConfigKey.CONF)); + range = GsonParser.jsonToRange(configuration.getString(Constant.ConfigKey.RANGE)); + + OTSServiceConfiguration configure = new OTSServiceConfiguration(); + configure.setRetryStrategy(new DefaultNoRetry()); + + ots = new OTSClientAsync( + conf.getEndpoint(), + conf.getAccessId(), + conf.getAccessKey(), + conf.getInstanceName(), + null, + configure, + null); + } + + @Override + public void close() { + ots.shutdown(); + } + + @Override + public void startRead(RecordSender recordSender) throws Exception { + RowPrimaryKey token = pKColumnList2RowPrimaryKey(range.getBegin()); + + List columns = CommonOld.getNormalColumnNameList(conf.getColumn()); + Direction direction = null; + switch (Common.getDirection(range.getBegin(), range.getEnd())){ + case FORWARD: + direction = Direction.FORWARD; + break; + case BACKWARD: + default: + direction = Direction.BACKWARD; + } + RequestItem request = null; + + do { + LOG.debug("Next token : {}", GsonParser.rowPrimaryKeyToJson(token)); + if (request == null) { + request = generateRequestItem(ots, conf, token, pKColumnList2RowPrimaryKey(range.getEnd()), direction, columns); + } else { + RequestItem req = request; + + GetRangeResult result = RetryHelperOld.executeWithRetry( + new GetRangeCallableOld(ots, req.getCriteria(), req.getFuture()), + conf.getRetry(), + // TODO + 100 + ); + if ((token = result.getNextStartPrimaryKey()) != null) { + request = generateRequestItem(ots, conf, token, pKColumnList2RowPrimaryKey(range.getEnd()), direction, columns); + } + + rowsToSender(result.getRows(), recordSender, conf.getColumn()); + } + } while (token != null); + } + + /** + * 将 {@link com.alicloud.openservices.tablestore.model.PrimaryKeyColumn}的列表转为{@link com.aliyun.openservices.ots.model.RowPrimaryKey} + * @param list + * @return + */ + public RowPrimaryKey pKColumnList2RowPrimaryKey(List list){ + RowPrimaryKey rowPrimaryKey = new RowPrimaryKey(); + for(PrimaryKeyColumn pk : list){ + PrimaryKeyValue v = null; + if(pk.getValue() == com.alicloud.openservices.tablestore.model.PrimaryKeyValue.INF_MAX){ + v = PrimaryKeyValue.INF_MAX; + } else if (pk.getValue() == com.alicloud.openservices.tablestore.model.PrimaryKeyValue.INF_MIN) { + v = PrimaryKeyValue.INF_MIN; + } + // 非INF_MAX 或 INF_MIN + else{ + switch (pk.getValue().getType()){ + case STRING: + v = PrimaryKeyValue.fromString(pk.getValue().asString()); + break; + case INTEGER: + v = PrimaryKeyValue.fromLong(pk.getValue().asLong()); + break; + case BINARY: + v = PrimaryKeyValue.fromBinary(pk.getValue().asBinary()); + break; + default: + throw new IllegalArgumentException("the pKColumnList to RowPrimaryKey conversion failed"); + } + } + + rowPrimaryKey.addPrimaryKeyColumn(pk.getName(),v); + } + return rowPrimaryKey; + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/ColumnAdaptor.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/ColumnAdaptor.java new file mode 100644 index 00000000..b2e14b5c --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/ColumnAdaptor.java @@ -0,0 +1,63 @@ +package com.alibaba.datax.plugin.reader.otsreader.adaptor; + +import com.alibaba.datax.common.element.*; +import com.google.gson.*; +import org.apache.commons.codec.binary.Base64; + +import java.lang.reflect.Type; + +public class ColumnAdaptor implements JsonDeserializer, JsonSerializer{ + private final static String TYPE = "type"; + private final static String RAW = "rawData"; + + @Override + public JsonElement serialize(Column obj, Type t, + JsonSerializationContext c) { + JsonObject json = new JsonObject(); + + String rawData = null; + switch (obj.getType()){ + case BOOL: + rawData = String.valueOf(obj.getRawData()); break; + case BYTES: + rawData = Base64.encodeBase64String((byte[]) obj.getRawData()); break; + case DOUBLE: + rawData = String.valueOf(obj.getRawData());break; + case LONG: + rawData = String.valueOf(obj.getRawData());break; + case STRING: + rawData = String.valueOf(obj.getRawData());break; + default: + throw new IllegalArgumentException("Unsupport parse the column type:" + obj.getType().toString()); + + } + json.add(TYPE, new JsonPrimitive(obj.getType().toString())); + json.add(RAW, new JsonPrimitive(rawData)); + return json; + } + + @Override + public Column deserialize(JsonElement ele, Type t, + JsonDeserializationContext c) throws JsonParseException { + JsonObject obj = ele.getAsJsonObject(); + + String strType = obj.getAsJsonPrimitive(TYPE).getAsString(); + String strRaw = obj.getAsJsonPrimitive(RAW).getAsString(); + Column.Type type = Column.Type.valueOf(strType); + switch (type){ + case BOOL: + return new BoolColumn(strRaw); + case BYTES: + return new BytesColumn(Base64.decodeBase64(strRaw)); + case DOUBLE: + return new DoubleColumn(strRaw); + case LONG: + return new LongColumn(strRaw); + case STRING: + return new StringColumn(strRaw); + default: + throw new IllegalArgumentException("Unsupport parse the column type:" + type.toString()); + + } + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/OTSColumnAdaptor.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/OTSColumnAdaptor.java deleted file mode 100644 index 25f9b682..00000000 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/OTSColumnAdaptor.java +++ /dev/null @@ -1,117 +0,0 @@ -package com.alibaba.datax.plugin.reader.otsreader.adaptor; - -import java.lang.reflect.Type; - -import org.apache.commons.codec.binary.Base64; - -import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; -import com.aliyun.openservices.ots.model.ColumnType; -import com.google.gson.JsonDeserializationContext; -import com.google.gson.JsonDeserializer; -import com.google.gson.JsonElement; -import com.google.gson.JsonObject; -import com.google.gson.JsonParseException; -import com.google.gson.JsonPrimitive; -import com.google.gson.JsonSerializationContext; -import com.google.gson.JsonSerializer; - -public class OTSColumnAdaptor implements JsonDeserializer, JsonSerializer{ - private final static String NAME = "name"; - private final static String COLUMN_TYPE = "column_type"; - private final static String VALUE_TYPE = "value_type"; - private final static String VALUE = "value"; - - private void serializeConstColumn(JsonObject json, OTSColumn obj) { - switch (obj.getValueType()) { - case STRING : - json.add(VALUE_TYPE, new JsonPrimitive(ColumnType.STRING.toString())); - json.add(VALUE, new JsonPrimitive(obj.getValue().asString())); - break; - case INTEGER : - json.add(VALUE_TYPE, new JsonPrimitive(ColumnType.INTEGER.toString())); - json.add(VALUE, new JsonPrimitive(obj.getValue().asLong())); - break; - case DOUBLE : - json.add(VALUE_TYPE, new JsonPrimitive(ColumnType.DOUBLE.toString())); - json.add(VALUE, new JsonPrimitive(obj.getValue().asDouble())); - break; - case BOOLEAN : - json.add(VALUE_TYPE, new JsonPrimitive(ColumnType.BOOLEAN.toString())); - json.add(VALUE, new JsonPrimitive(obj.getValue().asBoolean())); - break; - case BINARY : - json.add(VALUE_TYPE, new JsonPrimitive(ColumnType.BINARY.toString())); - json.add(VALUE, new JsonPrimitive(Base64.encodeBase64String(obj.getValue().asBytes()))); - break; - default: - throw new IllegalArgumentException("Unsupport serialize the type : " + obj.getValueType() + ""); - } - } - - private OTSColumn deserializeConstColumn(JsonObject obj) { - String strType = obj.getAsJsonPrimitive(VALUE_TYPE).getAsString(); - ColumnType type = ColumnType.valueOf(strType); - - JsonPrimitive jsonValue = obj.getAsJsonPrimitive(VALUE); - - switch (type) { - case STRING : - return OTSColumn.fromConstStringColumn(jsonValue.getAsString()); - case INTEGER : - return OTSColumn.fromConstIntegerColumn(jsonValue.getAsLong()); - case DOUBLE : - return OTSColumn.fromConstDoubleColumn(jsonValue.getAsDouble()); - case BOOLEAN : - return OTSColumn.fromConstBoolColumn(jsonValue.getAsBoolean()); - case BINARY : - return OTSColumn.fromConstBytesColumn(Base64.decodeBase64(jsonValue.getAsString())); - default: - throw new IllegalArgumentException("Unsupport deserialize the type : " + type + ""); - } - } - - private void serializeNormalColumn(JsonObject json, OTSColumn obj) { - json.add(NAME, new JsonPrimitive(obj.getName())); - } - - private OTSColumn deserializeNormarlColumn(JsonObject obj) { - return OTSColumn.fromNormalColumn(obj.getAsJsonPrimitive(NAME).getAsString()); - } - - @Override - public JsonElement serialize(OTSColumn obj, Type t, - JsonSerializationContext c) { - JsonObject json = new JsonObject(); - - switch (obj.getColumnType()) { - case CONST: - json.add(COLUMN_TYPE, new JsonPrimitive(OTSColumn.OTSColumnType.CONST.toString())); - serializeConstColumn(json, obj); - break; - case NORMAL: - json.add(COLUMN_TYPE, new JsonPrimitive(OTSColumn.OTSColumnType.NORMAL.toString())); - serializeNormalColumn(json, obj); - break; - default: - throw new IllegalArgumentException("Unsupport serialize the type : " + obj.getColumnType() + ""); - } - return json; - } - - @Override - public OTSColumn deserialize(JsonElement ele, Type t, - JsonDeserializationContext c) throws JsonParseException { - JsonObject obj = ele.getAsJsonObject(); - String strColumnType = obj.getAsJsonPrimitive(COLUMN_TYPE).getAsString(); - OTSColumn.OTSColumnType columnType = OTSColumn.OTSColumnType.valueOf(strColumnType); - - switch(columnType) { - case CONST: - return deserializeConstColumn(obj); - case NORMAL: - return deserializeNormarlColumn(obj); - default: - throw new IllegalArgumentException("Unsupport deserialize the type : " + columnType + ""); - } - } -} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/PrimaryKeyValueAdaptor.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/PrimaryKeyValueAdaptor.java index 1a49ea47..240427ae 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/PrimaryKeyValueAdaptor.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/adaptor/PrimaryKeyValueAdaptor.java @@ -1,18 +1,12 @@ package com.alibaba.datax.plugin.reader.otsreader.adaptor; -import java.lang.reflect.Type; +import com.alicloud.openservices.tablestore.model.ColumnType; +import com.alicloud.openservices.tablestore.model.PrimaryKeyType; +import com.alicloud.openservices.tablestore.model.PrimaryKeyValue; +import com.google.gson.*; +import org.apache.commons.codec.binary.Base64; -import com.aliyun.openservices.ots.model.ColumnType; -import com.aliyun.openservices.ots.model.PrimaryKeyType; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; -import com.google.gson.JsonDeserializationContext; -import com.google.gson.JsonDeserializer; -import com.google.gson.JsonElement; -import com.google.gson.JsonObject; -import com.google.gson.JsonParseException; -import com.google.gson.JsonPrimitive; -import com.google.gson.JsonSerializationContext; -import com.google.gson.JsonSerializer; +import java.lang.reflect.Type; /** * {"type":"INF_MIN", "value":""} @@ -31,27 +25,29 @@ public class PrimaryKeyValueAdaptor implements JsonDeserializer JsonSerializationContext c) { JsonObject json = new JsonObject(); - if (obj == PrimaryKeyValue.INF_MIN) { + if (obj.isInfMin()) { json.add(TYPE, new JsonPrimitive(INF_MIN)); - json.add(VALUE, new JsonPrimitive("")); return json; } - if (obj == PrimaryKeyValue.INF_MAX) { + if (obj.isInfMax()) { json.add(TYPE, new JsonPrimitive(INF_MAX)); - json.add(VALUE, new JsonPrimitive("")); return json; } switch (obj.getType()) { case STRING : - json.add(TYPE, new JsonPrimitive(ColumnType.STRING.toString())); + json.add(TYPE, new JsonPrimitive(ColumnType.STRING.toString())); json.add(VALUE, new JsonPrimitive(obj.asString())); break; case INTEGER : json.add(TYPE, new JsonPrimitive(ColumnType.INTEGER.toString())); json.add(VALUE, new JsonPrimitive(obj.asLong())); break; + case BINARY : + json.add(TYPE, new JsonPrimitive(ColumnType.BINARY.toString())); + json.add(VALUE, new JsonPrimitive(Base64.encodeBase64String(obj.asBinary()))); + break; default: throw new IllegalArgumentException("Unsupport serialize the type : " + obj.getType() + ""); } @@ -64,16 +60,17 @@ public class PrimaryKeyValueAdaptor implements JsonDeserializer JsonObject obj = ele.getAsJsonObject(); String strType = obj.getAsJsonPrimitive(TYPE).getAsString(); - JsonPrimitive jsonValue = obj.getAsJsonPrimitive(VALUE); - if (strType.equals(INF_MIN)) { + if (strType.equalsIgnoreCase(INF_MIN)) { return PrimaryKeyValue.INF_MIN; } - if (strType.equals(INF_MAX)) { + if (strType.equalsIgnoreCase(INF_MAX)) { return PrimaryKeyValue.INF_MAX; } + JsonPrimitive jsonValue = obj.getAsJsonPrimitive(VALUE); + PrimaryKeyValue value = null; PrimaryKeyType type = PrimaryKeyType.valueOf(strType); switch(type) { @@ -83,6 +80,9 @@ public class PrimaryKeyValueAdaptor implements JsonDeserializer case INTEGER : value = PrimaryKeyValue.fromLong(jsonValue.getAsLong()); break; + case BINARY : + value = PrimaryKeyValue.fromBinary(Base64.decodeBase64(jsonValue.getAsString())); + break; default: throw new IllegalArgumentException("Unsupport deserialize the type : " + type + ""); } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetFirstRowPrimaryKeyCallable.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetFirstRowPrimaryKeyCallable.java index f004c0ff..cdcae91a 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetFirstRowPrimaryKeyCallable.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetFirstRowPrimaryKeyCallable.java @@ -1,53 +1,42 @@ package com.alibaba.datax.plugin.reader.otsreader.callable; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.*; + +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; -import com.aliyun.openservices.ots.OTSClient; -import com.aliyun.openservices.ots.model.ColumnType; -import com.aliyun.openservices.ots.model.ColumnValue; -import com.aliyun.openservices.ots.model.GetRangeRequest; -import com.aliyun.openservices.ots.model.GetRangeResult; -import com.aliyun.openservices.ots.model.PrimaryKeyType; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; -import com.aliyun.openservices.ots.model.RangeRowQueryCriteria; -import com.aliyun.openservices.ots.model.Row; -import com.aliyun.openservices.ots.model.RowPrimaryKey; -import com.aliyun.openservices.ots.model.TableMeta; +public class GetFirstRowPrimaryKeyCallable implements Callable> { -public class GetFirstRowPrimaryKeyCallable implements Callable{ - - private OTSClient ots = null; + private SyncClientInterface ots = null; private TableMeta meta = null; private RangeRowQueryCriteria criteria = null; - - public GetFirstRowPrimaryKeyCallable(OTSClient ots, TableMeta meta, RangeRowQueryCriteria criteria) { + + public GetFirstRowPrimaryKeyCallable(SyncClientInterface ots, TableMeta meta, RangeRowQueryCriteria criteria) { this.ots = ots; this.meta = meta; this.criteria = criteria; } - + @Override - public RowPrimaryKey call() throws Exception { - RowPrimaryKey ret = new RowPrimaryKey(); + public List call() throws Exception { + List ret = new ArrayList<>(); GetRangeRequest request = new GetRangeRequest(); request.setRangeRowQueryCriteria(criteria); - GetRangeResult result = ots.getRange(request); - List rows = result.getRows(); - if(rows.isEmpty()) { + GetRangeResponse response = ots.getRange(request); + List rows = response.getRows(); + if (rows.isEmpty()) { return null;// no data - } + } Row row = rows.get(0); - Map pk = meta.getPrimaryKey(); - for (String key:pk.keySet()) { - ColumnValue v = row.getColumns().get(key); - if (v.getType() == ColumnType.INTEGER) { - ret.addPrimaryKeyColumn(key, PrimaryKeyValue.fromLong(v.asLong())); - } else { - ret.addPrimaryKeyColumn(key, PrimaryKeyValue.fromString(v.asString())); - } + Map pk = meta.getPrimaryKeyMap(); + + for (String key : pk.keySet()) { + PrimaryKeyColumn v = row.getPrimaryKey().getPrimaryKeyColumnsMap().get(key); + ret.add(v); } return ret; } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetRangeCallable.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetRangeCallable.java index 2cd1398a..995d491c 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetRangeCallable.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetRangeCallable.java @@ -1,35 +1,26 @@ package com.alibaba.datax.plugin.reader.otsreader.callable; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.GetRangeRequest; +import com.alicloud.openservices.tablestore.model.GetRangeResponse; +import com.alicloud.openservices.tablestore.model.RangeRowQueryCriteria; + import java.util.concurrent.Callable; -import com.aliyun.openservices.ots.OTSClientAsync; -import com.aliyun.openservices.ots.model.GetRangeRequest; -import com.aliyun.openservices.ots.model.GetRangeResult; -import com.aliyun.openservices.ots.model.OTSFuture; -import com.aliyun.openservices.ots.model.RangeRowQueryCriteria; - -public class GetRangeCallable implements Callable { +public class GetRangeCallable implements Callable { - private OTSClientAsync ots; + private SyncClientInterface ots; private RangeRowQueryCriteria criteria; - private OTSFuture future; - public GetRangeCallable(OTSClientAsync ots, RangeRowQueryCriteria criteria, OTSFuture future) { + public GetRangeCallable(SyncClientInterface ots, RangeRowQueryCriteria criteria) { this.ots = ots; this.criteria = criteria; - this.future = future; } @Override - public GetRangeResult call() throws Exception { - try { - return future.get(); - } catch (Exception e) { - GetRangeRequest request = new GetRangeRequest(); - request.setRangeRowQueryCriteria(criteria); - future = ots.getRange(request); - throw e; - } + public GetRangeResponse call() throws Exception { + GetRangeRequest request = new GetRangeRequest(); + request.setRangeRowQueryCriteria(criteria); + return ots.getRange(request); } - -} +} \ No newline at end of file diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetRangeCallableOld.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetRangeCallableOld.java new file mode 100644 index 00000000..c0434126 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetRangeCallableOld.java @@ -0,0 +1,35 @@ +package com.alibaba.datax.plugin.reader.otsreader.callable; + +import java.util.concurrent.Callable; + +import com.aliyun.openservices.ots.OTSClientAsync; +import com.aliyun.openservices.ots.model.GetRangeRequest; +import com.aliyun.openservices.ots.model.GetRangeResult; +import com.aliyun.openservices.ots.model.OTSFuture; +import com.aliyun.openservices.ots.model.RangeRowQueryCriteria; + +public class GetRangeCallableOld implements Callable { + + private OTSClientAsync ots; + private RangeRowQueryCriteria criteria; + private OTSFuture future; + + public GetRangeCallableOld(OTSClientAsync ots, RangeRowQueryCriteria criteria, OTSFuture future) { + this.ots = ots; + this.criteria = criteria; + this.future = future; + } + + @Override + public GetRangeResult call() throws Exception { + try { + return future.get(); + } catch (Exception e) { + GetRangeRequest request = new GetRangeRequest(); + request.setRangeRowQueryCriteria(criteria); + future = ots.getRange(request); + throw e; + } + } + +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetTableMetaCallable.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetTableMetaCallable.java index 2884e12b..36a122c2 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetTableMetaCallable.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetTableMetaCallable.java @@ -1,18 +1,19 @@ package com.alibaba.datax.plugin.reader.otsreader.callable; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.DescribeTableRequest; +import com.alicloud.openservices.tablestore.model.DescribeTableResponse; +import com.alicloud.openservices.tablestore.model.TableMeta; + import java.util.concurrent.Callable; -import com.aliyun.openservices.ots.OTSClient; -import com.aliyun.openservices.ots.model.DescribeTableRequest; -import com.aliyun.openservices.ots.model.DescribeTableResult; -import com.aliyun.openservices.ots.model.TableMeta; public class GetTableMetaCallable implements Callable{ - private OTSClient ots = null; + private SyncClientInterface ots = null; private String tableName = null; - public GetTableMetaCallable(OTSClient ots, String tableName) { + public GetTableMetaCallable(SyncClientInterface ots, String tableName) { this.ots = ots; this.tableName = tableName; } @@ -21,9 +22,9 @@ public class GetTableMetaCallable implements Callable{ public TableMeta call() throws Exception { DescribeTableRequest describeTableRequest = new DescribeTableRequest(); describeTableRequest.setTableName(tableName); - DescribeTableResult result = ots.describeTable(describeTableRequest); + DescribeTableResponse result = ots.describeTable(describeTableRequest); TableMeta tableMeta = result.getTableMeta(); return tableMeta; } -} +} \ No newline at end of file diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetTimeseriesSplitCallable.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetTimeseriesSplitCallable.java new file mode 100644 index 00000000..96521c41 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/GetTimeseriesSplitCallable.java @@ -0,0 +1,38 @@ +package com.alibaba.datax.plugin.reader.otsreader.callable; + +import com.alicloud.openservices.tablestore.SyncClient; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.TimeseriesClient; +import com.alicloud.openservices.tablestore.model.timeseries.SplitTimeseriesScanTaskRequest; +import com.alicloud.openservices.tablestore.model.timeseries.SplitTimeseriesScanTaskResponse; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesScanSplitInfo; + +import java.util.List; +import java.util.concurrent.Callable; + +public class GetTimeseriesSplitCallable implements Callable> { + + private TimeseriesClient client = null; + private String timeseriesTableName = null; + private String measurementName = null; + private int splitCountHint = 1; + + + public GetTimeseriesSplitCallable(SyncClientInterface ots, String timeseriesTableName, String measurementName, int splitCountHint) { + this.client = ((SyncClient) ots).asTimeseriesClient(); + this.timeseriesTableName = timeseriesTableName; + this.measurementName = measurementName; + this.splitCountHint = splitCountHint; + } + + @Override + public List call() throws Exception { + SplitTimeseriesScanTaskRequest request = new SplitTimeseriesScanTaskRequest(timeseriesTableName, splitCountHint); + if (measurementName.length() != 0) { + request.setMeasurementName(measurementName); + } + + SplitTimeseriesScanTaskResponse response = client.splitTimeseriesScanTask(request); + return response.getSplitInfos(); + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/ScanTimeseriesDataCallable.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/ScanTimeseriesDataCallable.java new file mode 100644 index 00000000..726d0e5d --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/callable/ScanTimeseriesDataCallable.java @@ -0,0 +1,27 @@ +package com.alibaba.datax.plugin.reader.otsreader.callable; + +import com.alicloud.openservices.tablestore.SyncClient; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.TimeseriesClient; +import com.alicloud.openservices.tablestore.model.timeseries.ScanTimeseriesDataRequest; +import com.alicloud.openservices.tablestore.model.timeseries.ScanTimeseriesDataResponse; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesScanSplitInfo; + +import java.util.List; +import java.util.concurrent.Callable; + +public class ScanTimeseriesDataCallable implements Callable { + + private TimeseriesClient client = null; + private ScanTimeseriesDataRequest request = null; + + public ScanTimeseriesDataCallable(SyncClientInterface ots, ScanTimeseriesDataRequest scanTimeseriesDataRequest){ + this.client = ((SyncClient) ots).asTimeseriesClient(); + this.request = scanTimeseriesDataRequest; + } + + @Override + public ScanTimeseriesDataResponse call() throws Exception { + return client.scanTimeseriesData(request); + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/DefaultNoRetry.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/DefaultNoRetry.java new file mode 100644 index 00000000..b286472d --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/DefaultNoRetry.java @@ -0,0 +1,32 @@ +package com.alibaba.datax.plugin.reader.otsreader.model; + + +import com.alicloud.openservices.tablestore.model.DefaultRetryStrategy; +import com.alicloud.openservices.tablestore.model.RetryStrategy; + +public class DefaultNoRetry extends DefaultRetryStrategy { + + public DefaultNoRetry() { + super(); + } + + @Override + public RetryStrategy clone() { + return super.clone(); + } + + @Override + public int getRetries() { + return super.getRetries(); + } + + @Override + public boolean shouldRetry(String action, Exception ex) { + return false; + } + + @Override + public long nextPause(String action, Exception ex) { + return super.nextPause(action, ex); + } +} \ No newline at end of file diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSColumn.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSColumn.java index 129ccd2f..809f4c38 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSColumn.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSColumn.java @@ -1,19 +1,18 @@ package com.alibaba.datax.plugin.reader.otsreader.model; -import com.alibaba.datax.common.element.BoolColumn; -import com.alibaba.datax.common.element.BytesColumn; -import com.alibaba.datax.common.element.Column; -import com.alibaba.datax.common.element.DoubleColumn; -import com.alibaba.datax.common.element.LongColumn; -import com.alibaba.datax.common.element.StringColumn; -import com.aliyun.openservices.ots.model.ColumnType; +import com.alibaba.datax.common.element.*; +import com.alicloud.openservices.tablestore.model.ColumnType; public class OTSColumn { private String name; private Column value; + private OTSColumnType columnType; + + // 时序数据column配置 private ColumnType valueType; - + private Boolean isTimeseriesTag; + public static enum OTSColumnType { NORMAL, // 普通列 CONST // 常量列 @@ -24,10 +23,9 @@ public class OTSColumn { this.columnType = OTSColumnType.NORMAL; } - private OTSColumn(Column value, ColumnType type) { + private OTSColumn(Column value) { this.value = value; this.columnType = OTSColumnType.CONST; - this.valueType = type; } public static OTSColumn fromNormalColumn(String name) { @@ -39,23 +37,23 @@ public class OTSColumn { } public static OTSColumn fromConstStringColumn(String value) { - return new OTSColumn(new StringColumn(value), ColumnType.STRING); + return new OTSColumn(new StringColumn(value)); } public static OTSColumn fromConstIntegerColumn(long value) { - return new OTSColumn(new LongColumn(value), ColumnType.INTEGER); + return new OTSColumn(new LongColumn(value)); } public static OTSColumn fromConstDoubleColumn(double value) { - return new OTSColumn(new DoubleColumn(value), ColumnType.DOUBLE); + return new OTSColumn(new DoubleColumn(value)); } public static OTSColumn fromConstBoolColumn(boolean value) { - return new OTSColumn(new BoolColumn(value), ColumnType.BOOLEAN); + return new OTSColumn(new BoolColumn(value)); } public static OTSColumn fromConstBytesColumn(byte[] value) { - return new OTSColumn(new BytesColumn(value), ColumnType.BINARY); + return new OTSColumn(new BytesColumn(value)); } public Column getValue() { @@ -65,12 +63,25 @@ public class OTSColumn { public OTSColumnType getColumnType() { return columnType; } - - public ColumnType getValueType() { - return valueType; - } + public String getName() { return name; } -} + + public ColumnType getValueType() { + return valueType; + } + + public void setValueType(ColumnType valueType) { + this.valueType = valueType; + } + + public Boolean getTimeseriesTag() { + return isTimeseriesTag; + } + + public void setTimeseriesTag(Boolean timeseriesTag) { + isTimeseriesTag = timeseriesTag; + } +} \ No newline at end of file diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSConf.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSConf.java index 8b109a39..cbfd8f6a 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSConf.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSConf.java @@ -1,90 +1,245 @@ package com.alibaba.datax.plugin.reader.otsreader.model; +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.reader.otsreader.utils.Constant; +import com.alibaba.datax.plugin.reader.otsreader.utils.Key; +import com.alibaba.datax.plugin.reader.otsreader.utils.ParamChecker; +import com.alicloud.openservices.tablestore.model.ColumnType; + import java.util.List; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; - public class OTSConf { - private String endpoint= null; + private String endpoint = null; private String accessId = null; - private String accesskey = null; + private String accessKey = null; private String instanceName = null; private String tableName = null; + private OTSRange range = null; + private List column = null; + private OTSMode mode = null; + + @Deprecated + private String metaMode = ""; + + private boolean newVersion = false; + /** + * 以下配置仅用于timeseries数据读取 + */ + private boolean isTimeseriesTable = false; + private String measurementName = null; + /** + * 以上配置仅用于timeseries数据读取 + */ + private OTSMultiVersionConf multi = null; - private List rangeBegin = null; - private List rangeEnd = null; - private List rangeSplit = null; - - private List columns = null; - - private int retry; - private int sleepInMilliSecond; - + private int retry = Constant.ConfigDefaultValue.RETRY; + private int retryPauseInMillisecond = Constant.ConfigDefaultValue.RETRY_PAUSE_IN_MILLISECOND; + private int ioThreadCount = Constant.ConfigDefaultValue.IO_THREAD_COUNT; + private int maxConnectionCount = Constant.ConfigDefaultValue.MAX_CONNECTION_COUNT; + private int socketTimeoutInMillisecond = Constant.ConfigDefaultValue.SOCKET_TIMEOUT_IN_MILLISECOND; + private int connectTimeoutInMillisecond = Constant.ConfigDefaultValue.CONNECT_TIMEOUT_IN_MILLISECOND; + + public int getIoThreadCount() { + return ioThreadCount; + } + + public void setIoThreadCount(int ioThreadCount) { + this.ioThreadCount = ioThreadCount; + } + + public int getMaxConnectCount() { + return maxConnectionCount; + } + + public void setMaxConnectCount(int maxConnectCount) { + this.maxConnectionCount = maxConnectCount; + } + + public int getSocketTimeoutInMillisecond() { + return socketTimeoutInMillisecond; + } + + public void setSocketTimeoutInMillisecond(int socketTimeoutInMillisecond) { + this.socketTimeoutInMillisecond = socketTimeoutInMillisecond; + } + + public int getConnectTimeoutInMillisecond() { + return connectTimeoutInMillisecond; + } + + public void setConnectTimeoutInMillisecond(int connectTimeoutInMillisecond) { + this.connectTimeoutInMillisecond = connectTimeoutInMillisecond; + } + + public int getRetry() { + return retry; + } + + public void setRetry(int retry) { + this.retry = retry; + } + + public int getRetryPauseInMillisecond() { + return retryPauseInMillisecond; + } + + public void setRetryPauseInMillisecond(int sleepInMillisecond) { + this.retryPauseInMillisecond = sleepInMillisecond; + } + public String getEndpoint() { return endpoint; } + public void setEndpoint(String endpoint) { this.endpoint = endpoint; } + public String getAccessId() { return accessId; } + public void setAccessId(String accessId) { this.accessId = accessId; } - public String getAccesskey() { - return accesskey; + + public String getAccessKey() { + return accessKey; } - public void setAccesskey(String accesskey) { - this.accesskey = accesskey; + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; } + public String getInstanceName() { return instanceName; } + public void setInstanceName(String instanceName) { this.instanceName = instanceName; } + public String getTableName() { return tableName; } + public void setTableName(String tableName) { this.tableName = tableName; } - public List getColumns() { - return columns; + public OTSRange getRange() { + return range; } - public void setColumns(List columns) { - this.columns = columns; + + public void setRange(OTSRange range) { + this.range = range; } - public int getRetry() { - return retry; + + public OTSMode getMode() { + return mode; } - public void setRetry(int retry) { - this.retry = retry; + + public void setMode(OTSMode mode) { + this.mode = mode; } - public int getSleepInMilliSecond() { - return sleepInMilliSecond; + + public OTSMultiVersionConf getMulti() { + return multi; } - public void setSleepInMilliSecond(int sleepInMilliSecond) { - this.sleepInMilliSecond = sleepInMilliSecond; + + public void setMulti(OTSMultiVersionConf multi) { + this.multi = multi; } - public List getRangeBegin() { - return rangeBegin; + + public List getColumn() { + return column; } - public void setRangeBegin(List rangeBegin) { - this.rangeBegin = rangeBegin; + + public void setColumn(List column) { + this.column = column; } - public List getRangeEnd() { - return rangeEnd; + + public boolean isNewVersion() { + return newVersion; } - public void setRangeEnd(List rangeEnd) { - this.rangeEnd = rangeEnd; + + public void setNewVersion(boolean newVersion) { + this.newVersion = newVersion; } - public List getRangeSplit() { - return rangeSplit; + + @Deprecated + public String getMetaMode() { + return metaMode; } - public void setRangeSplit(List rangeSplit) { - this.rangeSplit = rangeSplit; + + @Deprecated + public void setMetaMode(String metaMode) { + this.metaMode = metaMode; + } + + public boolean isTimeseriesTable() { + return isTimeseriesTable; + } + + public void setTimeseriesTable(boolean timeseriesTable) { + isTimeseriesTable = timeseriesTable; + } + + public String getMeasurementName() { + return measurementName; + } + + public void setMeasurementName(String measurementName) { + this.measurementName = measurementName; + } + + public static OTSConf load(Configuration param) throws OTSCriticalException { + OTSConf c = new OTSConf(); + + // account + c.setEndpoint(ParamChecker.checkStringAndGet(param, Key.OTS_ENDPOINT, true)); + c.setAccessId(ParamChecker.checkStringAndGet(param, Key.OTS_ACCESSID, true)); + c.setAccessKey(ParamChecker.checkStringAndGet(param, Key.OTS_ACCESSKEY, true)); + c.setInstanceName(ParamChecker.checkStringAndGet(param, Key.OTS_INSTANCE_NAME, true)); + c.setTableName(ParamChecker.checkStringAndGet(param, Key.TABLE_NAME, true)); + + c.setRetry(param.getInt(Constant.ConfigKey.RETRY, Constant.ConfigDefaultValue.RETRY)); + c.setRetryPauseInMillisecond(param.getInt(Constant.ConfigKey.RETRY_PAUSE_IN_MILLISECOND, Constant.ConfigDefaultValue.RETRY_PAUSE_IN_MILLISECOND)); + c.setIoThreadCount(param.getInt(Constant.ConfigKey.IO_THREAD_COUNT, Constant.ConfigDefaultValue.IO_THREAD_COUNT)); + c.setMaxConnectCount(param.getInt(Constant.ConfigKey.MAX_CONNECTION_COUNT, Constant.ConfigDefaultValue.MAX_CONNECTION_COUNT)); + c.setSocketTimeoutInMillisecond(param.getInt(Constant.ConfigKey.SOCKET_TIMEOUTIN_MILLISECOND, Constant.ConfigDefaultValue.SOCKET_TIMEOUT_IN_MILLISECOND)); + c.setConnectTimeoutInMillisecond(param.getInt(Constant.ConfigKey.CONNECT_TIMEOUT_IN_MILLISECOND, Constant.ConfigDefaultValue.CONNECT_TIMEOUT_IN_MILLISECOND)); + + // range + c.setRange(ParamChecker.checkRangeAndGet(param)); + + // mode 可选参数 + c.setMode(ParamChecker.checkModeAndGet(param)); + //isNewVersion 可选参数 + c.setNewVersion(param.getBool(Key.NEW_VERSION, false)); + // metaMode 旧版本配置 + c.setMetaMode(param.getString(Key.META_MODE, "")); + + + + // 读时序表配置项 + c.setTimeseriesTable(param.getBool(Key.IS_TIMESERIES_TABLE, false)); + // column + if(!c.isTimeseriesTable()){ + //非时序表 + c.setColumn(ParamChecker.checkOTSColumnAndGet(param, c.getMode())); + } + else{ + // 时序表 + c.setMeasurementName(param.getString(Key.MEASUREMENT_NAME, "")); + c.setColumn(ParamChecker.checkTimeseriesColumnAndGet(param)); + ParamChecker.checkTimeseriesMode(c.getMode(), c.isNewVersion()); + } + + if (c.getMode() == OTSMode.MULTI_VERSION) { + c.setMulti(OTSMultiVersionConf.load(param)); + } + return c; } } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSCriticalException.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSCriticalException.java new file mode 100644 index 00000000..f02346bc --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSCriticalException.java @@ -0,0 +1,24 @@ +package com.alibaba.datax.plugin.reader.otsreader.model; + +/** + * 插件错误异常,该异常主要用于描述插件的异常退出 + * @author redchen + */ +public class OTSCriticalException extends Exception{ + + private static final long serialVersionUID = 5820460098894295722L; + + public OTSCriticalException() {} + + public OTSCriticalException(String message) { + super(message); + } + + public OTSCriticalException(Throwable a) { + super(a); + } + + public OTSCriticalException(String message, Throwable a) { + super(message, a); + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSErrorCode.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSErrorCode.java new file mode 100644 index 00000000..0c537fce --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSErrorCode.java @@ -0,0 +1,115 @@ +/** + * Copyright (C) Alibaba Cloud Computing + * All rights reserved. + * + * 版权所有 (C)阿里云计算有限公司 + */ + +package com.alibaba.datax.plugin.reader.otsreader.model; + +/** + * 表示来自开放结构化数据服务(Open Table Service,OTS)的错误代码。 + * + */ +public class OTSErrorCode { + /** + * 用户身份验证失败。 + */ + public static final String AUTHORIZATION_FAILURE = "OTSAuthFailed"; + + /** + * 服务器内部错误。 + */ + public static final String INTERNAL_SERVER_ERROR = "OTSInternalServerError"; + + /** + * 参数错误。 + */ + public static final String INVALID_PARAMETER = "OTSParameterInvalid"; + + /** + * 整个请求过大。 + */ + public static final String REQUEST_TOO_LARGE = "OTSRequestBodyTooLarge"; + + /** + * 客户端请求超时。 + */ + public static final String REQUEST_TIMEOUT = "OTSRequestTimeout"; + + /** + * 用户的配额已经用满。 + */ + public static final String QUOTA_EXHAUSTED = "OTSQuotaExhausted"; + + /** + * 内部服务器发生failover,导致表的部分分区不可服务。 + */ + public static final String PARTITION_UNAVAILABLE = "OTSPartitionUnavailable"; + + /** + * 表刚被创建还无法立马提供服务。 + */ + public static final String TABLE_NOT_READY = "OTSTableNotReady"; + + /** + * 请求的表不存在。 + */ + public static final String OBJECT_NOT_EXIST = "OTSObjectNotExist"; + + /** + * 请求创建的表已经存在。 + */ + public static final String OBJECT_ALREADY_EXIST = "OTSObjectAlreadyExist"; + + /** + * 多个并发的请求写同一行数据,导致冲突。 + */ + public static final String ROW_OPEARTION_CONFLICT = "OTSRowOperationConflict"; + + /** + * 主键不匹配。 + */ + public static final String INVALID_PK = "OTSInvalidPK"; + + /** + * 读写能力调整过于频繁。 + */ + public static final String TOO_FREQUENT_RESERVED_THROUGHPUT_ADJUSTMENT = "OTSTooFrequentReservedThroughputAdjustment"; + + /** + * 该行总列数超出限制。 + */ + public static final String OUT_OF_COLUMN_COUNT_LIMIT = "OTSOutOfColumnCountLimit"; + + /** + * 该行所有列数据大小总和超出限制。 + */ + public static final String OUT_OF_ROW_SIZE_LIMIT = "OTSOutOfRowSizeLimit"; + + /** + * 剩余预留读写能力不足。 + */ + public static final String NOT_ENOUGH_CAPACITY_UNIT = "OTSNotEnoughCapacityUnit"; + + /** + * 预查条件检查失败。 + */ + public static final String CONDITION_CHECK_FAIL = "OTSConditionCheckFail"; + + /** + * 在OTS内部操作超时。 + */ + public static final String STORAGE_TIMEOUT = "OTSTimeout"; + + /** + * 在OTS内部有服务器不可访问。 + */ + public static final String SERVER_UNAVAILABLE = "OTSServerUnavailable"; + + /** + * OTS内部服务器繁忙。 + */ + public static final String SERVER_BUSY = "OTSServerBusy"; + +} \ No newline at end of file diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSMode.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSMode.java new file mode 100644 index 00000000..88c6ee67 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSMode.java @@ -0,0 +1,6 @@ +package com.alibaba.datax.plugin.reader.otsreader.model; + +public enum OTSMode { + NORMAL, + MULTI_VERSION +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSMultiVersionConf.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSMultiVersionConf.java new file mode 100644 index 00000000..72a8e1b7 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSMultiVersionConf.java @@ -0,0 +1,35 @@ +package com.alibaba.datax.plugin.reader.otsreader.model; + +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.reader.otsreader.utils.Constant; +import com.alibaba.datax.plugin.reader.otsreader.utils.ParamChecker; +import com.alicloud.openservices.tablestore.model.TimeRange; + +public class OTSMultiVersionConf { + + private TimeRange timeRange = null; + private int maxVersion = -1; + + public TimeRange getTimeRange() { + return timeRange; + } + + public void setTimeRange(TimeRange timeRange) { + this.timeRange = timeRange; + } + + public int getMaxVersion() { + return maxVersion; + } + + public void setMaxVersion(int maxVersion) { + this.maxVersion = maxVersion; + } + + public static OTSMultiVersionConf load(Configuration param) throws OTSCriticalException { + OTSMultiVersionConf conf = new OTSMultiVersionConf(); + conf.setTimeRange(ParamChecker.checkTimeRangeAndGet(param)); + conf.setMaxVersion(param.getInt(Constant.ConfigKey.MAX_VERSION, Constant.ConfigDefaultValue.MAX_VERSION)); + return conf; + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSPrimaryKeyColumn.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSPrimaryKeyColumn.java index eaec50ce..44a37c0c 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSPrimaryKeyColumn.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSPrimaryKeyColumn.java @@ -15,8 +15,41 @@ public class OTSPrimaryKeyColumn { public PrimaryKeyType getType() { return type; } + + public com.alicloud.openservices.tablestore.model.PrimaryKeyType getType(Boolean newVersion) { + com.alicloud.openservices.tablestore.model.PrimaryKeyType res = null; + switch (this.type){ + case BINARY: + res = com.alicloud.openservices.tablestore.model.PrimaryKeyType.BINARY; + break; + case INTEGER: + res = com.alicloud.openservices.tablestore.model.PrimaryKeyType.INTEGER; + break; + case STRING: + default: + res = com.alicloud.openservices.tablestore.model.PrimaryKeyType.STRING; + break; + } + return res; + } + public void setType(PrimaryKeyType type) { this.type = type; } + + public void setType(com.alicloud.openservices.tablestore.model.PrimaryKeyType type) { + switch (type){ + case BINARY: + this.type = PrimaryKeyType.BINARY; + break; + case INTEGER: + this.type = PrimaryKeyType.INTEGER; + break; + case STRING: + default: + this.type = PrimaryKeyType.STRING; + break; + } + } } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSRange.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSRange.java index 8ebfcf7e..eb3095e6 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSRange.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/model/OTSRange.java @@ -1,29 +1,31 @@ package com.alibaba.datax.plugin.reader.otsreader.model; -import com.aliyun.openservices.ots.model.RowPrimaryKey; +import com.alicloud.openservices.tablestore.model.PrimaryKeyColumn; + +import java.util.List; + public class OTSRange { + private List begin = null; + private List end = null; + private List split = null; - private RowPrimaryKey begin = null; - private RowPrimaryKey end = null; - - public OTSRange() {} - - public OTSRange(RowPrimaryKey begin, RowPrimaryKey end) { - this.begin = begin; - this.end = end; - } - - public RowPrimaryKey getBegin() { + public List getBegin() { return begin; } - public void setBegin(RowPrimaryKey begin) { + public void setBegin(List begin) { this.begin = begin; } - public RowPrimaryKey getEnd() { + public List getEnd() { return end; } - public void setEnd(RowPrimaryKey end) { + public void setEnd(List end) { this.end = end; } + public List getSplit() { + return split; + } + public void setSplit(List split) { + this.split = split; + } } 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 fb8c7feb..90065d5d 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 @@ -1,26 +1,85 @@ package com.alibaba.datax.plugin.reader.otsreader.utils; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSCriticalException; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSPrimaryKeyColumn; +import com.alicloud.openservices.tablestore.model.*; +import com.alicloud.openservices.tablestore.model.timeseries.ScanTimeseriesDataResponse; + +import java.lang.reflect.Field; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; -import com.alibaba.datax.common.element.BoolColumn; -import com.alibaba.datax.common.element.BytesColumn; -import com.alibaba.datax.common.element.DoubleColumn; -import com.alibaba.datax.common.element.LongColumn; -import com.alibaba.datax.common.element.Record; -import com.alibaba.datax.common.element.StringColumn; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSPrimaryKeyColumn; -import com.aliyun.openservices.ots.ClientException; -import com.aliyun.openservices.ots.OTSException; -import com.aliyun.openservices.ots.model.ColumnValue; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; -import com.aliyun.openservices.ots.model.Row; -import com.aliyun.openservices.ots.model.RowPrimaryKey; -import com.aliyun.openservices.ots.model.TableMeta; - public class Common { + public static List toColumnToGet(List columns, TableMeta meta) { + Map pk = meta.getPrimaryKeyMap(); + List names = new ArrayList(); + for (OTSColumn c : columns) { + if (c.getColumnType() == OTSColumn.OTSColumnType.NORMAL && !pk.containsKey(c.getName())) { + names.add(c.getName()); + } + } + return names; + } + + public static List getPrimaryKeyNameList(TableMeta meta) { + List names = new ArrayList(); + names.addAll(meta.getPrimaryKeyMap().keySet()); + return names; + } + + public static OTSPrimaryKeyColumn getPartitionKey(TableMeta meta) { + List keys = new ArrayList(); + keys.addAll(meta.getPrimaryKeyMap().keySet()); + + String key = keys.get(0); + + OTSPrimaryKeyColumn col = new OTSPrimaryKeyColumn(); + col.setName(key); + col.setType(meta.getPrimaryKeyMap().get(key)); + return col; + } + + public static Direction getDirection(List begin, List end) throws OTSCriticalException { + int cmp = CompareHelper.comparePrimaryKeyColumnList(begin, end); + if (cmp < 0) { + return Direction.FORWARD; + } else if (cmp > 0) { + return Direction.BACKWARD; + } else { + throw new OTSCriticalException("Bug branch, the begin of range equals end of range."); + } + } + + public static int compareRangeBeginAndEnd(TableMeta meta, List begin, List end) { + if (begin.size() != end.size()) { + throw new IllegalArgumentException("Input size of begin not equal size of end, begin size : " + begin.size() + + ", end size : " + end.size() + "."); + } + + Map beginMap = new HashMap<>(); + Map endMap = new HashMap<>(); + + for(PrimaryKeyColumn primaryKeyColumn : begin){ + beginMap.put(primaryKeyColumn.getName(), primaryKeyColumn.getValue()); + } + for(PrimaryKeyColumn primaryKeyColumn : end){ + endMap.put(primaryKeyColumn.getName(), primaryKeyColumn.getValue()); + } + + for (String key : meta.getPrimaryKeyMap().keySet()) { + PrimaryKeyValue v1 = beginMap.get(key); + PrimaryKeyValue v2 = endMap.get(key); + int cmp = primaryKeyValueCmp(v1, v2); + if (cmp != 0) { + return cmp; + } + } + return 0; + } + public static int primaryKeyValueCmp(PrimaryKeyValue v1, PrimaryKeyValue v2) { if (v1.getType() != null && v2.getType() != null) { @@ -29,14 +88,14 @@ public class Common { "Not same column type, column1:" + v1.getType() + ", column2:" + v2.getType()); } switch (v1.getType()) { - case INTEGER: - Long l1 = Long.valueOf(v1.asLong()); - Long l2 = Long.valueOf(v2.asLong()); - return l1.compareTo(l2); - case STRING: - return v1.asString().compareTo(v2.asString()); - default: - throw new IllegalArgumentException("Unsuporrt compare the type: " + v1.getType() + "."); + case INTEGER: + Long l1 = Long.valueOf(v1.asLong()); + Long l2 = Long.valueOf(v2.asLong()); + return l1.compareTo(l2); + case STRING: + return v1.asString().compareTo(v2.asString()); + default: + throw new IllegalArgumentException("Unsuporrt compare the type: " + v1.getType() + "."); } } else { if (v1 == v2) { @@ -46,116 +105,31 @@ public class Common { return -1; } else if (v1 == PrimaryKeyValue.INF_MAX) { return 1; - } + } if (v2 == PrimaryKeyValue.INF_MAX) { return -1; } else if (v2 == PrimaryKeyValue.INF_MIN) { return 1; - } - } - } - return 0; - } - - public static OTSPrimaryKeyColumn getPartitionKey(TableMeta meta) { - List keys = new ArrayList(); - keys.addAll(meta.getPrimaryKey().keySet()); - - String key = keys.get(0); - - OTSPrimaryKeyColumn col = new OTSPrimaryKeyColumn(); - col.setName(key); - col.setType(meta.getPrimaryKey().get(key)); - return col; - } - - public static List getPrimaryKeyNameList(TableMeta meta) { - List names = new ArrayList(); - names.addAll(meta.getPrimaryKey().keySet()); - return names; - } - - public static int compareRangeBeginAndEnd(TableMeta meta, RowPrimaryKey begin, RowPrimaryKey end) { - if (begin.getPrimaryKey().size() != end.getPrimaryKey().size()) { - throw new IllegalArgumentException("Input size of begin not equal size of end, begin size : " + begin.getPrimaryKey().size() + - ", end size : " + end.getPrimaryKey().size() + "."); - } - for (String key : meta.getPrimaryKey().keySet()) { - PrimaryKeyValue v1 = begin.getPrimaryKey().get(key); - PrimaryKeyValue v2 = end.getPrimaryKey().get(key); - int cmp = primaryKeyValueCmp(v1, v2); - if (cmp != 0) { - return cmp; - } - } - return 0; - } - - public static List getNormalColumnNameList(List columns) { - List normalColumns = new ArrayList(); - for (OTSColumn col : columns) { - if (col.getColumnType() == OTSColumn.OTSColumnType.NORMAL) { - normalColumns.add(col.getName()); - } - } - return normalColumns; - } - - public static Record parseRowToLine(Row row, List columns, Record line) { - Map values = row.getColumns(); - for (OTSColumn col : columns) { - if (col.getColumnType() == OTSColumn.OTSColumnType.CONST) { - line.addColumn(col.getValue()); - } else { - ColumnValue v = values.get(col.getName()); - if (v == null) { - line.addColumn(new StringColumn(null)); - } else { - switch(v.getType()) { - case STRING: line.addColumn(new StringColumn(v.asString())); break; - case INTEGER: line.addColumn(new LongColumn(v.asLong())); break; - case DOUBLE: line.addColumn(new DoubleColumn(v.asDouble())); break; - case BOOLEAN: line.addColumn(new BoolColumn(v.asBoolean())); break; - case BINARY: line.addColumn(new BytesColumn(v.asBinary())); break; - default: - throw new IllegalArgumentException("Unsupported transform the type: " + col.getValue().getType() + "."); - } } } } - return line; + return 0; } - - public static String getDetailMessage(Exception exception) { - if (exception instanceof OTSException) { - OTSException e = (OTSException) exception; - return "OTSException[ErrorCode:" + e.getErrorCode() + ", ErrorMessage:" + e.getMessage() + ", RequestId:" + e.getRequestId() + "]"; - } else if (exception instanceof ClientException) { - ClientException e = (ClientException) exception; - return "ClientException[ErrorCode:" + e.getErrorCode() + ", ErrorMessage:" + e.getMessage() + "]"; - } else if (exception instanceof IllegalArgumentException) { - IllegalArgumentException e = (IllegalArgumentException) exception; - return "IllegalArgumentException[ErrorMessage:" + e.getMessage() + "]"; - } else { - return "Exception[ErrorMessage:" + exception.getMessage() + "]"; - } - } - - public static long getDelaySendMillinSeconds(int hadRetryTimes, int initSleepInMilliSecond) { - if (hadRetryTimes <= 0) { - return 0; - } - - int sleepTime = initSleepInMilliSecond; - for (int i = 1; i < hadRetryTimes; i++) { - sleepTime += sleepTime; - if (sleepTime > 30000) { - sleepTime = 30000; + public static void checkTableStoreSDKVersion() throws OTSCriticalException { + Field[] fields = ScanTimeseriesDataResponse.class.getFields(); + String sdkVersion = null; + for (Field f : fields){ + if (f.getName().equals("_VERSION_")){ + sdkVersion = ScanTimeseriesDataResponse._VERSION_; break; - } + } + } + if (sdkVersion == null){ + throw new OTSCriticalException("Check ots java SDK failed. Please check the version of tableStore maven dependency."); + }else if (Integer.parseInt(sdkVersion) < 20230111){ + throw new OTSCriticalException("Check tableStore java SDK failed. The expected version number is greater than 20230111, actually version : " + sdkVersion + "."); } - return sleepTime; } } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/CommonOld.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/CommonOld.java new file mode 100644 index 00000000..d5c565f4 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/CommonOld.java @@ -0,0 +1,112 @@ +package com.alibaba.datax.plugin.reader.otsreader.utils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import com.alibaba.datax.common.element.BoolColumn; +import com.alibaba.datax.common.element.BytesColumn; +import com.alibaba.datax.common.element.DoubleColumn; +import com.alibaba.datax.common.element.LongColumn; +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.element.StringColumn; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSPrimaryKeyColumn; +import com.aliyun.openservices.ots.ClientException; +import com.aliyun.openservices.ots.OTSException; +import com.aliyun.openservices.ots.model.ColumnValue; +import com.aliyun.openservices.ots.model.PrimaryKeyValue; +import com.aliyun.openservices.ots.model.Row; +import com.aliyun.openservices.ots.model.RowPrimaryKey; +import com.aliyun.openservices.ots.model.TableMeta; + +public class CommonOld { + public static int primaryKeyValueCmp(PrimaryKeyValue v1, PrimaryKeyValue v2) { + if (v1.getType() != null && v2.getType() != null) { + if (v1.getType() != v2.getType()) { + throw new IllegalArgumentException( + "Not same column type, column1:" + v1.getType() + ", column2:" + v2.getType()); + } + switch (v1.getType()) { + case INTEGER: + Long l1 = Long.valueOf(v1.asLong()); + Long l2 = Long.valueOf(v2.asLong()); + return l1.compareTo(l2); + case STRING: + return v1.asString().compareTo(v2.asString()); + default: + throw new IllegalArgumentException("Unsuporrt compare the type: " + v1.getType() + "."); + } + } else { + if (v1 == v2) { + return 0; + } else { + if (v1 == PrimaryKeyValue.INF_MIN) { + return -1; + } else if (v1 == PrimaryKeyValue.INF_MAX) { + return 1; + } + + if (v2 == PrimaryKeyValue.INF_MAX) { + return -1; + } else if (v2 == PrimaryKeyValue.INF_MIN) { + return 1; + } + } + } + return 0; + } + + + public static List getNormalColumnNameList(List columns) { + List normalColumns = new ArrayList(); + for (OTSColumn col : columns) { + if (col.getColumnType() == OTSColumn.OTSColumnType.NORMAL) { + normalColumns.add(col.getName()); + } + } + return normalColumns; + } + + public static Record parseRowToLine(Row row, List columns, Record line) { + Map values = row.getColumns(); + for (OTSColumn col : columns) { + if (col.getColumnType() == OTSColumn.OTSColumnType.CONST) { + line.addColumn(col.getValue()); + } else { + ColumnValue v = values.get(col.getName()); + if (v == null) { + line.addColumn(new StringColumn(null)); + } else { + switch(v.getType()) { + case STRING: line.addColumn(new StringColumn(v.asString())); break; + case INTEGER: line.addColumn(new LongColumn(v.asLong())); break; + case DOUBLE: line.addColumn(new DoubleColumn(v.asDouble())); break; + 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() + "."); + } + } + } + } + return line; + } + + public static long getDelaySendMillinSeconds(int hadRetryTimes, int initSleepInMilliSecond) { + + if (hadRetryTimes <= 0) { + return 0; + } + + int sleepTime = initSleepInMilliSecond; + for (int i = 1; i < hadRetryTimes; i++) { + sleepTime += sleepTime; + if (sleepTime > 30000) { + sleepTime = 30000; + break; + } + } + return sleepTime; + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/CompareHelper.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/CompareHelper.java new file mode 100644 index 00000000..19e06421 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/CompareHelper.java @@ -0,0 +1,37 @@ +package com.alibaba.datax.plugin.reader.otsreader.utils; + +import com.alicloud.openservices.tablestore.model.PrimaryKeyColumn; + +import java.util.List; + + +public class CompareHelper { + /** + * 比较PrimaryKeyColumn List的大小 + * 返回 + * -1 表示before小于after + * 0 表示before等于after + * 1 表示before大于after + * + * @param before + * @param after + * @return + */ + public static int comparePrimaryKeyColumnList(List before, List after) { + int size = before.size() < after.size() ? before.size() : after.size(); + + for (int i = 0; i < size; i++) { + int cmp = before.get(i).compareTo(after.get(i)); + if (cmp != 0) { + return cmp; + } + } + + if (before.size() < after.size() ) { + return -1; + } else if (before.size() > after.size() ) { + return 1; + } + return 0; + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Constant.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Constant.java new file mode 100644 index 00000000..90273bfb --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Constant.java @@ -0,0 +1,92 @@ +package com.alibaba.datax.plugin.reader.otsreader.utils; + +public class Constant { + /** + * Json中的Key名字定义 + */ +public class ConfigKey { + public static final String CONF = "conf"; + public static final String RANGE = "range"; + public static final String META = "meta"; + public static final String SPLIT_INFO = "splitInfo"; + + public static final String TIME_RANGE = "timeRange"; + public static final String MAX_VERSION = "maxVersion"; + + public static final String RETRY = "maxRetryTime"; + public static final String RETRY_PAUSE_IN_MILLISECOND = "retryPauseInMillisecond"; + public static final String IO_THREAD_COUNT = "ioThreadCount"; + public static final String MAX_CONNECTION_COUNT = "maxConnectionCount"; + public static final String SOCKET_TIMEOUTIN_MILLISECOND = "socketTimeoutInMillisecond"; + public static final String CONNECT_TIMEOUT_IN_MILLISECOND = "connectTimeoutInMillisecond"; + + public class Range { + public static final String BEGIN = "begin"; + public static final String END = "end"; + public static final String SPLIT = "split"; + }; + + public class PrimaryKeyColumn { + public static final String TYPE = "type"; + public static final String VALUE = "value"; + }; + + public class TimeseriesPKColumn { + public static final String MEASUREMENT_NAME = "_m_name"; + public static final String DATA_SOURCE = "_data_source"; + public static final String TAGS = "_tags"; + public static final String TIME = "_time"; + } + + public class Column { + public static final String NAME = "name"; + public static final String TYPE = "type"; + public static final String VALUE = "value"; + public static final String IS_TAG = "is_timeseries_tag"; + }; + + public class TimeRange { + public static final String BEGIN = "begin"; + public static final String END = "end"; + } + }; + + /** + * 定义的配置文件中value type中可取的值 + */ + public class ValueType { + public static final String INF_MIN = "INF_MIN"; + public static final String INF_MAX = "INF_MAX"; + public static final String STRING = "string"; + public static final String INTEGER = "int"; + public static final String BINARY = "binary"; + public static final String DOUBLE = "double"; + public static final String BOOLEAN = "bool"; + }; + + /** + * 全局默认常量定义 + */ + public class ConfigDefaultValue { + public static final int RETRY = 18; + public static final int RETRY_PAUSE_IN_MILLISECOND = 100; + public static final int IO_THREAD_COUNT = 1; + public static final int MAX_CONNECTION_COUNT = 1; + public static final int SOCKET_TIMEOUT_IN_MILLISECOND = 10000; + public static final int CONNECT_TIMEOUT_IN_MILLISECOND = 10000; + + public static final int MAX_VERSION = Integer.MAX_VALUE; + + public static final String DEFAULT_NAME = "DEFAULT_NAME"; + + public class Mode { + public static final String NORMAL = "normal"; + public static final String MULTI_VERSION = "multiVersion"; + } + + public class TimeRange { + public static final long MIN = 0; + public static final long MAX = Long.MAX_VALUE; + } + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/GsonParser.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/GsonParser.java index a82f3350..205f536d 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/GsonParser.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/GsonParser.java @@ -1,23 +1,26 @@ package com.alibaba.datax.plugin.reader.otsreader.utils; -import com.alibaba.datax.plugin.reader.otsreader.adaptor.OTSColumnAdaptor; +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.plugin.reader.otsreader.adaptor.ColumnAdaptor; import com.alibaba.datax.plugin.reader.otsreader.adaptor.PrimaryKeyValueAdaptor; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; +import com.alicloud.openservices.tablestore.model.PrimaryKeyValue; +import com.alicloud.openservices.tablestore.model.TableMeta; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesScanSplitInfo; import com.aliyun.openservices.ots.model.Direction; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; import com.aliyun.openservices.ots.model.RowPrimaryKey; -import com.aliyun.openservices.ots.model.TableMeta; import com.google.gson.Gson; import com.google.gson.GsonBuilder; +import java.util.Map; + public class GsonParser { private static Gson gsonBuilder() { return new GsonBuilder() - .registerTypeAdapter(OTSColumn.class, new OTSColumnAdaptor()) .registerTypeAdapter(PrimaryKeyValue.class, new PrimaryKeyValueAdaptor()) + .registerTypeAdapter(Column.class, new ColumnAdaptor()) .create(); } @@ -40,24 +43,39 @@ public class GsonParser { Gson g = gsonBuilder(); return g.fromJson(jsonStr, OTSConf.class); } - - public static String directionToJson (Direction direction) { + + public static String metaToJson (TableMeta meta) { Gson g = gsonBuilder(); - return g.toJson(direction); + return g.toJson(meta); + } + + public static TableMeta jsonToMeta (String jsonStr) { + Gson g = gsonBuilder(); + return g.fromJson(jsonStr, TableMeta.class); + } + + public static String timeseriesScanSplitInfoToString(TimeseriesScanSplitInfo timeseriesScanSplitInfo){ + Gson g = gsonBuilder(); + return g.toJson(timeseriesScanSplitInfo); + } + + public static TimeseriesScanSplitInfo stringToTimeseriesScanSplitInfo(String jsonStr){ + Gson g = gsonBuilder(); + return g.fromJson(jsonStr, TimeseriesScanSplitInfo.class); } public static Direction jsonToDirection (String jsonStr) { Gson g = gsonBuilder(); return g.fromJson(jsonStr, Direction.class); } - - public static String metaToJson (TableMeta meta) { - Gson g = gsonBuilder(); - return g.toJson(meta); - } - + public static String rowPrimaryKeyToJson (RowPrimaryKey row) { Gson g = gsonBuilder(); return g.toJson(row); } + + public static String mapToJson (Map map) { + Gson g = gsonBuilder(); + return g.toJson(map); + } } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/Key.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Key.java similarity index 81% rename from otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/Key.java rename to otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Key.java index da6d4a5f..6628e4d3 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/Key.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Key.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.alibaba.datax.plugin.reader.otsreader; +package com.alibaba.datax.plugin.reader.otsreader.utils; public final class Key { /* ots account configuration */ @@ -46,5 +46,13 @@ public final class Key { public final static String RANGE_END = "end"; public final static String RANGE_SPLIT = "split"; + + public final static String META_MODE = "metaMode"; + + public final static String MODE = "mode"; + public final static String NEW_VERSION = "newVersion"; + + public final static String IS_TIMESERIES_TABLE = "isTimeseriesTable"; + public final static String MEASUREMENT_NAME = "measurementName"; } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/OtsHelper.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/OtsHelper.java new file mode 100644 index 00000000..060507b6 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/OtsHelper.java @@ -0,0 +1,82 @@ +package com.alibaba.datax.plugin.reader.otsreader.utils; + +import com.alibaba.datax.plugin.reader.otsreader.callable.GetRangeCallable; +import com.alibaba.datax.plugin.reader.otsreader.callable.GetTableMetaCallable; +import com.alibaba.datax.plugin.reader.otsreader.callable.GetTimeseriesSplitCallable; +import com.alibaba.datax.plugin.reader.otsreader.callable.ScanTimeseriesDataCallable; +import com.alibaba.datax.plugin.reader.otsreader.model.DefaultNoRetry; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSConf; +import com.alicloud.openservices.tablestore.ClientConfiguration; +import com.alicloud.openservices.tablestore.SyncClient; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.core.utils.Pair; +import com.alicloud.openservices.tablestore.model.ColumnType; +import com.alicloud.openservices.tablestore.model.GetRangeResponse; +import com.alicloud.openservices.tablestore.model.RangeRowQueryCriteria; +import com.alicloud.openservices.tablestore.model.TableMeta; +import com.alicloud.openservices.tablestore.model.timeseries.ScanTimeseriesDataRequest; +import com.alicloud.openservices.tablestore.model.timeseries.ScanTimeseriesDataResponse; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesScanSplitInfo; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class OtsHelper { + + public static SyncClientInterface getOTSInstance(OTSConf conf) { + ClientConfiguration clientConfigure = new ClientConfiguration(); + clientConfigure.setIoThreadCount(conf.getIoThreadCount()); + clientConfigure.setMaxConnections(conf.getMaxConnectCount()); + clientConfigure.setSocketTimeoutInMillisecond(conf.getSocketTimeoutInMillisecond()); + clientConfigure.setConnectionTimeoutInMillisecond(conf.getConnectTimeoutInMillisecond()); + clientConfigure.setRetryStrategy(new DefaultNoRetry()); + + SyncClient ots = new SyncClient( + conf.getEndpoint(), + conf.getAccessId(), + conf.getAccessKey(), + conf.getInstanceName(), + clientConfigure); + + + Map extraHeaders = new HashMap(); + extraHeaders.put("x-ots-sdk-type", "public"); + extraHeaders.put("x-ots-request-source", "datax-otsreader"); + ots.setExtraHeaders(extraHeaders); + + return ots; + } + + public static TableMeta getTableMeta(SyncClientInterface ots, String tableName, int retry, int sleepInMillisecond) throws Exception { + return RetryHelper.executeWithRetry( + new GetTableMetaCallable(ots, tableName), + retry, + sleepInMillisecond + ); + } + + public static GetRangeResponse getRange(SyncClientInterface ots, RangeRowQueryCriteria rangeRowQueryCriteria, int retry, int sleepInMillisecond) throws Exception { + return RetryHelper.executeWithRetry( + new GetRangeCallable(ots, rangeRowQueryCriteria), + retry, + sleepInMillisecond + ); + } + + public static List splitTimeseriesScan(SyncClientInterface ots, String tableName, String measurementName, int splitCountHint, int retry, int sleepInMillisecond) throws Exception { + return RetryHelper.executeWithRetry( + new GetTimeseriesSplitCallable(ots, tableName, measurementName, splitCountHint), + retry, + sleepInMillisecond + ); + } + + public static ScanTimeseriesDataResponse scanTimeseriesData(SyncClientInterface ots, ScanTimeseriesDataRequest scanTimeseriesDataRequest, int retry, int sleepInMillisecond) throws Exception { + return RetryHelper.executeWithRetry( + new ScanTimeseriesDataCallable(ots, scanTimeseriesDataRequest), + retry, + sleepInMillisecond + ); + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderError.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/OtsReaderError.java similarity index 76% rename from otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderError.java rename to otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/OtsReaderError.java index 05a13c1a..b578dcde 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/OtsReaderError.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/OtsReaderError.java @@ -1,4 +1,4 @@ -package com.alibaba.datax.plugin.reader.otsreader; +package com.alibaba.datax.plugin.reader.otsreader.utils; import com.alibaba.datax.common.spi.ErrorCode; @@ -14,10 +14,10 @@ public class OtsReaderError implements ErrorCode { public final static OtsReaderError ERROR = new OtsReaderError( "OtsReaderError", - "该错误表示插件的内部错误,表示系统没有处理到的异常"); + "This error represents an internal error of the otsreader plugin, which indicates that the system is not processed."); public final static OtsReaderError INVALID_PARAM = new OtsReaderError( "OtsReaderInvalidParameter", - "该错误表示参数错误,表示用户输入了错误的参数格式等"); + "This error represents a parameter error, indicating that the user entered the wrong parameter format."); public OtsReaderError (String code) { this.code = code; diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamChecker.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamChecker.java index fbcdc972..b2139fc1 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamChecker.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamChecker.java @@ -1,162 +1,40 @@ package com.alibaba.datax.plugin.reader.otsreader.utils; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - +import com.alibaba.datax.common.element.Column; import com.alibaba.datax.common.util.Configuration; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSPrimaryKeyColumn; -import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; -import com.aliyun.openservices.ots.model.Direction; -import com.aliyun.openservices.ots.model.PrimaryKeyType; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; -import com.aliyun.openservices.ots.model.RowPrimaryKey; -import com.aliyun.openservices.ots.model.TableMeta; +import com.alibaba.datax.plugin.reader.otsreader.model.*; +import com.alicloud.openservices.tablestore.model.*; + +import java.util.*; public class ParamChecker { - private static void throwNotExistException(String key) { - throw new IllegalArgumentException("The param '" + key + "' is not exist."); + private static void throwNotExistException() { + throw new IllegalArgumentException("missing the key."); } - private static void throwStringLengthZeroException(String key) { - throw new IllegalArgumentException("The param length of '" + key + "' is zero."); + private static void throwStringLengthZeroException() { + throw new IllegalArgumentException("input the key is empty string."); } - private static void throwEmptyException(String key) { - throw new IllegalArgumentException("The param '" + key + "' is empty."); - } - - private static void throwNotListException(String key) { - throw new IllegalArgumentException("The param '" + key + "' is not a json array."); - } - - private static void throwNotMapException(String key) { - throw new IllegalArgumentException("The param '" + key + "' is not a json map."); - } - - public static String checkStringAndGet(Configuration param, String key) { - String value = param.getString(key); - if (null == value) { - throwNotExistException(key); - } else if (value.length() == 0) { - throwStringLengthZeroException(key); - } - return value; - } - - public static List checkListAndGet(Configuration param, String key, boolean isCheckEmpty) { - List value = null; + public static String checkStringAndGet(Configuration param, String key, boolean isTrim) throws OTSCriticalException { try { - value = param.getList(key); - } catch (ClassCastException e) { - throwNotListException(key); - } - if (null == value) { - throwNotExistException(key); - } else if (isCheckEmpty && value.isEmpty()) { - throwEmptyException(key); - } - return value; - } - - public static List checkListAndGet(Map range, String key) { - Object obj = range.get(key); - if (null == obj) { - return null; - } - return checkListAndGet(range, key, false); - } - - public static List checkListAndGet(Map range, String key, boolean isCheckEmpty) { - Object obj = range.get(key); - if (null == obj) { - throwNotExistException(key); - } - if (obj instanceof List) { - @SuppressWarnings("unchecked") - List value = (List)obj; - if (isCheckEmpty && value.isEmpty()) { - throwEmptyException(key); + String value = param.getString(key); + if (isTrim) { + value = value != null ? value.trim() : null; + } + if (null == value) { + throwNotExistException(); + } else if (value.length() == 0) { + throwStringLengthZeroException(); } return value; - } else { - throw new IllegalArgumentException("Can not parse list of '" + key + "' from map."); + } catch(RuntimeException e) { + throw new OTSCriticalException("Parse '"+ key +"' fail, " + e.getMessage(), e); } } - public static List checkListAndGet(Map range, String key, List defaultList) { - Object obj = range.get(key); - if (null == obj) { - return defaultList; - } - if (obj instanceof List) { - @SuppressWarnings("unchecked") - List value = (List)obj; - return value; - } else { - throw new IllegalArgumentException("Can not parse list of '" + key + "' from map."); - } - } - - public static Map checkMapAndGet(Configuration param, String key, boolean isCheckEmpty) { - Map value = null; - try { - value = param.getMap(key); - } catch (ClassCastException e) { - throwNotMapException(key); - } - if (null == value) { - throwNotExistException(key); - } else if (isCheckEmpty && value.isEmpty()) { - throwEmptyException(key); - } - return value; - } - - public static RowPrimaryKey checkInputPrimaryKeyAndGet(TableMeta meta, List range) { - if (meta.getPrimaryKey().size() != range.size()) { - throw new IllegalArgumentException(String.format( - "Input size of values not equal size of primary key. input size:%d, primary key size:%d .", - range.size(), meta.getPrimaryKey().size())); - } - RowPrimaryKey pk = new RowPrimaryKey(); - int i = 0; - for (Entry e: meta.getPrimaryKey().entrySet()) { - PrimaryKeyValue value = range.get(i); - if (e.getValue() != value.getType() && value != PrimaryKeyValue.INF_MIN && value != PrimaryKeyValue.INF_MAX) { - throw new IllegalArgumentException( - "Input range type not match primary key. Input type:" + value.getType() + ", Primary Key Type:"+ e.getValue() +", Index:" + i - ); - } else { - pk.addPrimaryKeyColumn(e.getKey(), value); - } - i++; - } - return pk; - } - - public static OTSRange checkRangeAndGet(TableMeta meta, List begin, List end) { - OTSRange range = new OTSRange(); - if (begin.size() == 0 && end.size() == 0) { - RowPrimaryKey beginRow = new RowPrimaryKey(); - RowPrimaryKey endRow = new RowPrimaryKey(); - for (String name : meta.getPrimaryKey().keySet()) { - beginRow.addPrimaryKeyColumn(name, PrimaryKeyValue.INF_MIN); - endRow.addPrimaryKeyColumn(name, PrimaryKeyValue.INF_MAX); - } - range.setBegin(beginRow); - range.setEnd(endRow); - } else { - RowPrimaryKey beginRow = checkInputPrimaryKeyAndGet(meta, begin); - RowPrimaryKey endRow = checkInputPrimaryKeyAndGet(meta, end); - range.setBegin(beginRow); - range.setEnd(endRow); - } - return range; - } - - public static Direction checkDirectionAndEnd(TableMeta meta, RowPrimaryKey begin, RowPrimaryKey end) { + public static Direction checkDirectionAndEnd(TableMeta meta, List begin, List end) { Direction direction = null; int cmp = Common.compareRangeBeginAndEnd(meta, begin, end) ; @@ -170,76 +48,420 @@ public class ParamChecker { return direction; } - /** - * 检查类型是否一致,是否重复,方向是否一致 - * @param direction - * @param before - * @param after - */ - private static void checkDirection(Direction direction, PrimaryKeyValue before, PrimaryKeyValue after) { - int cmp = Common.primaryKeyValueCmp(before, after); - if (cmp > 0) { // 反向 - if (direction == Direction.FORWARD) { - throw new IllegalArgumentException("Input direction of 'range-split' is FORWARD, but direction of 'range' is BACKWARD."); + public static List checkInputPrimaryKeyAndGet(TableMeta meta, List range) { + if (meta.getPrimaryKeyMap().size() != range.size()) { + throw new IllegalArgumentException(String.format( + "Input size of values not equal size of primary key. input size:%d, primary key size:%d .", + range.size(), meta.getPrimaryKeyMap().size())); + } + List pk = new ArrayList<>(); + int i = 0; + for (Map.Entry e: meta.getPrimaryKeyMap().entrySet()) { + PrimaryKeyValue value = range.get(i); + if (e.getValue() != value.getType() && value != PrimaryKeyValue.INF_MIN && value != PrimaryKeyValue.INF_MAX) { + throw new IllegalArgumentException( + "Input range type not match primary key. Input type:" + value.getType() + ", Primary Key Type:"+ e.getValue() +", Index:" + i + ); + } else { + pk.add(new PrimaryKeyColumn(e.getKey(), value)); } - } else if (cmp < 0) { // 正向 - if (direction == Direction.BACKWARD) { - throw new IllegalArgumentException("Input direction of 'range-split' is BACKWARD, but direction of 'range' is FORWARD."); + i++; + } + return pk; + } + + public static OTSRange checkRangeAndGet(Configuration param) throws OTSCriticalException { + try { + OTSRange range = new OTSRange(); + Map value = param.getMap(Key.RANGE); + // 用户可以不用配置range,默认表示导出全表 + if (value == null) { + return range; } - } else { // 重复列 - throw new IllegalArgumentException("Multi same column in 'range-split'."); + + /** + * Range格式:{ + * "begin":[], + * "end":[] + * } + */ + + // begin + // 如果不存在,表示从表开始位置读取 + Object arrayObj = value.get(Constant.ConfigKey.Range.BEGIN); + if (arrayObj != null) { + range.setBegin(ParamParser.parsePrimaryKeyColumnArray(arrayObj)); + } + + // end + // 如果不存在,表示读取到表的结束位置 + arrayObj = value.get(Constant.ConfigKey.Range.END); + if (arrayObj != null) { + range.setEnd(ParamParser.parsePrimaryKeyColumnArray(arrayObj)); + } + + // split + // 如果不存在,表示不做切分 + arrayObj = value.get(Constant.ConfigKey.Range.SPLIT); + if (arrayObj != null) { + range.setSplit(ParamParser.parsePrimaryKeyColumnArray(arrayObj)); + } + + return range; + } catch (RuntimeException e) { + throw new OTSCriticalException("Parse 'range' fail, " + e.getMessage(), e); + } + + } + + public static TimeRange checkTimeRangeAndGet(Configuration param) throws OTSCriticalException { + try { + + long begin = Constant.ConfigDefaultValue.TimeRange.MIN; + long end = Constant.ConfigDefaultValue.TimeRange.MAX; + + Map value = param.getMap(Constant.ConfigKey.TIME_RANGE); + // 用户可以不用配置time range,默认表示导出全表 + if (value == null) { + return new TimeRange(begin, end); + } + + /** + * TimeRange格式:{ + * "begin":, + * "end": + * } + */ + + // begin + // 如果不存在,表示从表开始位置读取 + Object obj = value.get(Constant.ConfigKey.TimeRange.BEGIN); + if (obj != null) { + begin = ParamParser.parseTimeRangeItem(obj, Constant.ConfigKey.TimeRange.BEGIN); + } + + // end + // 如果不存在,表示读取到表的结束位置 + obj = value.get(Constant.ConfigKey.TimeRange.END); + if (obj != null) { + end = ParamParser.parseTimeRangeItem(obj, Constant.ConfigKey.TimeRange.END); + } + + TimeRange range = new TimeRange(begin, end); + return range; + } catch (RuntimeException e) { + throw new OTSCriticalException("Parse 'timeRange' fail, " + e.getMessage(), e); } } - /** - * 检查 points中的所有点是否是在Begin和end之间 - * @param begin - * @param end - * @param points - */ - private static void checkPointsRange(Direction direction, PrimaryKeyValue begin, PrimaryKeyValue end, List points) { - if (direction == Direction.FORWARD) { - if (!(Common.primaryKeyValueCmp(begin, points.get(0)) < 0 && Common.primaryKeyValueCmp(end, points.get(points.size() - 1)) > 0)) { - throw new IllegalArgumentException("The item of 'range-split' is not within scope of 'range-begin' and 'range-end'."); + private static void checkColumnByMode(List columns , OTSMode mode) { + if (mode == OTSMode.MULTI_VERSION) { + for (OTSColumn c : columns) { + if (c.getColumnType() != OTSColumn.OTSColumnType.NORMAL) { + throw new IllegalArgumentException("in mode:'multiVersion', the 'column' only support specify column_name not const column."); + } } } else { - if (!(Common.primaryKeyValueCmp(begin, points.get(0)) > 0 && Common.primaryKeyValueCmp(end, points.get(points.size() - 1)) < 0)) { - throw new IllegalArgumentException("The item of 'range-split' is not within scope of 'range-begin' and 'range-end'."); + if (columns.isEmpty()) { + throw new IllegalArgumentException("in mode:'normal', the 'column' must specify at least one column_name or const column."); + } + } + } + + public static List checkOTSColumnAndGet(Configuration param, OTSMode mode) throws OTSCriticalException { + try { + List value = param.getList(Key.COLUMN); + // 用户可以不用配置Column + if (value == null) { + value = Collections.emptyList(); + } + + /** + * Column格式:[ + * {"Name":"pk1"}, + * {"type":"Binary","value" : "base64()"} + * ] + */ + List columns = ParamParser.parseOTSColumnArray(value); + checkColumnByMode(columns, mode); + return columns; + } catch (RuntimeException e) { + throw new OTSCriticalException("Parse 'column' fail, " + e.getMessage(), e); + } + } + + public static List checkTimeseriesColumnAndGet(Configuration param) throws OTSCriticalException { + try { + List value = param.getList(Key.COLUMN); + List columns = ParamParser.parseOTSColumnArray(value); + + List columnTypes = checkColumnTypeAndGet(param); + List isTags = checkColumnIsTagAndGet(param); + + for (int i = 0; i < columns.size(); i++) { + columns.get(i).setValueType(columnTypes.get(i)); + columns.get(i).setTimeseriesTag(isTags.get(i)); + } + + checkColumnByMode(columns, OTSMode.NORMAL); + return columns; + } catch (RuntimeException e) { + throw new OTSCriticalException("Parse 'column' fail, " + e.getMessage(), e); + } + } + + public static List checkColumnTypeAndGet(Configuration param) throws OTSCriticalException { + try { + List value = param.getList(Key.COLUMN); + List columnTypes = ParamParser.parseColumnTypeArray(value); + return columnTypes; + } catch (RuntimeException e) { + throw new OTSCriticalException("Parse 'type of column' fail, " + e.getMessage(), e); + } + } + + public static List checkColumnIsTagAndGet(Configuration param) throws OTSCriticalException { + try { + List value = param.getList(Key.COLUMN); + List columnIsTag = ParamParser.parseColumnIsTagArray(value); + return columnIsTag; + } catch (RuntimeException e) { + throw new OTSCriticalException("Parse 'isTag of column' fail, " + e.getMessage(), e); + } + } + + public static OTSMode checkModeAndGet(Configuration param) throws OTSCriticalException { + try { + String modeValue = param.getString(Key.MODE, "normal"); + if (modeValue.equalsIgnoreCase(Constant.ConfigDefaultValue.Mode.NORMAL)) { + return OTSMode.NORMAL; + } else if (modeValue.equalsIgnoreCase(Constant.ConfigDefaultValue.Mode.MULTI_VERSION)) { + return OTSMode.MULTI_VERSION; + } else { + throw new IllegalArgumentException("the 'mode' only support 'normal' and 'multiVersion' not '"+ modeValue +"'."); + } + } catch(RuntimeException e) { + throw new OTSCriticalException("Parse 'mode' fail, " + e.getMessage(), e); + } + } + + public static void checkTimeseriesMode(OTSMode mode, Boolean isNewVersion) throws OTSCriticalException { + if (mode == OTSMode.MULTI_VERSION){ + throw new OTSCriticalException("Timeseries table do not support mode : multiVersion." ); + } else if (!isNewVersion){ + throw new OTSCriticalException("Timeseries table is only supported in newVersion, please set \"newVersion\": \"true\"." ); + } + } + + public static List checkAndGetPrimaryKey( + List pk, + List pkSchema, + String jsonKey){ + List result = new ArrayList(); + if(pk != null) { + if (pk.size() > pkSchema.size()) { + throw new IllegalArgumentException("The '"+ jsonKey +"', input primary key column size more than table meta, input size: "+ pk.size() + +", meta pk size:" + pkSchema.size()); + } else { + //类型检查 + for (int i = 0; i < pk.size(); i++) { + PrimaryKeyValue pkc = pk.get(i).getValue(); + PrimaryKeySchema pkcs = pkSchema.get(i); + + if (!pkc.isInfMin() && !pkc.isInfMax() ) { + if (pkc.getType() != pkcs.getType()) { + throw new IllegalArgumentException( + "The '"+ jsonKey +"', input primary key column type mismath table meta, input type:"+ pkc.getType() + +", meta pk type:"+ pkcs.getType() + +", index:" + i); + } + } + result.add(new PrimaryKeyColumn(pkcs.getName(), pkc)); + } + } + return result; + } else { + return new ArrayList(); + } + } + + /** + * 检查split的类型是否和PartitionKey一致 + * @param points + * @param pkSchema + */ + private static List checkAndGetSplit( + List points, + List pkSchema){ + List result = new ArrayList(); + if (points == null) { + return result; + } + + // check 类型是否和PartitionKey一致即可 + PrimaryKeySchema partitionKeySchema = pkSchema.get(0); + for (int i = 0 ; i < points.size(); i++) { + PrimaryKeyColumn p = points.get(i); + if (!p.getValue().isInfMin() && !p.getValue().isInfMax()) { + if (p.getValue().getType() != partitionKeySchema.getType()) { + throw new IllegalArgumentException("The 'split', input primary key column type is mismatch partition key, input type: "+ p.getValue().getType().toString() + +", partition key type:" + partitionKeySchema.getType().toString() + +", index:" + i); + } + } + result.add(new PrimaryKeyColumn(partitionKeySchema.getName(), p.getValue())); + } + + return result; + } + + public static void fillPrimaryKey(List pkSchema, List pk, PrimaryKeyValue fillValue) { + for(int i = pk.size(); i < pkSchema.size(); i++) { + pk.add(new PrimaryKeyColumn(pkSchema.get(i).getName(), fillValue)); + } + } + + private static void fillBeginAndEnd( + List begin, + List end, + List pkSchema) { + if (begin.isEmpty()) { + fillPrimaryKey(pkSchema, begin, PrimaryKeyValue.INF_MIN); + } + if (end.isEmpty()) { + fillPrimaryKey(pkSchema, end, PrimaryKeyValue.INF_MAX); + } + int cmp = CompareHelper.comparePrimaryKeyColumnList(begin, end); + if (cmp == 0) { + // begin.size()和end.size()理论上必然相等,但是考虑到语义的清晰性,显示的给出begin.size() == end.size() + if (begin.size() == end.size() && begin.size() < pkSchema.size()) { + fillPrimaryKey(pkSchema, begin, PrimaryKeyValue.INF_MIN); + fillPrimaryKey(pkSchema, end, PrimaryKeyValue.INF_MAX); + } else { + throw new IllegalArgumentException("The 'begin' can not be equal with 'end'."); + } + } else if (cmp < 0) { // 升序 + fillPrimaryKey(pkSchema, begin, PrimaryKeyValue.INF_MIN); + fillPrimaryKey(pkSchema, end, PrimaryKeyValue.INF_MAX); + } else { // 降序 + fillPrimaryKey(pkSchema, begin, PrimaryKeyValue.INF_MAX); + fillPrimaryKey(pkSchema, end, PrimaryKeyValue.INF_MIN); + } + } + + private static void checkBeginAndEndAndSplit( + List begin, + List end, + List split) { + int cmp = CompareHelper.comparePrimaryKeyColumnList(begin, end); + + if (!split.isEmpty()) { + if (cmp < 0) { // 升序 + // 检查是否是升序 + for (int i = 0 ; i < split.size() - 1; i++) { + PrimaryKeyColumn before = split.get(i); + PrimaryKeyColumn after = split.get(i + 1); + if (before.compareTo(after) >=0) { // 升序 + throw new IllegalArgumentException("In 'split', the item value is not increasing, index: " + i); + } + } + if (begin.get(0).compareTo(split.get(0)) >= 0) { + throw new IllegalArgumentException("The 'begin' must be less than head of 'split'."); + } + if (split.get(split.size() - 1).compareTo(end.get(0)) >= 0) { + throw new IllegalArgumentException("tail of 'split' must be less than 'end'."); + } + } else if (cmp > 0) {// 降序 + // 检查是否是降序 + for (int i = 0 ; i < split.size() - 1; i++) { + PrimaryKeyColumn before = split.get(i); + PrimaryKeyColumn after = split.get(i + 1); + if (before.compareTo(after) <= 0) { // 升序 + throw new IllegalArgumentException("In 'split', the item value is not descending, index: " + i); + } + } + if (begin.get(0).compareTo(split.get(0)) <= 0) { + throw new IllegalArgumentException("The 'begin' must be large than head of 'split'."); + } + if (split.get(split.size() - 1).compareTo(end.get(0)) <= 0) { + throw new IllegalArgumentException("tail of 'split' must be large than 'end'."); + } + } else { + throw new IllegalArgumentException("The 'begin' can not equal with 'end'."); } } } /** - * 1.检测用户的输入类型是否和PartitionKey一致 - * 2.顺序是否和Range一致 - * 3.是否有重复列 - * 4.检查points的范围是否在range内 - * @param meta - * @param points + * 填充不完整的PK + * 检查Begin、End、Split 3者之间的关系是否符合预期 + * @param begin + * @param end + * @param split */ - public static void checkInputSplitPoints(TableMeta meta, OTSRange range, Direction direction, List points) { - if (null == points || points.isEmpty()) { - return; - } + private static void fillAndcheckBeginAndEndAndSplit( + List begin, + List end, + List split, + List pkSchema + ) { - OTSPrimaryKeyColumn part = Common.getPartitionKey(meta); - - // 处理第一个 - PrimaryKeyValue item = points.get(0); - if ( item.getType() != part.getType()) { - throw new IllegalArgumentException("Input type of 'range-split' not match partition key. " - + "Item of 'range-split' type:" + item.getType()+ ", Partition type:" + part.getType()); - } - - for (int i = 0 ; i < points.size() - 1; i++) { - PrimaryKeyValue before = points.get(i); - PrimaryKeyValue after = points.get(i + 1); - checkDirection(direction, before, after); - } - - PrimaryKeyValue begin = range.getBegin().getPrimaryKey().get(part.getName()); - PrimaryKeyValue end = range.getEnd().getPrimaryKey().get(part.getName()); - - checkPointsRange(direction, begin, end, points); + fillBeginAndEnd(begin, end, pkSchema); + checkBeginAndEndAndSplit(begin, end, split); } + + public static void checkAndSetOTSRange(OTSRange range, TableMeta meta) throws OTSCriticalException { + try { + List pkSchema = meta.getPrimaryKeyList(); + + // 检查是begin和end否和PK类型一致 + range.setBegin(checkAndGetPrimaryKey(range.getBegin(), pkSchema, Constant.ConfigKey.Range.BEGIN)); + range.setEnd(checkAndGetPrimaryKey(range.getEnd(), pkSchema, Constant.ConfigKey.Range.END)); + range.setSplit(checkAndGetSplit(range.getSplit(), pkSchema)); + + // 1.填充Begin和End + // 2.检查begin,end,split顺序是否正确 + fillAndcheckBeginAndEndAndSplit(range.getBegin(), range.getEnd(), range.getSplit(), pkSchema); + } catch(RuntimeException e) { + throw new OTSCriticalException("Parse 'range' fail, " + e.getMessage(), e); + } + } + + public static void checkAndSetColumn(List columns, TableMeta meta, OTSMode mode) throws OTSCriticalException { + try { + if (mode == OTSMode.MULTI_VERSION) { + Set uniqueColumn = new HashSet(); + Map pk = meta.getPrimaryKeyMap(); + for (OTSColumn c : columns) { + // 是否包括PK列 + if (pk.get(c.getName()) != null) { + throw new IllegalArgumentException("in mode:'multiVersion', the 'column' can not include primary key column, input:"+ c.getName() +"."); + } + // 是否有重复列 + if (uniqueColumn.contains(c.getName())) { + throw new IllegalArgumentException("in mode:'multiVersion', the 'column' can not include same column, input:"+ c.getName() +"."); + } else { + uniqueColumn.add(c.getName()); + } + } + } + + } catch(RuntimeException e) { + throw new OTSCriticalException("Parse 'column' fail, " + e.getMessage(), e); + } + } + + public static void normalCheck(OTSConf conf) { + // 旧版本不支持multiVersion模式 + if(!conf.isNewVersion() && conf.getMode() == OTSMode.MULTI_VERSION){ + throw new IllegalArgumentException("in mode:'multiVersion' :The old version do not support multiVersion mode. Please add config in otsreader: \"newVersion\":\"true\" ."); + } + } + + public static void checkAndSetOTSConf(OTSConf conf, TableMeta meta) throws OTSCriticalException { + normalCheck(conf); + checkAndSetOTSRange(conf.getRange(), meta); + checkAndSetColumn(conf.getColumn(), meta, conf.getMode()); + } + } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamCheckerOld.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamCheckerOld.java new file mode 100644 index 00000000..3489ab35 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamCheckerOld.java @@ -0,0 +1,36 @@ +package com.alibaba.datax.plugin.reader.otsreader.utils; + +import com.alibaba.datax.common.util.Configuration; + +import java.util.List; + +public class ParamCheckerOld { + + private static void throwNotExistException(String key) { + throw new IllegalArgumentException("The param '" + key + "' is not exist."); + } + + private static void throwEmptyException(String key) { + throw new IllegalArgumentException("The param '" + key + "' is empty."); + } + + private static void throwNotListException(String key) { + throw new IllegalArgumentException("The param '" + key + "' is not a json array."); + } + + public static List checkListAndGet(Configuration param, String key, boolean isCheckEmpty) { + List value = null; + try { + value = param.getList(key); + } catch (ClassCastException e) { + throwNotListException(key); + } + if (null == value) { + throwNotExistException(key); + } else if (isCheckEmpty && value.isEmpty()) { + throwEmptyException(key); + } + return value; + } + +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamParser.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamParser.java new file mode 100644 index 00000000..862b915c --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ParamParser.java @@ -0,0 +1,255 @@ +package com.alibaba.datax.plugin.reader.otsreader.utils; + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSColumn; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSCriticalException; +import com.alicloud.openservices.tablestore.model.ColumnType; +import com.alicloud.openservices.tablestore.model.PrimaryKeyColumn; +import com.alicloud.openservices.tablestore.model.PrimaryKeyValue; +import org.apache.commons.codec.binary.Base64; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class ParamParser { + + // ------------------------------------------------------------------------ + // Range解析相关的逻辑 + // ------------------------------------------------------------------------ + + private static PrimaryKeyValue parsePrimaryKeyValue(String type) { + return parsePrimaryKeyValue(type, null); + } + + private static PrimaryKeyValue parsePrimaryKeyValue(String type, String value) { + if (type.equalsIgnoreCase(Constant.ValueType.INF_MIN)) { + return PrimaryKeyValue.INF_MIN; + } else if (type.equalsIgnoreCase(Constant.ValueType.INF_MAX)) { + return PrimaryKeyValue.INF_MAX; + } else { + if (value != null) { + if (type.equalsIgnoreCase(Constant.ValueType.STRING)) { + return PrimaryKeyValue.fromString(value); + } else if (type.equalsIgnoreCase(Constant.ValueType.INTEGER)) { + return PrimaryKeyValue.fromLong(Long.valueOf(value)); + } else if (type.equalsIgnoreCase(Constant.ValueType.BINARY)) { + return PrimaryKeyValue.fromBinary(Base64.decodeBase64(value)); + } else { + throw new IllegalArgumentException("the column type only support :['INF_MIN', 'INF_MAX', 'string', 'int', 'binary']"); + } + } else { + throw new IllegalArgumentException("the column is missing the field 'value', input 'type':" + type); + } + } + } + + private static PrimaryKeyColumn parsePrimaryKeyColumn(Map item) { + Object typeObj = item.get(Constant.ConfigKey.PrimaryKeyColumn.TYPE); + Object valueObj = item.get(Constant.ConfigKey.PrimaryKeyColumn.VALUE); + + if (typeObj != null && valueObj != null) { + if (typeObj instanceof String && valueObj instanceof String) { + return new PrimaryKeyColumn( + Constant.ConfigDefaultValue.DEFAULT_NAME, + parsePrimaryKeyValue((String)typeObj, (String)valueObj) + ); + } else { + throw new IllegalArgumentException( + "the column's 'type' and 'value' must be string value, " + + "but type of 'type' is :" + typeObj.getClass() + + ", type of 'value' is :" + valueObj.getClass() + ); + } + } else if (typeObj != null) { + if (typeObj instanceof String) { + return new PrimaryKeyColumn( + Constant.ConfigDefaultValue.DEFAULT_NAME, + parsePrimaryKeyValue((String)typeObj) + ); + } else { + throw new IllegalArgumentException( + "the column's 'type' must be string value, " + + "but type of 'type' is :" + typeObj.getClass() + ); + } + } else { + throw new IllegalArgumentException("the column must include 'type' and 'value'."); + } + } + + @SuppressWarnings("unchecked") + public static List parsePrimaryKeyColumnArray(Object arrayObj) throws OTSCriticalException { + try { + List columns = new ArrayList(); + if (arrayObj instanceof List) { + List array = (List) arrayObj; + for (Object o : array) { + if (o instanceof Map) { + Map column = (Map) o; + columns.add(parsePrimaryKeyColumn(column)); + } else { + throw new IllegalArgumentException("input primary key column must be map object, but input type:" + o.getClass()); + } + } + } else { + throw new IllegalArgumentException("input 'begin','end','split' must be list object, but input type:" + arrayObj.getClass()); + } + return columns; + } catch (RuntimeException e) { + // 因为基础模块本身可能抛出一些错误,为了方便定位具体的出错位置,在此把Range加入到Error Message中 + throw new OTSCriticalException("Parse 'range' fail, " + e.getMessage(), e); + } + } + + // ------------------------------------------------------------------------ + // Column解析相关的逻辑 + // ------------------------------------------------------------------------ + + private static OTSColumn parseOTSColumn(Object obj) { + if (obj instanceof String) { + return OTSColumn.fromNormalColumn((String)obj); + } else { + throw new IllegalArgumentException("the 'name' must be string, but input:" + obj.getClass()); + } + } + + private static OTSColumn parseOTSColumn(Object typeObj, Object valueObj) { + if (typeObj instanceof String && valueObj instanceof String) { + String type = (String)typeObj; + String value = (String)valueObj; + + if (type.equalsIgnoreCase(Constant.ValueType.STRING)) { + return OTSColumn.fromConstStringColumn(value); + } else if (type.equalsIgnoreCase(Constant.ValueType.INTEGER)) { + return OTSColumn.fromConstIntegerColumn(Long.valueOf(value)); + } else if (type.equalsIgnoreCase(Constant.ValueType.DOUBLE)) { + return OTSColumn.fromConstDoubleColumn(Double.valueOf(value)); + } else if (type.equalsIgnoreCase(Constant.ValueType.BOOLEAN)) { + return OTSColumn.fromConstBoolColumn(Boolean.valueOf(value)); + } else if (type.equalsIgnoreCase(Constant.ValueType.BINARY)) { + return OTSColumn.fromConstBytesColumn(Base64.decodeBase64(value)); + } else { + throw new IllegalArgumentException("the const column type only support :['string', 'int', 'double', 'bool', 'binary']"); + } + } else { + throw new IllegalArgumentException("the 'type' and 'value' must be string, but 'type''s type:" + typeObj.getClass() + " 'value''s type:" + valueObj.getClass()); + } + } + + private static OTSColumn parseOTSColumn(Map column) { + Object typeObj = column.get(Constant.ConfigKey.Column.TYPE); + Object valueObj = column.get(Constant.ConfigKey.Column.VALUE); + Object nameObj = column.get(Constant.ConfigKey.Column.NAME); + + if (nameObj != null) { + return parseOTSColumn(nameObj); + } else if (typeObj != null && valueObj != null) { + return parseOTSColumn(typeObj, valueObj); + } else { + throw new IllegalArgumentException("the item of column format support '{\"name\":\"\"}' or '{\"type\":\"\", \"value\":\"\"}'."); + } + } + + @SuppressWarnings("unchecked") + public static List parseOTSColumnArray(List value) throws OTSCriticalException { + try { + List result = new ArrayList(); + for (Object item:value) { + if (item instanceof Map){ + Map column = (Map) item; + result.add(ParamParser.parseOTSColumn(column)); + } else { + throw new IllegalArgumentException("the item of column must be map object, but input: " + item.getClass()); + } + } + return result; + } catch (RuntimeException e) { + // 因为基础模块本身可能抛出一些错误,为了方便定位具体的出错位置,在此把Column加入到Error Message中 + throw new OTSCriticalException("Parse 'column' fail. " + e.getMessage(), e); + } + } + + private static ColumnType parseTimeseriesColumnType(Map column) { + Object typeObj = column.getOrDefault(Constant.ConfigKey.Column.TYPE, ""); + if (typeObj instanceof String) { + String type = (String)typeObj; + + if (type.equalsIgnoreCase(Constant.ValueType.STRING)) { + return ColumnType.STRING; + } else if (type.equalsIgnoreCase(Constant.ValueType.INTEGER)) { + return ColumnType.INTEGER; + } else if (type.equalsIgnoreCase(Constant.ValueType.DOUBLE)) { + return ColumnType.DOUBLE; + } else if (type.equalsIgnoreCase(Constant.ValueType.BOOLEAN)) { + return ColumnType.BOOLEAN; + } else if (type.equalsIgnoreCase(Constant.ValueType.BINARY)) { + return ColumnType.BINARY; + } else if (type.length() == 0){ + return ColumnType.STRING; + }else { + throw new IllegalArgumentException("the timeseries column type only support :['string', 'int', 'double', 'bool', 'binary']"); + } + } else { + throw new IllegalArgumentException("the 'type' must be string, but 'type''s type:" + typeObj.getClass()); + } + } + + public static List parseColumnTypeArray(List value) throws OTSCriticalException { + try { + List result = new ArrayList(); + for (Object item:value) { + if (item instanceof Map){ + Map column = (Map) item; + result.add(ParamParser.parseTimeseriesColumnType(column)); + } else { + throw new IllegalArgumentException("the item of column must be map object, but input: " + item.getClass()); + } + } + return result; + } catch (RuntimeException e) { + throw new OTSCriticalException("Parse 'timeseries column type' fail. " + e.getMessage(), e); + } + } + + private static Boolean parseTimeseriesColumnIsTag(Map column) { + Object isTagParameter = column.getOrDefault(Constant.ConfigKey.Column.IS_TAG, ""); + if (isTagParameter instanceof String) { + String isTag = (String)isTagParameter; + return Boolean.valueOf(isTag); + } else { + throw new IllegalArgumentException("the 'isTag' must be string, but 'isTag''s type:" + isTagParameter.getClass()); + } + } + + public static List parseColumnIsTagArray(List value) throws OTSCriticalException { + try { + List result = new ArrayList(); + for (Object item:value) { + if (item instanceof Map){ + Map column = (Map) item; + result.add(ParamParser.parseTimeseriesColumnIsTag(column)); + } else { + throw new IllegalArgumentException("the item of column must be map object, but input: " + item.getClass()); + } + } + return result; + } catch (RuntimeException e) { + throw new OTSCriticalException("Parse 'timeseries column isTag' fail. " + e.getMessage(), e); + } + } + + // ------------------------------------------------------------------------ + // TimeRange解析相关的逻辑 + // ------------------------------------------------------------------------ + + public static long parseTimeRangeItem(Object obj, String key) { + if (obj instanceof Integer) { + return (Integer)obj; + } else if (obj instanceof Long) { + return (Long)obj; + } else { + throw new IllegalArgumentException("the '"+ key +"' must be int, but input:" + obj.getClass()); + } + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RangeSplit.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RangeSplit.java index 74caac3f..fbef9279 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RangeSplit.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RangeSplit.java @@ -1,17 +1,15 @@ package com.alibaba.datax.plugin.reader.otsreader.utils; -import java.math.BigInteger; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; - import com.alibaba.datax.plugin.reader.otsreader.model.OTSPrimaryKeyColumn; import com.alibaba.datax.plugin.reader.otsreader.model.OTSRange; -import com.aliyun.openservices.ots.model.PrimaryKeyType; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; -import com.aliyun.openservices.ots.model.RowPrimaryKey; -import com.aliyun.openservices.ots.model.TableMeta; +import com.alicloud.openservices.tablestore.model.PrimaryKeyColumn; +import com.alicloud.openservices.tablestore.model.PrimaryKeyType; +import com.alicloud.openservices.tablestore.model.PrimaryKeyValue; +import com.alicloud.openservices.tablestore.model.TableMeta; + + +import java.math.BigInteger; +import java.util.*; /** * 主要提供对范围的解析 @@ -35,8 +33,8 @@ public class RangeSplit { * * 注意:该方法只支持begin小于end * - * @param beginStr - * @param endStr + * @param begin + * @param end * @param count * @return */ @@ -88,7 +86,6 @@ public class RangeSplit { * @return */ public static List splitStringRange(String begin, String end, int count) { - if (count <= 1) { throw new IllegalArgumentException("Input count <= 1 ."); } @@ -136,15 +133,14 @@ public class RangeSplit { } results.add(end); - return results; } /** * begin 一定要小于 end - * @param begin - * @param end - * @param count + * @param bigBegin + * @param bigEnd + * @param bigCount * @return */ private static List splitIntegerRange(BigInteger bigBegin, BigInteger bigEnd, BigInteger bigCount) { @@ -228,20 +224,31 @@ public class RangeSplit { } public static List rangeSplitByCount(TableMeta meta, - RowPrimaryKey begin, RowPrimaryKey end, int count) { + List begin, List end, int count) { List results = new ArrayList(); OTSPrimaryKeyColumn partitionKey = Common.getPartitionKey(meta); - PrimaryKeyValue beginPartitionKey = begin.getPrimaryKey().get( + Map beginMap = new HashMap<>(); + Map endMap = new HashMap<>(); + + for(PrimaryKeyColumn primaryKeyColumn : begin){ + beginMap.put(primaryKeyColumn.getName(), primaryKeyColumn.getValue()); + } + for(PrimaryKeyColumn primaryKeyColumn : end){ + endMap.put(primaryKeyColumn.getName(), primaryKeyColumn.getValue()); + } + + + PrimaryKeyValue beginPartitionKey = beginMap.get( partitionKey.getName()); - PrimaryKeyValue endPartitionKey = end.getPrimaryKey().get( + PrimaryKeyValue endPartitionKey = endMap.get( partitionKey.getName()); // 第一,先对PartitionKey列进行拆分 List ranges = RangeSplit.splitRangeByPrimaryKeyType( - partitionKey.getType(), beginPartitionKey, endPartitionKey, + partitionKey.getType(true), beginPartitionKey, endPartitionKey, count); if (ranges.isEmpty()) { @@ -250,130 +257,44 @@ public class RangeSplit { int size = ranges.size(); for (int i = 0; i < size - 1; i++) { - RowPrimaryKey bPk = new RowPrimaryKey(); - RowPrimaryKey ePk = new RowPrimaryKey(); + List bPk = new ArrayList<>(); + List ePk = new ArrayList<>(); - bPk.addPrimaryKeyColumn(partitionKey.getName(), ranges.get(i)); - ePk.addPrimaryKeyColumn(partitionKey.getName(), ranges.get(i + 1)); + bPk.add(new PrimaryKeyColumn(partitionKey.getName(), ranges.get(i))); + ePk.add(new PrimaryKeyColumn(partitionKey.getName(), ranges.get(i + 1))); - results.add(new OTSRange(bPk, ePk)); + OTSRange range = new OTSRange(); + range.setBegin(bPk); + range.setEnd(ePk); + results.add(range); } // 第二,填充非PartitionKey的ParimaryKey列 // 注意:在填充过程中,需要使用用户给定的Begin和End来替换切分出来的第一个Range // 的Begin和最后一个Range的End - List keys = new ArrayList(meta.getPrimaryKey().size()); - keys.addAll(meta.getPrimaryKey().keySet()); + List keys = new ArrayList(meta.getPrimaryKeyMap().size()); + keys.addAll(meta.getPrimaryKeyMap().keySet()); for (int i = 0; i < results.size(); i++) { for (int j = 1; j < keys.size(); j++) { OTSRange c = results.get(i); - RowPrimaryKey beginPK = c.getBegin(); - RowPrimaryKey endPK = c.getEnd(); + List beginPK = c.getBegin(); + List endPK = c.getEnd(); String key = keys.get(j); if (i == 0) { // 第一行 - beginPK.addPrimaryKeyColumn(key, - begin.getPrimaryKey().get(key)); - endPK.addPrimaryKeyColumn(key, PrimaryKeyValue.INF_MIN); + beginPK.add(new PrimaryKeyColumn(key, + beginMap.get(key))); + endPK.add(new PrimaryKeyColumn(key, PrimaryKeyValue.INF_MIN)); } else if (i == results.size() - 1) {// 最后一行 - beginPK.addPrimaryKeyColumn(key, PrimaryKeyValue.INF_MIN); - endPK.addPrimaryKeyColumn(key, end.getPrimaryKey().get(key)); + beginPK.add(new PrimaryKeyColumn(key, PrimaryKeyValue.INF_MIN)); + endPK.add(new PrimaryKeyColumn(key, endMap.get(key))); } else { - beginPK.addPrimaryKeyColumn(key, PrimaryKeyValue.INF_MIN); - endPK.addPrimaryKeyColumn(key, PrimaryKeyValue.INF_MIN); + beginPK.add(new PrimaryKeyColumn(key, PrimaryKeyValue.INF_MIN)); + endPK.add(new PrimaryKeyColumn(key, PrimaryKeyValue.INF_MIN)); } } } return results; } - - private static List getCompletePK(int num, - PrimaryKeyValue value) { - List values = new ArrayList(); - for (int j = 0; j < num; j++) { - if (j == 0) { - values.add(value); - } else { - // 这里在填充PK时,系统需要选择特定的值填充于此 - // 系统默认填充INF_MIN - values.add(PrimaryKeyValue.INF_MIN); - } - } - return values; - } - - /** - * 根据输入的范围begin和end,从target中取得对应的point - * @param begin - * @param end - * @param target - * @return - */ - public static List getSplitPoint(PrimaryKeyValue begin, PrimaryKeyValue end, List target) { - List result = new ArrayList(); - - int cmp = Common.primaryKeyValueCmp(begin, end); - - if (cmp == 0) { - return result; - } - - result.add(begin); - - Comparator comparator = new Comparator(){ - public int compare(PrimaryKeyValue arg0, PrimaryKeyValue arg1) { - return Common.primaryKeyValueCmp(arg0, arg1); - } - }; - - if (cmp > 0) { // 如果是逆序,则 reverse Comparator - comparator = Collections.reverseOrder(comparator); - } - - Collections.sort(target, comparator); - - for (PrimaryKeyValue value:target) { - if (comparator.compare(value, begin) > 0 && comparator.compare(value, end) < 0) { - result.add(value); - } - } - result.add(end); - - return result; - } - - public static List rangeSplitByPoint(TableMeta meta, RowPrimaryKey beginPK, RowPrimaryKey endPK, - List splits) { - - List results = new ArrayList(); - - int pkCount = meta.getPrimaryKey().size(); - - String partName = Common.getPartitionKey(meta).getName(); - PrimaryKeyValue begin = beginPK.getPrimaryKey().get(partName); - PrimaryKeyValue end = endPK.getPrimaryKey().get(partName); - - List newSplits = getSplitPoint(begin, end, splits); - - if (newSplits.isEmpty()) { - return results; - } - - for (int i = 0; i < newSplits.size() - 1; i++) { - OTSRange item = new OTSRange( - ParamChecker.checkInputPrimaryKeyAndGet(meta, - getCompletePK(pkCount, newSplits.get(i))), - ParamChecker.checkInputPrimaryKeyAndGet(meta, - getCompletePK(pkCount, newSplits.get(i + 1)))); - results.add(item); - } - // replace first and last - OTSRange first = results.get(0); - OTSRange last = results.get(results.size() - 1); - - first.setBegin(beginPK); - last.setEnd(endPK); - return results; - } } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ReaderModelParser.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ReaderModelParser.java index 8e1dfd41..081532a6 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ReaderModelParser.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/ReaderModelParser.java @@ -55,7 +55,7 @@ public class ReaderModelParser { } public static OTSColumn parseOTSColumn(Map item) { - if (item.containsKey(OTSConst.NAME) && item.size() == 1) { + if (item.containsKey(OTSConst.NAME)) { Object name = item.get(OTSConst.NAME); if (name instanceof String) { String nameStr = (String) name; diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RetryHelper.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RetryHelper.java index 8ed41267..318b7b51 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RetryHelper.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RetryHelper.java @@ -1,16 +1,15 @@ package com.alibaba.datax.plugin.reader.otsreader.utils; +import com.alibaba.datax.plugin.reader.otsreader.model.OTSErrorCode; +import com.alicloud.openservices.tablestore.ClientException; +import com.alicloud.openservices.tablestore.TableStoreException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.HashSet; import java.util.Set; import java.util.concurrent.Callable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.aliyun.openservices.ots.ClientException; -import com.aliyun.openservices.ots.OTSErrorCode; -import com.aliyun.openservices.ots.OTSException; - public class RetryHelper { private static final Logger LOG = LoggerFactory.getLogger(RetryHelper.class); @@ -19,7 +18,7 @@ public class RetryHelper { public static V executeWithRetry(Callable callable, int maxRetryTimes, int sleepInMilliSecond) throws Exception { int retryTimes = 0; while (true){ - Thread.sleep(Common.getDelaySendMillinSeconds(retryTimes, sleepInMilliSecond)); + Thread.sleep(getDelaySendMillinSeconds(retryTimes, sleepInMilliSecond)); try { return callable.call(); } catch (Exception e) { @@ -60,9 +59,9 @@ public class RetryHelper { } public static boolean canRetry(Exception exception) { - OTSException e = null; - if (exception instanceof OTSException) { - e = (OTSException) exception; + TableStoreException e = null; + if (exception instanceof TableStoreException) { + e = (TableStoreException) exception; LOG.warn( "OTSException:ErrorCode:{}, ErrorMsg:{}, RequestId:{}", new Object[]{e.getErrorCode(), e.getMessage(), e.getRequestId()} @@ -72,12 +71,29 @@ public class RetryHelper { } else if (exception instanceof ClientException) { ClientException ce = (ClientException) exception; LOG.warn( - "ClientException:{}, ErrorMsg:{}", - new Object[]{ce.getErrorCode(), ce.getMessage()} + "ClientException:{}", + new Object[]{ce.getMessage()} ); return true; } else { return false; } } + + public static long getDelaySendMillinSeconds(int hadRetryTimes, int initSleepInMilliSecond) { + + if (hadRetryTimes <= 0) { + return 0; + } + + int sleepTime = initSleepInMilliSecond; + for (int i = 1; i < hadRetryTimes; i++) { + sleepTime += sleepTime; + if (sleepTime > 30000) { + sleepTime = 30000; + break; + } + } + return sleepTime; + } } diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RetryHelperOld.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RetryHelperOld.java new file mode 100644 index 00000000..28ad4ee3 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/RetryHelperOld.java @@ -0,0 +1,83 @@ +package com.alibaba.datax.plugin.reader.otsreader.utils; + +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.Callable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.aliyun.openservices.ots.ClientException; +import com.aliyun.openservices.ots.OTSErrorCode; +import com.aliyun.openservices.ots.OTSException; + +public class RetryHelperOld { + + private static final Logger LOG = LoggerFactory.getLogger(RetryHelperOld.class); + private static final Set noRetryErrorCode = prepareNoRetryErrorCode(); + + public static V executeWithRetry(Callable callable, int maxRetryTimes, int sleepInMilliSecond) throws Exception { + int retryTimes = 0; + while (true){ + Thread.sleep(CommonOld.getDelaySendMillinSeconds(retryTimes, sleepInMilliSecond)); + try { + return callable.call(); + } catch (Exception e) { + LOG.warn("Call callable fail, {}", e.getMessage()); + if (!canRetry(e)){ + LOG.error("Can not retry for Exception.", e); + throw e; + } else if (retryTimes >= maxRetryTimes) { + LOG.error("Retry times more than limition. maxRetryTimes : {}", maxRetryTimes); + throw e; + } + retryTimes++; + LOG.warn("Retry time : {}", retryTimes); + } + } + } + + private static Set prepareNoRetryErrorCode() { + Set pool = new HashSet(); + pool.add(OTSErrorCode.AUTHORIZATION_FAILURE); + pool.add(OTSErrorCode.INVALID_PARAMETER); + pool.add(OTSErrorCode.REQUEST_TOO_LARGE); + pool.add(OTSErrorCode.OBJECT_NOT_EXIST); + pool.add(OTSErrorCode.OBJECT_ALREADY_EXIST); + pool.add(OTSErrorCode.INVALID_PK); + pool.add(OTSErrorCode.OUT_OF_COLUMN_COUNT_LIMIT); + pool.add(OTSErrorCode.OUT_OF_ROW_SIZE_LIMIT); + pool.add(OTSErrorCode.CONDITION_CHECK_FAIL); + return pool; + } + + public static boolean canRetry(String otsErrorCode) { + if (noRetryErrorCode.contains(otsErrorCode)) { + return false; + } else { + return true; + } + } + + public static boolean canRetry(Exception exception) { + OTSException e = null; + if (exception instanceof OTSException) { + e = (OTSException) exception; + LOG.warn( + "OTSException:ErrorCode:{}, ErrorMsg:{}, RequestId:{}", + new Object[]{e.getErrorCode(), e.getMessage(), e.getRequestId()} + ); + return canRetry(e.getErrorCode()); + + } else if (exception instanceof ClientException) { + ClientException ce = (ClientException) exception; + LOG.warn( + "ClientException:{}, ErrorMsg:{}", + new Object[]{ce.getErrorCode(), ce.getMessage()} + ); + return true; + } else { + return false; + } + } +} diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/TranformHelper.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/TranformHelper.java new file mode 100644 index 00000000..b082e658 --- /dev/null +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/TranformHelper.java @@ -0,0 +1,42 @@ +package com.alibaba.datax.plugin.reader.otsreader.utils; + +import com.alibaba.datax.common.element.*; +import com.alicloud.openservices.tablestore.model.PrimaryKeyColumn; + +public class TranformHelper { + + public static Column otsPrimaryKeyColumnToDataxColumn(PrimaryKeyColumn pkc) { + switch (pkc.getValue().getType()) { + case STRING:return new StringColumn(pkc.getValue().asString()); + case INTEGER:return new LongColumn(pkc.getValue().asLong()); + case BINARY:return new BytesColumn(pkc.getValue().asBinary()); + default: + throw new IllegalArgumentException("PrimaryKey unsuporrt tranform the type: " + pkc.getValue().getType() + "."); + } + } + + public static Column otsColumnToDataxColumn(com.alicloud.openservices.tablestore.model.Column c) { + switch (c.getValue().getType()) { + case STRING:return new StringColumn(c.getValue().asString()); + case INTEGER:return new LongColumn(c.getValue().asLong()); + case BINARY:return new BytesColumn(c.getValue().asBinary()); + case BOOLEAN:return new BoolColumn(c.getValue().asBoolean()); + case DOUBLE:return new DoubleColumn(c.getValue().asDouble()); + default: + throw new IllegalArgumentException("Column unsuporrt tranform the type: " + c.getValue().getType() + "."); + + } + } + + public static Column otsColumnToDataxColumn(com.alicloud.openservices.tablestore.model.ColumnValue c) { + switch (c.getType()) { + case STRING:return new StringColumn(c.asString()); + case INTEGER:return new LongColumn(c.asLong()); + case BINARY:return new BytesColumn(c.asBinary()); + case BOOLEAN:return new BoolColumn(c.asBoolean()); + case DOUBLE:return new DoubleColumn(c.asDouble()); + default: + throw new IllegalArgumentException("Column unsuporrt tranform the type: " + c.getType() + "."); + } + } +} diff --git a/otsreader/src/main/resources/plugin.json b/otsreader/src/main/resources/plugin.json index bfd95627..4b55e374 100644 --- a/otsreader/src/main/resources/plugin.json +++ b/otsreader/src/main/resources/plugin.json @@ -3,4 +3,4 @@ "class": "com.alibaba.datax.plugin.reader.otsreader.OtsReader", "description": "", "developer": "alibaba" -} \ No newline at end of file +} diff --git a/otsstreamreader/README.md b/otsstreamreader/README.md index c861a737..5e68f1eb 100644 --- a/otsstreamreader/README.md +++ b/otsstreamreader/README.md @@ -1,127 +1,152 @@ ## TableStore增量数据导出通道:TableStoreStreamReader +本文为您介绍OTSStream Reader支持的数据类型、读取方式、字段映射和数据源等参数及配置示例。 +## 列模式 -### 快速介绍 -TableStoreStreamReader插件主要用于TableStore的增量数据导出,增量数据可以看作操作日志,除了数据本身外还附有操作信息。 +### 背景信息 -与全量导出插件不同,增量导出插件只有多版本模式,同时不支持指定列。这是与增量导出的原理有关的,导出的格式下面有详细介绍。 +OTSStream Reader插件主要用于导出Table Store的增量数据。您可以将增量数据看作操作日志,除数据本身外还附有操作信息。 -使用插件前必须确保表上已经开启Stream功能,可以在建表的时候指定开启,或者使用SDK的UpdateTable接口开启。 +与全量导出插件不同,增量导出插件只有多版本模式,且不支持指定列。使用插件前,您必须确保表上已经开启Stream功能。您可以在建表时指定开启,也可以使用SDK的UpdateTable接口开启。 - 开启Stream的方法: - SyncClient client = new SyncClient("", "", "", ""); - 1. 建表的时候开启: - CreateTableRequest createTableRequest = new CreateTableRequest(tableMeta); - createTableRequest.setStreamSpecification(new StreamSpecification(true, 24)); // 24代表增量数据保留24小时 - client.createTable(createTableRequest); - - 2. 如果建表时未开启,可以通过UpdateTable开启: - UpdateTableRequest updateTableRequest = new UpdateTableRequest("tableName"); - updateTableRequest.setStreamSpecification(new StreamSpecification(true, 24)); - client.updateTable(updateTableRequest); +开启Stream的方法,如下所示。 +```java +SyncClient client = new SyncClient("", "", "", ""); +#建表的时候开启: +CreateTableRequest createTableRequest = new CreateTableRequest(tableMeta); +createTableRequest.setStreamSpecification(new StreamSpecification(true, 24)); // 24代表增量数据保留24小时。 +client.createTable(createTableRequest); +#如果建表时未开启,您可以通过UpdateTable开启: +UpdateTableRequest updateTableRequest = new UpdateTableRequest("tableName"); +updateTableRequest.setStreamSpecification(new StreamSpecification(true, 24)); +client.updateTable(updateTableRequest); +``` +您使用SDK的UpdateTable功能,指定开启Stream并设置过期时间,即开启了Table Store增量数据导出功能。开启后,Table Store服务端就会将您的操作日志额外保存起来,每个分区有一个有序的操作日志队列,每条操作日志会在一定时间后被垃圾回收,该时间即为您指定的过期时间。 -### 实现原理 +Table Store的SDK提供了几个Stream相关的API用于读取这部分的操作日志,增量插件也是通过Table Store SDK的接口获取到增量数据,默认情况下会将增量数据转化为多个6元组的形式(pk、colName、version、colValue、opType和sequenceInfo)导入至MaxCompute中。 -首先用户使用SDK的UpdateTable功能,指定开启Stream并设置过期时间,即开启了增量功能。 +### 列模式 -开启后,TableStore服务端就会将用户的操作日志额外保存起来, -每个分区有一个有序的操作日志队列,每条操作日志会在一定时间后被垃圾回收,这个时间即用户指定的过期时间。 +在Table Store多版本模型下,表中的数据组织为行>列>版本三级的模式, 一行可以有任意列,列名并不是固定的,每一列可以含有多个版本,每个版本都有一个特定的时间戳(版本号)。 -TableStore的SDK提供了几个Stream相关的API用于将这部分操作日志读取出来,增量插件也是通过TableStore SDK的接口获取到增量数据的,并将 -增量数据转化为多个6元组的形式(pk, colName, version, colValue, opType, sequenceInfo)导入到ODPS中。 +您可以通过Table Store的API进行一系列读写操作,Table Store通过记录您最近对表的一系列写操作(或数据更改操作)来实现记录增量数据的目的,所以您也可以把增量数据看作一批操作记录。 + +Table Store支持**PutRow**、**UpdateRow**和**DeleteRow**操作: +- **PutRow**:写入一行,如果该行已存在即覆盖该行。 +- **UpdateRow**:更新一行,不更改原行的其它数据。更新包括新增或覆盖(如果对应列的对应版本已存在)一些列值、删除某一列的全部版本、删除某一列的某个版本。 +- **DeleteRow**:删除一行。 + +Table Store会根据每种操作生成对应的增量数据记录,Reader插件会读出这些记录,并导出为数据集成的数据格式。 + +同时,由于Table Store具有动态列、多版本的特性,所以Reader插件导出的一行不对应Table Store中的一行,而是对应Table Store中的一列的一个版本。即Table Store中的一行可能会导出很多行,每行包含主键值、该列的列名、该列下该版本的时间戳(版本号)、该版本的值、操作类型。如果设置isExportSequenceInfo为true,还会包括时序信息。 + +转换为数据集成的数据格式后,定义了以下四种操作类型: +- **U(UPDATE)**:写入一列的一个版本。 +- **DO(DELETE_ONE_VERSION)**:删除某一列的某个版本。 +- **DA(DELETE_ALL_VERSION)**:删除某一列的全部版本,此时需要根据主键和列名,删除对应列的全部版本。 +- **DR(DELETE_ROW)**:删除某一行,此时需要根据主键,删除该行数据。 + +假设该表有两个主键列,主键列名分别为pkName1, pkName2,示例如下。 + +| **pkName1** | **pkName2** | **columnName** | **timestamp** | **columnValue** | **opType** | +| --- | --- | --- | --- | --- | --- | +| pk1_V1 | pk2_V1 | col_a | 1441803688001 | col_val1 | U | +| pk1_V1 | pk2_V1 | col_a | 1441803688002 | col_val2 | U | +| pk1_V1 | pk2_V1 | col_b | 1441803688003 | col_val3 | U | +| pk1_V2 | pk2_V2 | col_a | 1441803688000 | — | DO | +| pk1_V2 | pk2_V2 | col_b | — | — | DA | +| pk1_V3 | pk2_V3 | — | — | — | DR | +| pk1_V3 | pk2_V3 | col_a | 1441803688005 | col_val1 | U | + +假设导出的数据如上,共7行,对应Table Store表内的3行,主键分别是(pk1_V1,pk2_V1),(pk1_V2, pk2_V2),(pk1_V3, pk2_V3): +- 对于主键为(pk1_V1,pk2_V1)的一行,包括写入col_a列的两个版本和col_b列的一个版本等操作。 +- 对于主键为(pk1_V2,pk2_V2)的一行,包括删除col_a列的一个版本和删除col_b列的全部版本等操作。 +- 对于主键为(pk1_V3,pk2_V3)的一行,包括删除整行和写入col_a列的一个版本等操作。 + +### 行模式 +#### 宽行表 +您可以通过行模式导出数据,该模式将用户每次更新的记录,抽取成行的形式导出,需要设置mode属性并配置列名。 +```json +"parameter": { + #parameter中配置下面三项配置(例如datasource、table等其它配置项照常配置)。 + "mode": "single_version_and_update_only", # 配置导出模式。 + "column":[ #按照需求添加需要导出TableStore中的列,您可以自定义设置配置个数。 + { + "name": "uid" #列名示例,可以是主键或属性列。 + }, + { + "name": "name" #列名示例,可以是主键或属性列。 + }, + ], + "isExportSequenceInfo": false, #single_version_and_update_only模式下只能是false。 +} +``` +#### 时序表 +`otsstreamreader`支持导出时序表中的增量数据,当表为时序表时,需要配置的信息如下: +```json +"parameter": { + #parameter中配置下面四项配置(例如datasource、table等其它配置项照常配置)。 + "mode": "single_version_and_update_only", # 配置导出模式。 + "isTimeseriesTable":"true", # 配置导出为时序表。 + "column":[ #按照需求添加需要导出TableStore中的列,您可以自定义设置配置个数。 + { + "name": "_m_name" #度量名称字段。 + }, + { + "name": "_data_source" #数据源字段。 + }, + { + "name": "_tags" #标签字段,将tags转换为string类型。 + }, + { + "name": "tag1_1", #标签内部字段键名称。 + "is_timeseries_tag":"true" #表明改字段为tags内部字段。 + }, + { + "name": "time" #时间戳字段。 + }, + { + "name": "name" #属性列名称。 + }, + ], + "isExportSequenceInfo": false, #single_version_and_update_only模式下只能是false。 +} +``` + +行模式导出的数据更接近于原始的行,易于后续处理,但需要注意以下问题: +- 每次导出的行是从用户每次更新的记录中抽取,每一行数据与用户的写入或更新操作一一对应。如果用户存在单独更新某些列的行为,则会出现有一些记录只有被更新的部分列,其它列为空的情况。 +- 行模式不会导出数据的版本号(即每列的时间戳),也无法进行删除操作。 + +### 数据类型转换列表 +目前OTSStream Reader支持所有的Table Store类型,其针对Table Store类型的转换列表,如下所示。 + +| **类型分类** | **OTSStream数据类型** | +| --- | --- | +| 整数类 | INTEGER | +| 浮点类 | DOUBLE | +| 字符串类 | STRING | +| 布尔类 | BOOLEAN | +| 二进制类 | BINARY | -### Reader的配置模版: - "reader": { - "name" : "otsstreamreader", - "parameter" : { - "endpoint" : "", - "accessId" : "", - "accessKey" : "", - "instanceName" : "", - //dataTable即需要导出数据的表。 - "dataTable" : "", - //statusTable是Reader用于保存状态的表,若该表不存在,Reader会自动创建该表。 - //一次离线导出任务完成后,用户不应删除该表,该表中记录的状态可用于下次导出任务中。 - "statusTable" : "TableStoreStreamReaderStatusTable", - //增量数据的时间范围(左闭右开)的左边界。 - "startTimestampMillis" : "", - //增量数据的时间范围(左闭右开)的右边界。 - "endTimestampMillis" : "", - //采云间调度只支持天级别,所以提供该配置,作用与startTimestampMillis和endTimestampMillis类似。 - "date": "", - //是否导出时序信息。 - "isExportSequenceInfo": true, - //从TableStore中读增量数据时,每次请求的最大重试次数,默认为30。 - "maxRetries" : 30 - } - } ### 参数说明 -| 名称 | 说明 | 类型 | 必选 | -| ---- | ---- | ---- | ---- | -| endpoint | TableStoreServer的Endpoint地址。| String | 是 | -| accessId | 用于访问TableStore服务的accessId。| String | 是 | -| accessKey | 用于访问TableStore服务的accessKey。 | String | 是 | -| instanceName | TableStore的实例名称。 | String | 是 | -| dataTable | 需要导出增量数据的表的名称。该表需要开启Stream,可以在建表时开启,或者使用UpdateTable接口开启。 | String | 是 | -| statusTable | Reader插件用于记录状态的表的名称,这些状态可用于减少对非目标范围内的数据的扫描,从而加快导出速度。
1. 用户不需要创建该表,只需要给出一个表名。Reader插件会尝试在用户的instance下创建该表,若该表不存在即创建新表,若该表已存在,会判断该表的Meta是否与期望一致,若不一致会抛出异常。
2. 在一次导出完成之后,用户不应删除该表,该表的状态可用于下次导出任务。
3. 该表会开启TTL,数据自动过期,因此可认为其数据量很小。
4. 针对同一个instance下的多个不同的dataTable的Reader配置,可以使用同一个statusTable,记录的状态信息互不影响。
综上,用户配置一个类似TableStoreStreamReaderStatusTable之类的名称即可,注意不要与业务相关的表重名。| String | 是 | -| startTimestampMillis | 增量数据的时间范围(左闭右开)的左边界,单位毫秒。
1. Reader插件会从statusTable中找对应startTimestampMillis的位点,从该点开始读取开始导出数据。
2. 若statusTable中找不到对应的位点,则从系统保留的增量数据的第一条开始读取,并跳过写入时间小于startTimestampMillis的数据。| Long | 否 | -| endTimestampMillis | 增量数据的时间范围(左闭右开)的右边界,单位毫秒。
1. Reader插件从startTimestampMillis位置开始导出数据后,当遇到第一条时间戳大于等于endTimestampMillis的数据时,结束导出数据,导出完成。
2. 当读取完当前全部的增量数据时,结束读取,即使未达到endTimestampMillis。 | Long | 否 | -| date | 日期格式为yyyyMMdd,如20151111,表示导出该日的数据。
若没有指定date,则必须指定startTimestampMillis和endTimestampMillis,反之也成立。 | String | 否 | -| isExportSequenceInfo | 是否导出时序信息,时序信息包含了数据的写入时间等。默认该值为false,即不导出。 | Boolean | 否 | -| maxRetries | 从TableStore中读增量数据时,每次请求的最大重试次数,默认为30,重试之间有间隔,30次重试总时间约为5分钟,一般无需更改。| Int | 否 | +| **参数** | **描述** | **是否必选** | **默认值** | +| --- |---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| --- |---------| +| **dataSource** | 数据源名称,脚本模式支持添加数据源,该配置项填写的内容必须与添加的数据源名称保持一致。 | 是 | 无 | +| **dataTable** | 导出增量数据的表的名称。该表需要开启Stream,可以在建表时开启,或者使用UpdateTable接口开启。 | 是 | 无 | +| **statusTable** | Reader插件用于记录状态的表的名称,这些状态可用于减少对非目标范围内的数据的扫描,从而加快导出速度。statusTable是Reader用于保存状态的表,如果该表不存在,Reader会自动创建该表。一次离线导出任务完成后,您无需删除该表,该表中记录的状态可用于下次导出任务中:
  • 您无需创建该表,只需要给出一个表名。Reader插件会尝试在您的instance下创建该表,如果该表不存在即创建新表。如果该表已存在,会判断该表的Meta是否与期望一致,如果不一致会抛出异常。
  • 在一次导出完成之后,您无需删除该表,该表的状态可以用于下次的导出任务。
  • 该表会开启TTL,数据自动过期,会认为其数据量很小。
  • 针对同一个instance下的多个不同的dataTable的Reader配置,可以使用同一个statusTable,记录的状态信息互不影响。您配置一个类似**TableStoreStreamReaderStatusTable**的名称即可,请注意不要与业务相关的表重名。 | 是 | 无 | +| **startTimestampMillis** | 增量数据的时间范围(左闭右开)的左边界,单位为毫秒:
  • Reader插件会从statusTable中找对应**startTimestampMillis**的位点,从该点开始读取开始导出数据。
  • 如果statusTable中找不到对应的位点,则从系统保留的增量数据的第一条开始读取,并跳过写入时间小于**startTimestampMillis**的数据。 | 否 | 无 | +| **endTimestampMillis** | 增量数据的时间范围(左闭右开)的右边界,单位为毫秒:
  • Reader插件从**startTimestampMillis**位置开始导出数据后,当遇到第一条时间戳大于等于**endTimestampMillis**的数据时,结束导出数据,导出完成。
  • 当读取完当前全部的增量数据时,即使未达到**endTimestampMillis**,也会结束读取。 | 否 | 无 | +| **date** | 日期格式为**yyyyMMdd**,例如20151111,表示导出该日的数据。如果没有指定**date**,则需要指定**startTimestampMillis**和**endTimestampMillis**或**startTimeString**和**endTimeString**,反之也成立。例如,采云间调度仅支持天级别,所以提供该配置,作用与**startTimestampMillis**和**endTimestampMillis**或**startTimeString**和**endTimeString**类似。 | 否 | 无 | +| **isExportSequenceInfo** | 是否导出时序信息,时序信息包含了数据的写入时间等。默认该值为false,即不导出。 | 否 | false | +| **maxRetries** | 从TableStore中读增量数据时,每次请求的最大重试次数,默认为30次。重试之间有间隔,重试30次的总时间约为5分钟,通常无需更改。 | 否 | 30 | +| **startTimeString** | 任务的开始时间,即增量数据的时间范围(左闭右开)的左边界,格式为**yyyymmddhh24miss**,单位为秒。 | 否 | 无 | +| **endTimeString** | 任务的结束时间,即增量数据的时间范围(左闭右开)的右边界,格式为**yyyymmddhh24miss**,单位为秒。 | 否 | 无 | +| **enableSeekIterator** | Reader插件需要先确定增量位点,然后再拉取数据,如果是经常运行的任务,插件会根据之前扫描的位点来确定位置。如果之前没运行过这个插件,将会从增量开始位置(默认增量保留7天,即7天前)开始扫描,因此当还没有扫描到设置的开始时间之后的数据时,会存在开始一段时间没有数据导出的情况,您可以在reader的配置参数里增加** "enableSeekIterator": true**的配置,帮助您加快位点定位。 | 否 | false | +| **mode** | 导出模式,设置为**single_version_and_update_only**时为行模式,默认不设置为列模式。 | 否 | 无 | +| **isTimeseriesTable** | 是否为时序表,只有在行模式,即**mode**为**single_version_and_update_only**时配置生效。 | 否 | false | -### 导出的数据格式 -首先,在TableStore多版本模型下,表中的数据组织为“行-列-版本”三级的模式, -一行可以有任意列,列名也并非固定的,每一列可以含有多个版本,每个版本都有一个特定的时间戳(版本号)。 -用户可以通过TableStore的API进行一系列读写操作, -TableStore通过记录用户最近对表的一系列写操作(或称为数据更改操作)来实现记录增量数据的目的, -所以也可以把增量数据看作一批操作记录。 -TableStore有三类数据更改操作:PutRow、UpdateRow、DeleteRow。 - - + PutRow的语义是写入一行,若该行已存在即覆盖该行。 - - + UpdateRow的语义是更新一行,对原行其他数据不做更改, - 更新可能包括新增或覆盖(若对应列的对应版本已存在)一些列值、删除某一列的全部版本、删除某一列的某个版本。 - - + DeleteRow的语义是删除一行。 - -TableStore会根据每种操作生成对应的增量数据记录,Reader插件会读出这些记录,并导出成Datax的数据格式。 - -同时,由于TableStore具有动态列、多版本的特性,所以Reader插件导出的一行不对应TableStore中的一行,而是对应TableStore中的一列的一个版本。 -即TableStore中的一行可能会导出很多行,每行包含主键值、该列的列名、该列下该版本的时间戳(版本号)、该版本的值、操作类型。若设置isExportSequenceInfo为true,还会包括时序信息。 - -转换为Datax的数据格式后,我们定义了四种操作类型,分别为: - - + U(UPDATE): 写入一列的一个版本 - - + DO(DELETE_ONE_VERSION): 删除某一列的某个版本 - - + DA(DELETE_ALL_VERSION): 删除某一列的全部版本,此时需要根据主键和列名,将对应列的全部版本删除 - - + DR(DELETE_ROW): 删除某一行,此时需要根据主键,将该行数据全部删除 - - -举例如下,假设该表有两个主键列,主键列名分别为pkName1, pkName2: - -| pkName1 | pkName2 | columnName | timestamp | columnValue | opType | -| ------- | ------- | ---------- | --------- | ----------- | ------ | -| pk1_V1 | pk2_V1 | col_a | 1441803688001 | col_val1 | U | -| pk1_V1 | pk2_V1 | col_a | 1441803688002 | col_val2 | U | -| pk1_V1 | pk2_V1 | col_b | 1441803688003 | col_val3 | U | -| pk1_V2 | pk2_V2 | col_a | 1441803688000 | | DO | -| pk1_V2 | pk2_V2 | col_b | | | DA | -| pk1_V3 | pk2_V3 | | | | DR | -| pk1_V3 | pk2_V3 | col_a | 1441803688005 | col_val1 | U | - -假设导出的数据如上,共7行,对应TableStore表内的3行,主键分别是(pk1_V1,pk2_V1), (pk1_V2, pk2_V2), (pk1_V3, pk2_V3)。 - -对于主键为(pk1_V1, pk2_V1)的一行,包含三个操作,分别是写入col_a列的两个版本和col_b列的一个版本。 - -对于主键为(pk1_V2, pk2_V2)的一行,包含两个操作,分别是删除col_a列的一个版本、删除col_b列的全部版本。 - -对于主键为(pk1_V3, pk2_V3)的一行,包含两个操作,分别是删除整行、写入col_a列的一个版本。 diff --git a/otsstreamreader/pom.xml b/otsstreamreader/pom.xml index cb4a6206..db75ba1e 100644 --- a/otsstreamreader/pom.xml +++ b/otsstreamreader/pom.xml @@ -10,19 +10,20 @@ com.alibaba.datax otsstreamreader - 0.0.1 + 0.0.1-SNAPSHOT + - org.apache.logging.log4j - log4j-api - 2.17.1 - - - - org.apache.logging.log4j - log4j-core - 2.17.1 + com.aliyun.openservices + tablestore-streamclient + 1.0.0 + + + com.aliyun.openservices + tablestore + + com.alibaba.datax @@ -33,22 +34,28 @@ slf4j-log4j12 org.slf4j - - logback-classic - ch.qos.logback - + + org.slf4j + slf4j-api + + + ch.qos.logback + logback-classic + + + com.alibaba + fastjson + 1.2.83_noneautotype + compile + com.aliyun.openservices - tablestore-streamclient - 1.0.0 + tablestore + 5.13.12 - - log4j-api - org.apache.logging.log4j - log4j-core org.apache.logging.log4j @@ -60,12 +67,6 @@ gson 2.2.4 - - com.google.guava - guava - 18.0 - test - @@ -106,6 +107,18 @@ + + + org.apache.maven.plugins + maven-surefire-plugin + 2.5 + + + **/unittest/*.java + **/functiontest/*.java + + + diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_en_US.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_en_US.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_ja_JP.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_ja_JP.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_zh_CN.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_zh_CN.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_zh_HK.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_zh_HK.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_zh_TW.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/LocalStrings_zh_TW.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReader.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReader.java index 67313467..a41b19d4 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReader.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReader.java @@ -4,17 +4,27 @@ 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.RetryUtil; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConfig; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConstants; +import com.alibaba.datax.plugin.reader.otsstreamreader.internal.core.CheckpointTimeTracker; +import com.alibaba.datax.plugin.reader.otsstreamreader.internal.model.OTSStreamJobShard; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.model.StreamJob; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.utils.GsonParser; +import com.alibaba.datax.plugin.reader.otsstreamreader.internal.utils.OTSHelper; +import com.alibaba.datax.plugin.reader.otsstreamreader.internal.utils.OTSStreamJobShardUtil; +import com.alicloud.openservices.tablestore.SyncClientInterface; import com.alicloud.openservices.tablestore.TableStoreException; import com.alicloud.openservices.tablestore.model.StreamShard; +import java.util.ArrayList; import java.util.HashSet; import java.util.List; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentSkipListSet; +import static com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConstants.*; + public class OTSStreamReader { public static class Job extends Reader.Job { @@ -46,20 +56,34 @@ public class OTSStreamReader { private OTSStreamReaderSlaveProxy proxy = new OTSStreamReaderSlaveProxy(); @Override - public void startRead(RecordSender recordSender) { - proxy.startRead(recordSender); - } - public void init() { try { OTSStreamReaderConfig config = GsonParser.jsonToConfig( (String) this.getPluginJobConf().get(OTSStreamReaderConstants.CONF)); - StreamJob streamJob = StreamJob.fromJson( - (String) this.getPluginJobConf().get(OTSStreamReaderConstants.STREAM_JOB)); List ownedShards = GsonParser.jsonToList( - (String) this.getPluginJobConf().get(OTSStreamReaderConstants.OWNED_SHARDS)); - List allShards = GsonParser.fromJson( - (String) this.getPluginJobConf().get(OTSStreamReaderConstants.ALL_SHARDS)); + (String) this.getPluginJobConf().get(OTSStreamReaderConstants.OWNED_SHARDS)); + + boolean confSimplifyEnable = this.getPluginJobConf().getBool(CONF_SIMPLIFY_ENABLE, + DEFAULT_CONF_SIMPLIFY_ENABLE_VALUE); + + StreamJob streamJob; + List allShards; + + if (confSimplifyEnable) { + //不要从conf里获取, 避免分布式模式下Job Split切分出来的Config膨胀过大 + String version = this.getPluginJobConf().getString(OTSStreamReaderConstants.VERSION); + OTSStreamJobShard otsStreamJobShard = OTSStreamJobShardUtil.getOTSStreamJobShard(config, version); + + streamJob = otsStreamJobShard.getStreamJob(); + allShards = otsStreamJobShard.getAllShards(); + + } else { + streamJob = StreamJob.fromJson( + (String) this.getPluginJobConf().get(OTSStreamReaderConstants.STREAM_JOB)); + allShards = GsonParser.fromJson( + (String) this.getPluginJobConf().get(OTSStreamReaderConstants.ALL_SHARDS)); + } + proxy.init(config, streamJob, allShards, new HashSet(ownedShards)); } catch (TableStoreException ex) { throw DataXException.asDataXException(new OTSReaderError(ex.getErrorCode(), "OTS ERROR"), ex.toString(), ex); @@ -68,6 +92,11 @@ public class OTSStreamReader { } } + @Override + public void startRead(RecordSender recordSender) { + proxy.startRead(recordSender); + } + public void destroy() { proxy.close(); } diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReaderMasterProxy.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReaderMasterProxy.java index 473e2c81..5c6a5b4b 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReaderMasterProxy.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReaderMasterProxy.java @@ -15,6 +15,8 @@ import org.slf4j.LoggerFactory; import java.util.*; +import static com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConstants.CONF_SIMPLIFY_ENABLE; + public class OTSStreamReaderMasterProxy { private OTSStreamReaderConfig conf = null; @@ -22,6 +24,7 @@ public class OTSStreamReaderMasterProxy { private StreamJob streamJob; private List allShards; + private String version; private static final Logger LOG = LoggerFactory.getLogger(OTSStreamReaderConfig.class); @@ -41,19 +44,20 @@ public class OTSStreamReaderMasterProxy { checker.checkAndCreateStatusTableIfNotExist(); // 删除StatusTable记录的对应EndTime时刻的Checkpoint信息。防止本次任务受到之前导出任务的影响。 - String streamId = OTSHelper.getStreamDetails(ots, config.getDataTable()).getStreamId(); + String streamId = OTSHelper.getStreamResponse(ots, config.getDataTable(), config.isTimeseriesTable()).getStreamId(); CheckpointTimeTracker checkpointInfoTracker = new CheckpointTimeTracker(ots, config.getStatusTable(), streamId); checkpointInfoTracker.clearAllCheckpoints(config.getEndTimestampMillis()); SyncClientInterface ots = OTSHelper.getOTSInstance(config); - allShards = OTSHelper.getOrderedShardList(ots, streamId); + allShards = OTSHelper.getOrderedShardList(ots, streamId, conf.isTimeseriesTable()); List shardIds = new ArrayList(); for (StreamShard shard : allShards) { shardIds.add(shard.getShardId()); } - String version = "" + System.currentTimeMillis() + "-" + UUID.randomUUID(); + this.version = "" + System.currentTimeMillis() + "-" + UUID.randomUUID(); + LOG.info("version is: {}", this.version); streamJob = new StreamJob(conf.getDataTable(), streamId, version, new HashSet(shardIds), conf.getStartTimestampMillis(), conf.getEndTimestampMillis()); @@ -97,8 +101,16 @@ public class OTSStreamReaderMasterProxy { Configuration configuration = Configuration.newDefault(); configuration.set(OTSStreamReaderConstants.CONF, GsonParser.configToJson(conf)); - configuration.set(OTSStreamReaderConstants.STREAM_JOB, streamJob.toJson()); - configuration.set(OTSStreamReaderConstants.ALL_SHARDS, GsonParser.toJson(allShards)); + + // Fix #39430646 [离线同步分布式]DataX OTSStreamReader插件分布式模式优化瘦身 + if (conf.isConfSimplifyEnable()) { + configuration.set(OTSStreamReaderConstants.VERSION, this.version); + configuration.set(CONF_SIMPLIFY_ENABLE, true); + } else { + configuration.set(OTSStreamReaderConstants.STREAM_JOB, streamJob.toJson()); + configuration.set(OTSStreamReaderConstants.ALL_SHARDS, GsonParser.toJson(allShards)); + } + configuration.set(OTSStreamReaderConstants.OWNED_SHARDS, GsonParser.listToJson(shardIds.subList(start, end))); configurations.add(configuration); } diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReaderSlaveProxy.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReaderSlaveProxy.java index 22035851..cdfbed28 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReaderSlaveProxy.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/OTSStreamReaderSlaveProxy.java @@ -36,16 +36,18 @@ public class OTSStreamReaderSlaveProxy { private boolean findCheckpoints; // whether find checkpoint for last job, if so, we should read from checkpoint and skip nothing. private String slaveId = UUID.randomUUID().toString(); private StreamDetails streamDetails; + private boolean enableSeekIteratorByTimestamp; public void init(final OTSStreamReaderConfig otsStreamReaderConfig, StreamJob streamJob, List allShards, Set ownedShardIds) { slaveNumber.getAndIncrement(); this.config = otsStreamReaderConfig; this.ots = OTSHelper.getOTSInstance(config); this.streamJob = streamJob; - this.streamDetails = OTSHelper.getStreamDetails(ots, this.streamJob.getTableName()); + this.streamDetails = OTSHelper.getStreamDetails(ots, this.streamJob.getTableName(),config.isTimeseriesTable()); this.checkpointInfoTracker = new CheckpointTimeTracker(ots, config.getStatusTable(), this.streamJob.getStreamId()); this.checker = new OTSStreamReaderChecker(ots, config); this.allShardsMap = OTSHelper.toShardMap(allShards); + this.enableSeekIteratorByTimestamp = otsStreamReaderConfig.getEnableSeekIteratorByTimestamp(); LOG.info("SlaveId: {}, ShardIds: {}, OwnedShards: {}.", slaveId, allShards, ownedShardIds); this.ownedShards = new HashMap(); @@ -58,12 +60,12 @@ public class OTSStreamReaderSlaveProxy { } findCheckpoints = checker.checkAndSetCheckpoints(checkpointInfoTracker, allShardsMap, streamJob, shardToCheckpointMap); - if (!findCheckpoints) { - LOG.info("Checkpoint for stream '{}' in timestamp '{}' is not found.", streamJob.getStreamId(), streamJob.getStartTimeInMillis()); + if (!findCheckpoints && !enableSeekIteratorByTimestamp) { + LOG.info("Checkpoint for stream '{}' in timestamp '{}' is not found. EnableSeekIteratorByTimestamp: {}", streamJob.getStreamId(), streamJob.getStartTimeInMillis(), this.enableSeekIteratorByTimestamp); setWithNearestCheckpoint(); } - LOG.info("Find checkpoints: {}.", findCheckpoints); + LOG.info("Find checkpoints: {}, EnableSeekIteratorByTimestamp: {}", findCheckpoints, enableSeekIteratorByTimestamp); for (Map.Entry shard : ownedShards.entrySet()) { LOG.info("Shard to process, ShardInfo: [{}], StartCheckpoint: [{}].", shard.getValue(), shardToCheckpointMap.get(shard.getKey())); } diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_en_US.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_en_US.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_ja_JP.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_ja_JP.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_zh_CN.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_zh_CN.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_zh_HK.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_zh_HK.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_zh_TW.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/LocalStrings_zh_TW.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/OTSStreamReaderConfig.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/OTSStreamReaderConfig.java index c89d7a37..bef910e3 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/OTSStreamReaderConfig.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/OTSStreamReaderConfig.java @@ -13,6 +13,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import static com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConstants.CONF_SIMPLIFY_ENABLE; +import static com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConstants.DEFAULT_CONF_SIMPLIFY_ENABLE_VALUE; + public class OTSStreamReaderConfig { private static final Logger LOG = LoggerFactory.getLogger(OTSStreamReaderConfig.class); @@ -33,6 +36,11 @@ public class OTSStreamReaderConfig { private static final String KEY_MODE = "mode"; private static final String KEY_COLUMN = "column"; private static final String KEY_THREAD_NUM = "threadNum"; + private static final String KEY_ENABLE_TABLE_GROUP_SUPPORT = "enableTableGroupSupport"; + + private static final String ENABLE_SEEK_SHARD_ITERATOR = "enableSeekIterator"; + + private static final String IS_TIMESERIES_TABLE = "isTimeseriesTable"; private static final int DEFAULT_MAX_RETRIES = 30; private static final long DEFAULT_SLAVE_LOOP_INTERVAL = 10 * TimeUtils.SECOND_IN_MILLIS; @@ -51,12 +59,19 @@ public class OTSStreamReaderConfig { private int threadNum = 32; private long slaveLoopInterval = DEFAULT_SLAVE_LOOP_INTERVAL; private long slaveLoggingStatusInterval = DEFAULT_SLAVE_LOGGING_STATUS_INTERVAL; + private boolean enableSeekIteratorByTimestamp; + private boolean enableTableGroupSupport; private Mode mode; private List columns; + private List columnsIsTimeseriesTags; private transient SyncClientInterface otsForTest; + private boolean confSimplifyEnable; + + private boolean isTimeseriesTable; + public String getEndpoint() { return endpoint; } @@ -129,6 +144,22 @@ public class OTSStreamReaderConfig { this.isExportSequenceInfo = isExportSequenceInfo; } + public boolean isEnableTableGroupSupport() { + return enableTableGroupSupport; + } + + public void setEnableTableGroupSupport(boolean enableTableGroupSupport) { + this.enableTableGroupSupport = enableTableGroupSupport; + } + + public boolean getEnableSeekIteratorByTimestamp() { + return enableSeekIteratorByTimestamp; + } + + public void setEnableSeekIteratorByTimestamp(boolean enableSeekIteratorByTimestamp) { + this.enableSeekIteratorByTimestamp = enableSeekIteratorByTimestamp; + } + public Mode getMode() { return mode; } @@ -145,24 +176,62 @@ public class OTSStreamReaderConfig { this.columns = columns; } + public List getColumnsIsTimeseriesTags() { + return columnsIsTimeseriesTags; + } + + public void setColumnsIsTimeseriesTags(List columnsIsTimeseriesTags) { + this.columnsIsTimeseriesTags = columnsIsTimeseriesTags; + } + + public boolean isTimeseriesTable() { + return isTimeseriesTable; + } + + public void setTimeseriesTable(boolean timeseriesTable) { + isTimeseriesTable = timeseriesTable; + } + private static void parseConfigForSingleVersionAndUpdateOnlyMode(OTSStreamReaderConfig config, Configuration param) { + try { + Boolean isTimeseriesTable = param.getBool(IS_TIMESERIES_TABLE); + if (isTimeseriesTable != null) { + config.setTimeseriesTable(isTimeseriesTable); + } else { + config.setTimeseriesTable(false); + } + } catch (RuntimeException ex) { + throw new OTSStreamReaderException("Parse timeseries stream settings fail, please check your config.", ex); + } + try { List values = param.getList(KEY_COLUMN); if (values == null) { config.setColumns(new ArrayList()); + config.setColumnsIsTimeseriesTags(new ArrayList()); return; } List columns = new ArrayList(); + List columnsIsTimeseriesTags = new ArrayList(); + Boolean isTimeseriesTable = config.isTimeseriesTable(); + for (Object item : values) { if (item instanceof Map) { String columnName = (String) ((Map) item).get("name"); columns.add(columnName); + + boolean columnsIsTimeseriesTag = false; + if (isTimeseriesTable && Boolean.parseBoolean((String) ((Map) item).getOrDefault("is_timeseries_tag", "false"))) { + columnsIsTimeseriesTag = true; + } + columnsIsTimeseriesTags.add(columnsIsTimeseriesTag); } else { throw new IllegalArgumentException("The item of column must be map object, please check your input."); } } config.setColumns(columns); + config.setColumnsIsTimeseriesTags(columnsIsTimeseriesTags); } catch (RuntimeException ex) { throw new OTSStreamReaderException("Parse column fail, please check your config.", ex); } @@ -178,56 +247,59 @@ public class OTSStreamReaderConfig { config.setDataTable(ParamChecker.checkStringAndGet(param, KEY_DATA_TABLE_NAME, true)); config.setStatusTable(ParamChecker.checkStringAndGet(param, KEY_STATUS_TABLE_NAME, true)); config.setIsExportSequenceInfo(param.getBool(KEY_IS_EXPORT_SEQUENCE_INFO, false)); + config.setEnableSeekIteratorByTimestamp(param.getBool(ENABLE_SEEK_SHARD_ITERATOR, false)); + config.setConfSimplifyEnable(param.getBool(CONF_SIMPLIFY_ENABLE, DEFAULT_CONF_SIMPLIFY_ENABLE_VALUE)); + config.setEnableTableGroupSupport(param.getBool(KEY_ENABLE_TABLE_GROUP_SUPPORT, false)); if (param.getInt(KEY_THREAD_NUM) != null) { config.setThreadNum(param.getInt(KEY_THREAD_NUM)); } if (param.getString(KEY_DATE) == null && - (param.getLong(KEY_START_TIMESTAMP_MILLIS) == null || param.getLong(KEY_END_TIMESTAMP_MILLIS) == null) && + (param.getLong(KEY_START_TIMESTAMP_MILLIS) == null || param.getLong(KEY_END_TIMESTAMP_MILLIS) == null) && (param.getLong(KEY_START_TIME_STRING) == null || param.getLong(KEY_END_TIME_STRING) == null)) { throw new OTSStreamReaderException("Must set date or time range millis or time range string, please check your config."); } - + if (param.get(KEY_DATE) != null && (param.getLong(KEY_START_TIMESTAMP_MILLIS) != null || param.getLong(KEY_END_TIMESTAMP_MILLIS) != null) && (param.getLong(KEY_START_TIME_STRING) != null || param.getLong(KEY_END_TIME_STRING) != null)) { throw new OTSStreamReaderException("Can't set date and time range millis and time range string, please check your config."); } - + if (param.get(KEY_DATE) != null && (param.getLong(KEY_START_TIMESTAMP_MILLIS) != null || param.getLong(KEY_END_TIMESTAMP_MILLIS) != null)) { throw new OTSStreamReaderException("Can't set date and time range both, please check your config."); } - + if (param.get(KEY_DATE) != null && (param.getLong(KEY_START_TIME_STRING) != null || param.getLong(KEY_END_TIME_STRING) != null)) { throw new OTSStreamReaderException("Can't set date and time range string both, please check your config."); } - - if ((param.getLong(KEY_START_TIMESTAMP_MILLIS) != null || param.getLong(KEY_END_TIMESTAMP_MILLIS) != null)&& + + if ((param.getLong(KEY_START_TIMESTAMP_MILLIS) != null || param.getLong(KEY_END_TIMESTAMP_MILLIS) != null) && (param.getLong(KEY_START_TIME_STRING) != null || param.getLong(KEY_END_TIME_STRING) != null)) { - throw new OTSStreamReaderException("Can't set time range millis and time range string both, please check your config."); + throw new OTSStreamReaderException("Can't set time range millis and time range string both, expect timestamp like '1516010400000'."); } if (param.getString(KEY_START_TIME_STRING) != null && param.getString(KEY_END_TIME_STRING) != null) { - String startTime=ParamChecker.checkStringAndGet(param, KEY_START_TIME_STRING, true); - String endTime=ParamChecker.checkStringAndGet(param, KEY_END_TIME_STRING, true); + String startTime = ParamChecker.checkStringAndGet(param, KEY_START_TIME_STRING, true); + String endTime = ParamChecker.checkStringAndGet(param, KEY_END_TIME_STRING, true); try { long startTimestampMillis = TimeUtils.parseTimeStringToTimestampMillis(startTime); config.setStartTimestampMillis(startTimestampMillis); } catch (Exception ex) { - throw new OTSStreamReaderException("Can't parse startTimeString: " + startTime); + throw new OTSStreamReaderException("Can't parse startTimeString: " + startTime + ", expect format date like '201801151612'."); } try { long endTimestampMillis = TimeUtils.parseTimeStringToTimestampMillis(endTime); config.setEndTimestampMillis(endTimestampMillis); } catch (Exception ex) { - throw new OTSStreamReaderException("Can't parse startTimeString: " + startTime); - } - - }else if (param.getString(KEY_DATE) == null) { + throw new OTSStreamReaderException("Can't parse endTimeString: " + endTime + ", expect format date like '201801151612'."); + } + + } else if (param.getString(KEY_DATE) == null) { config.setStartTimestampMillis(param.getLong(KEY_START_TIMESTAMP_MILLIS)); config.setEndTimestampMillis(param.getLong(KEY_END_TIMESTAMP_MILLIS)); } else { @@ -241,8 +313,6 @@ public class OTSStreamReaderConfig { } } - - if (config.getStartTimestampMillis() >= config.getEndTimestampMillis()) { throw new OTSStreamReaderException("EndTimestamp must be larger than startTimestamp."); @@ -262,15 +332,21 @@ public class OTSStreamReaderConfig { config.setMode(Mode.MULTI_VERSION); List values = param.getList(KEY_COLUMN); if (values != null) { - throw new OTSStreamReaderException("The multi version mode doesn't support setting columns."); + LOG.warn("The multi version mode doesn't support setting columns, column config will ignore."); + } + Boolean isTimeseriesTable = param.getBool(IS_TIMESERIES_TABLE); + if (isTimeseriesTable != null) { + LOG.warn("The multi version mode doesn't support setting Timeseries stream, stream config will ignore."); } } - LOG.info("endpoint: {}, accessId: {}, accessKey: {}, instanceName: {}, dataTableName: {}, statusTableName: {}," + - " isExportSequenceInfo: {}, startTimestampMillis: {}, endTimestampMillis:{}, maxRetries:{}.", config.getEndpoint(), + LOG.info("endpoint: {}, accessKeyId: {}, accessKeySecret: {}, instanceName: {}, dataTableName: {}, statusTableName: {}," + + " isExportSequenceInfo: {}, startTimestampMillis: {}, endTimestampMillis:{}, maxRetries:{}, enableSeekIteratorByTimestamp: {}, " + + "confSimplifyEnable: {}, isTimeseriesTable: {}.", config.getEndpoint(), config.getAccessId(), config.getAccessKey(), config.getInstanceName(), config.getDataTable(), config.getStatusTable(), config.isExportSequenceInfo(), config.getStartTimestampMillis(), - config.getEndTimestampMillis(), config.getMaxRetries()); + config.getEndTimestampMillis(), config.getMaxRetries(), config.getEnableSeekIteratorByTimestamp(), + config.isConfSimplifyEnable(), config.isTimeseriesTable()); return config; } @@ -282,7 +358,6 @@ public class OTSStreamReaderConfig { public SyncClientInterface getOtsForTest() { return otsForTest; } - /** * test use * @param otsForTest @@ -290,36 +365,36 @@ public class OTSStreamReaderConfig { public void setOtsForTest(SyncClientInterface otsForTest) { this.otsForTest = otsForTest; } - public int getMaxRetries() { return maxRetries; } - public void setMaxRetries(int maxRetries) { this.maxRetries = maxRetries; } - public int getThreadNum() { return threadNum; } - public void setSlaveLoopInterval(long slaveLoopInterval) { this.slaveLoopInterval = slaveLoopInterval; } - public void setSlaveLoggingStatusInterval(long slaveLoggingStatusInterval) { this.slaveLoggingStatusInterval = slaveLoggingStatusInterval; } - public long getSlaveLoopInterval() { return slaveLoopInterval; } - public long getSlaveLoggingStatusInterval() { return slaveLoggingStatusInterval; } - public void setThreadNum(int threadNum) { this.threadNum = threadNum; } + + public boolean isConfSimplifyEnable() { + return confSimplifyEnable; + } + + public void setConfSimplifyEnable(boolean confSimplifyEnable) { + this.confSimplifyEnable = confSimplifyEnable; + } } diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/OTSStreamReaderConstants.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/OTSStreamReaderConstants.java index 19db148a..c95fdf2c 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/OTSStreamReaderConstants.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/config/OTSStreamReaderConstants.java @@ -21,7 +21,20 @@ public class OTSStreamReaderConstants { public static final String STREAM_JOB = "STREAM_JOB"; public static final String OWNED_SHARDS = "OWNED_SHARDS"; public static final String ALL_SHARDS = "ALL_SHARDS"; + public static final String VERSION = "STREAM_VERSION"; + /** + * 是否开启OTS分布式模式降低Job Split阶段切分的Task Conf大小启动优化, + * 新增该参数的目的是为了保证DataX灰度过程,避免因为OTS分布式任务运行部分子进程运行在老版本、部分运行在新版本导致任务失败问题, + * 当DataX版本集群粒度已全量升级到新版本以后,再开启该参数为"true",默认值是"false" + */ + public static final String CONF_SIMPLIFY_ENABLE = "confSimplifyEnable"; + + public static final Integer RETRY_TIMES = 3; + + public static final Long DEFAULT_SLEEP_TIME_IN_MILLS = 500l; + + public static final boolean DEFAULT_CONF_SIMPLIFY_ENABLE_VALUE = false; static { String beforeOffsetMillis = System.getProperty("BEFORE_OFFSET_TIME_MILLIS"); diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_en_US.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_en_US.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_ja_JP.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_ja_JP.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_zh_CN.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_zh_CN.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_zh_HK.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_zh_HK.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_zh_TW.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/LocalStrings_zh_TW.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/OTSStreamReaderChecker.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/OTSStreamReaderChecker.java index 086d0159..560dcb7c 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/OTSStreamReaderChecker.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/OTSStreamReaderChecker.java @@ -40,11 +40,11 @@ public class OTSStreamReaderChecker { * 为了避免时间误差影响,允许导出的范围为: [now - expirationTime + beforeOffset, now - afterOffset] */ public void checkStreamEnabledAndTimeRangeOK() { - boolean exists = OTSHelper.checkTableExists(ots, config.getDataTable()); + boolean exists = OTSHelper.checkTableExists(ots, config.getDataTable(), config.isTimeseriesTable()); if (!exists) { throw new OTSStreamReaderException("The data table is not exist."); } - StreamDetails streamDetails = OTSHelper.getStreamDetails(ots, config.getDataTable()); + StreamDetails streamDetails = OTSHelper.getStreamDetails(ots, config.getDataTable(), config.isTimeseriesTable()); if (streamDetails == null || !streamDetails.isEnableStream()) { throw new OTSStreamReaderException("The stream of data table is not enabled."); } @@ -81,7 +81,7 @@ public class OTSStreamReaderChecker { * 检查statusTable是否存在,如果不存在就创建statusTable,并等待表ready。 */ public void checkAndCreateStatusTableIfNotExist() { - boolean tableExist = OTSHelper.checkTableExists(ots, config.getStatusTable()); + boolean tableExist = OTSHelper.checkTableExists(ots, config.getStatusTable(), false); if (tableExist) { DescribeTableResponse describeTableResult = OTSHelper.describeTable(ots, config.getStatusTable()); checkTableMetaOfStatusTable(describeTableResult.getTableMeta()); @@ -135,23 +135,6 @@ public class OTSStreamReaderChecker { } } - // 检查是否有丢失的shard - for (Map.Entry entry : allShardsMap.entrySet()) { - StreamShard shard = entry.getValue(); - String parentId = shard.getParentId(); - // shard不在本次任务中,且shard也不在上一次任务中 - if (parentId != null && !allShardsMap.containsKey(parentId) && !allCheckpoints.containsKey(parentId)) { - LOG.error("Shard is lost: {}.", shard); - throw new OTSStreamReaderException("Can't find checkpoint for shard: " + parentId); - } - - parentId = shard.getParentSiblingId(); - if (parentId != null && !allShardsMap.containsKey(parentId) && !allCheckpoints.containsKey(parentId)) { - LOG.error("Shard is lost: {}.", shard); - throw new OTSStreamReaderException("Can't find checkpoint for shard: " + parentId); - } - } - return true; } } diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/RecordProcessor.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/RecordProcessor.java index ba17bd9c..feb99722 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/RecordProcessor.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/RecordProcessor.java @@ -1,5 +1,6 @@ package com.alibaba.datax.plugin.reader.otsstreamreader.internal.core; +import com.alibaba.datax.common.exception.DataXException; import com.alibaba.datax.common.plugin.RecordSender; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.Mode; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConfig; @@ -48,6 +49,9 @@ public class RecordProcessor implements Runnable { private AtomicBoolean stop; private AtomicLong sendRecordCount; + //enable seek shardIterator by timestamp + private boolean enableSeekShardIteratorByTimestamp; + public enum State { READY, // initialized but not start RUNNING, // start to read and process records @@ -78,6 +82,7 @@ public class RecordProcessor implements Runnable { this.recordSender = recordSender; this.isExportSequenceInfo = config.isExportSequenceInfo(); this.lastRecordCheckpointTime = 0; + this.enableSeekShardIteratorByTimestamp = config.getEnableSeekIteratorByTimestamp(); // set init state startTime = 0; @@ -107,22 +112,31 @@ public class RecordProcessor implements Runnable { if (readerConfig.getMode().equals(Mode.MULTI_VERSION)) { this.otsStreamRecordSender = new MultiVerModeRecordSender(recordSender, shard.getShardId(), isExportSequenceInfo); } else if (readerConfig.getMode().equals(Mode.SINGLE_VERSION_AND_UPDATE_ONLY)) { - this.otsStreamRecordSender = new SingleVerAndUpOnlyModeRecordSender(recordSender, shard.getShardId(), isExportSequenceInfo, readerConfig.getColumns()); + this.otsStreamRecordSender = new SingleVerAndUpOnlyModeRecordSender(recordSender, shard.getShardId(), isExportSequenceInfo, readerConfig.getColumns(), readerConfig.getColumnsIsTimeseriesTags()); } else { throw new OTSStreamReaderException("Internal Error. Unhandled Mode: " + readerConfig.getMode()); } if (startCheckpoint.getCheckpoint().equals(CheckpointPosition.TRIM_HORIZON)) { lastShardIterator = null; - nextShardIterator = ots.getShardIterator(new GetShardIteratorRequest(stream.getStreamId(), shard.getShardId())).getShardIterator(); + if (enableSeekShardIteratorByTimestamp) { + long beginTimeStamp = startTimestampMillis - 10 * 60 * 1000; + if (beginTimeStamp > 0) { + nextShardIterator = getShardIteratorWithBeginTime((startTimestampMillis - 10 * 60 * 1000) * 1000); + } else { + nextShardIterator = ots.getShardIterator(new GetShardIteratorRequest(stream.getStreamId(), shard.getShardId())).getShardIterator(); + } + } else { + nextShardIterator = ots.getShardIterator(new GetShardIteratorRequest(stream.getStreamId(), shard.getShardId())).getShardIterator(); + } skipCount = startCheckpoint.getSkipCount(); } else { lastShardIterator = null; nextShardIterator = startCheckpoint.getCheckpoint(); skipCount = startCheckpoint.getSkipCount(); } - LOG.info("Initialize record processor. Mode: {}, StartCheckpoint: [{}], ShardId: {}, ShardIterator: {}, SkipCount: {}.", - readerConfig.getMode(), startCheckpoint, shard.getShardId(), nextShardIterator, skipCount); + LOG.info("Initialize record processor. Mode: {}, StartCheckpoint: [{}], ShardId: {}, ShardIterator: {}, SkipCount: {}, enableSeekShardIteratorByTimestamp: {}, startTimestamp: {}.", + readerConfig.getMode(), startCheckpoint, shard.getShardId(), nextShardIterator, skipCount, enableSeekShardIteratorByTimestamp, startTimestampMillis); } private long getTimestamp(StreamRecord record) { @@ -181,15 +195,32 @@ public class RecordProcessor implements Runnable { * * @param records * @param nextShardIterator + * @param mayMoreRecord * @return */ - boolean process(List records, String nextShardIterator) { + boolean process(List records, String nextShardIterator, Boolean mayMoreRecord) { if (records.isEmpty() && nextShardIterator != null) { - LOG.info("ProcessFinished: No more data in shard, shardId: {}.", shard.getShardId()); - ShardCheckpoint checkpoint = new ShardCheckpoint(shard.getShardId(), stream.getVersion(), nextShardIterator, 0); - checkpointTimeTracker.writeCheckpoint(endTimestampMillis, checkpoint, sendRecordCount.get()); - checkpointTimeTracker.setShardTimeCheckpoint(shard.getShardId(), endTimestampMillis, nextShardIterator); - return true; + // 没有读到更多数据 + if (!readerConfig.isEnableTableGroupSupport()) { + LOG.info("ProcessFinished: No more data in shard, shardId: {}.", shard.getShardId()); + ShardCheckpoint checkpoint = new ShardCheckpoint(shard.getShardId(), stream.getVersion(), nextShardIterator, 0); + checkpointTimeTracker.writeCheckpoint(endTimestampMillis, checkpoint, sendRecordCount.get()); + checkpointTimeTracker.setShardTimeCheckpoint(shard.getShardId(), endTimestampMillis, nextShardIterator); + return true; + } else { + if (mayMoreRecord == null) { + LOG.error("mayMoreRecord can not be null when tablegroup is true"); + throw DataXException.asDataXException("mayMoreRecord can not be null when tablegroup is true"); + } else if (mayMoreRecord) { + return false; + } else { + LOG.info("ProcessFinished: No more data in shard, shardId: {}.", shard.getShardId()); + ShardCheckpoint checkpoint = new ShardCheckpoint(shard.getShardId(), stream.getVersion(), nextShardIterator, 0); + checkpointTimeTracker.writeCheckpoint(endTimestampMillis, checkpoint, sendRecordCount.get()); + checkpointTimeTracker.setShardTimeCheckpoint(shard.getShardId(), endTimestampMillis, nextShardIterator); + return true; + } + } } int size = records.size(); @@ -212,17 +243,19 @@ public class RecordProcessor implements Runnable { continue; } shouldSkip = false; - if (skipCount > 0) { - LOG.debug("Skip record. Timestamp: {}, SkipCount: {}.", timestamp, skipCount); - skipCount -= 1; - continue; - } LOG.debug("Send record. Timestamp: {}.", timestamp); sendRecord(records.get(i)); } else { LOG.info("ProcessFinished: Record in shard reach boundary of endTime, shardId: {}. Timestamp: {}, EndTime: {}", shard.getShardId(), timestamp, endTimestampMillis); - ShardCheckpoint checkpoint = new ShardCheckpoint(shard.getShardId(), stream.getVersion(), lastShardIterator, i); + + String newIterator = lastShardIterator; + if (i > 0) { + newIterator = GetStreamRecordWithLimitRowCount(lastShardIterator, i); + } + + ShardCheckpoint checkpoint = new ShardCheckpoint(shard.getShardId(), stream.getVersion(), newIterator, 0); + checkpointTimeTracker.writeCheckpoint(endTimestampMillis, checkpoint, sendRecordCount.get()); return true; } @@ -240,14 +273,35 @@ public class RecordProcessor implements Runnable { private boolean readAndProcessRecords() { LOG.debug("Read and process records. ShardId: {}, ShardIterator: {}.", shard.getShardId(), nextShardIterator); + if (enableSeekShardIteratorByTimestamp && nextShardIterator == null) { + LOG.info("ProcessFinished: Shard has reach to end, shardId: {}.", shard.getShardId()); + ShardCheckpoint checkpoint = new ShardCheckpoint(shard.getShardId(), stream.getVersion(), CheckpointPosition.SHARD_END, 0); + checkpointTimeTracker.writeCheckpoint(endTimestampMillis, checkpoint, sendRecordCount.get()); + return true; + } + GetStreamRecordRequest request = new GetStreamRecordRequest(nextShardIterator); + if (readerConfig.isEnableTableGroupSupport()) { + request.setTableName(stream.getTableName()); + } + if (readerConfig.isTimeseriesTable()){ + request.setParseInTimeseriesDataFormat(true); + } GetStreamRecordResponse response = ots.getStreamRecord(request); lastShardIterator = nextShardIterator; nextShardIterator = response.getNextShardIterator(); - return processRecords(response.getRecords(), nextShardIterator); + return processRecords(response.getRecords(), nextShardIterator, response.getMayMoreRecord()); } - public boolean processRecords(List records, String nextShardIterator) { + private String GetStreamRecordWithLimitRowCount(String beginIterator, int expectedRowCount) { + LOG.debug("Read and process records. ShardId: {}, ShardIterator: {}, expectedRowCount: {}..", shard.getShardId(), beginIterator, expectedRowCount); + GetStreamRecordRequest request = new GetStreamRecordRequest(beginIterator); + request.setLimit(expectedRowCount); + GetStreamRecordResponse response = ots.getStreamRecord(request); + return response.getNextShardIterator(); + } + + public boolean processRecords(List records, String nextShardIterator, Boolean mayMoreRecord) { long startTime = System.currentTimeMillis(); if (records.isEmpty()) { @@ -256,7 +310,7 @@ public class RecordProcessor implements Runnable { LOG.debug("StartProcessRecords: size: {}, recordTime: {}.", records.size(), getTimestamp(records.get(0))); } - if (process(records, nextShardIterator)) { + if (process(records, nextShardIterator, mayMoreRecord)) { return true; } @@ -264,4 +318,27 @@ public class RecordProcessor implements Runnable { shard.getShardId(), System.currentTimeMillis() - startTime, records.size(), nextShardIterator); return false; } -} + + private String getShardIteratorWithBeginTime(long timestamp){ + LOG.info("Begin to seek shard iterator with timestamp, shardId: {}, timestamp: {}.", shard.getShardId(), timestamp); + GetShardIteratorRequest getShardIteratorRequest = new GetShardIteratorRequest(stream.getStreamId(), shard.getShardId()); + getShardIteratorRequest.setTimestamp(timestamp); + + GetShardIteratorResponse response = ots.getShardIterator(getShardIteratorRequest); + String nextToken = response.getNextToken(); + + if (nextToken == null) { + return response.getShardIterator(); + } + + while (nextToken != null) { + getShardIteratorRequest = new GetShardIteratorRequest(stream.getStreamId(), shard.getShardId()); + getShardIteratorRequest.setTimestamp(timestamp); + getShardIteratorRequest.setToken(nextToken); + + response = ots.getShardIterator(getShardIteratorRequest); + nextToken = response.getNextToken(); + } + return response.getShardIterator(); + } +} \ No newline at end of file diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/SingleVerAndUpOnlyModeRecordSender.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/SingleVerAndUpOnlyModeRecordSender.java index 1cc32bad..d962af76 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/SingleVerAndUpOnlyModeRecordSender.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/core/SingleVerAndUpOnlyModeRecordSender.java @@ -5,6 +5,7 @@ import com.alibaba.datax.common.element.StringColumn; import com.alibaba.datax.common.plugin.RecordSender; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.OTSStreamReaderException; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.utils.ColumnValueTransformHelper; +import com.alicloud.openservices.tablestore.core.protocol.timeseries.TimeseriesResponseFactory; import com.alicloud.openservices.tablestore.model.*; import java.util.HashMap; @@ -17,21 +18,23 @@ import java.util.Map; * | pk1 | pk2 | col1 | col2 | col3 | sequence id | * | --- | --- | ---- | ---- | ---- | ----------- | * | a | b | c1 | null | null | 001 | - * + *

    * 注意:删除整行,删除某列(某个版本或所有),这些增量信息都会被忽略。 */ public class SingleVerAndUpOnlyModeRecordSender implements IStreamRecordSender { private final RecordSender dataxRecordSender; - private String shardId; private final boolean isExportSequenceInfo; + private String shardId; private List columnNames; + private List columnsIsTimeseriesTags; - public SingleVerAndUpOnlyModeRecordSender(RecordSender dataxRecordSender, String shardId, boolean isExportSequenceInfo, List columnNames) { + public SingleVerAndUpOnlyModeRecordSender(RecordSender dataxRecordSender, String shardId, boolean isExportSequenceInfo, List columnNames, List columnsIsTimeseriesTags) { this.dataxRecordSender = dataxRecordSender; this.shardId = shardId; this.isExportSequenceInfo = isExportSequenceInfo; this.columnNames = columnNames; + this.columnsIsTimeseriesTags = columnsIsTimeseriesTags; } @Override @@ -57,25 +60,49 @@ public class SingleVerAndUpOnlyModeRecordSender implements IStreamRecordSender { map.put(pkCol.getName(), pkCol.getValue()); } + /** + * 将时序数据中tags字段的字符串转化为Map + */ + Map tagsMap = new HashMap<>(); + if (columnsIsTimeseriesTags != null && columnsIsTimeseriesTags.contains(true)) { + try{ + tagsMap = TimeseriesResponseFactory.parseTagsOrAttrs(String.valueOf(map.get("_tags"))); + } + catch (Exception ex){ + throw new OTSStreamReaderException("Parse \"_tags\" fail, please check your config.", ex); + } + + } + for (RecordColumn recordColumn : columns) { if (recordColumn.getColumnType().equals(RecordColumn.ColumnType.PUT)) { map.put(recordColumn.getColumn().getName(), recordColumn.getColumn().getValue()); } } - boolean findColumn = false; + boolean findColumn = false; - for (String colName : columnNames) { - Object value = map.get(colName); - if (value != null) { - findColumn = true; - if (value instanceof ColumnValue) { - line.addColumn(ColumnValueTransformHelper.otsColumnValueToDataxColumn((ColumnValue) value)); + for (int i = 0; i < columnNames.size(); i++) { + if (columnsIsTimeseriesTags != null && columnsIsTimeseriesTags.get(i)) { + String value = tagsMap.get(columnNames.get(i)); + if (value != null) { + findColumn = true; + line.addColumn(new StringColumn(value)); } else { - line.addColumn(ColumnValueTransformHelper.otsPrimaryKeyValueToDataxColumn((PrimaryKeyValue) value)); + line.addColumn(new StringColumn(null)); } } else { - line.addColumn(new StringColumn(null)); + Object value = map.get(columnNames.get(i)); + if (value != null) { + findColumn = true; + if (value instanceof ColumnValue) { + line.addColumn(ColumnValueTransformHelper.otsColumnValueToDataxColumn((ColumnValue) value)); + } else { + line.addColumn(ColumnValueTransformHelper.otsPrimaryKeyValueToDataxColumn((PrimaryKeyValue) value)); + } + } else { + line.addColumn(new StringColumn(null)); + } } } diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_en_US.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_en_US.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_ja_JP.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_ja_JP.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_zh_CN.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_zh_CN.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_zh_HK.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_zh_HK.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_zh_TW.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/LocalStrings_zh_TW.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/OTSStreamJobShard.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/OTSStreamJobShard.java new file mode 100644 index 00000000..d5d5f971 --- /dev/null +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/model/OTSStreamJobShard.java @@ -0,0 +1,42 @@ +package com.alibaba.datax.plugin.reader.otsstreamreader.internal.model; + +import com.alicloud.openservices.tablestore.model.StreamShard; + +import java.util.List; + +/** + * OTS streamJob & allShards model + * + * @author mingya.wmy (云时) + */ +public class OTSStreamJobShard { + + private StreamJob streamJob; + + private List allShards; + + public OTSStreamJobShard() { + } + + public OTSStreamJobShard(StreamJob streamJob, List allShards) { + this.streamJob = streamJob; + this.allShards = allShards; + } + + public StreamJob getStreamJob() { + return streamJob; + } + + public void setStreamJob(StreamJob streamJob) { + this.streamJob = streamJob; + } + + public List getAllShards() { + return allShards; + } + + public void setAllShards(List allShards) { + this.allShards = allShards; + } + +} diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_en_US.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_en_US.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_ja_JP.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_ja_JP.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_zh_CN.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_zh_CN.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_zh_HK.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_zh_HK.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_zh_TW.properties b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/LocalStrings_zh_TW.properties new file mode 100644 index 00000000..e69de29b diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/OTSHelper.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/OTSHelper.java index 79b6c1d7..24ea732a 100644 --- a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/OTSHelper.java +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/OTSHelper.java @@ -2,11 +2,19 @@ package com.alibaba.datax.plugin.reader.otsstreamreader.internal.utils; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSRetryStrategyForStreamReader; import com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConfig; +import com.alicloud.openservices.tablestore.ClientConfiguration; +import com.alicloud.openservices.tablestore.SyncClient; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.TableStoreException; import com.alicloud.openservices.tablestore.model.*; -import com.alicloud.openservices.tablestore.*; +import com.alicloud.openservices.tablestore.model.timeseries.DescribeTimeseriesTableRequest; +import com.alicloud.openservices.tablestore.model.timeseries.DescribeTimeseriesTableResponse; import com.aliyun.openservices.ots.internal.streamclient.utils.TimeUtils; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; public class OTSHelper { @@ -35,14 +43,55 @@ public class OTSHelper { return ots; } + public static DescribeStreamResponse getStreamResponse(SyncClientInterface ots, String tableName, boolean isTimeseriesTable) { + /** + * 对于时序表,需要通过listStream&describeStream两次交互,获取streamID与expirationTime + */ + ListStreamRequest request = new ListStreamRequest(tableName); + ListStreamResponse response = ots.listStream(request); + String streamID = null; + for (Stream stream : response.getStreams()) { + if (stream.getTableName().equals(tableName)) { + streamID = stream.getStreamId(); + break; + } + } + if (streamID == null) { + throw new RuntimeException(String.format("Did not get any stream from table : (\"%s\") .", tableName)); + } + DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest(streamID); + if (isTimeseriesTable) { + describeStreamRequest.setSupportTimeseriesTable(true); + } + DescribeStreamResponse result = ots.describeStream(describeStreamRequest); + if(isTimeseriesTable && !result.isTimeseriesDataTable()){ + throw new RuntimeException(String.format("The table [%s] is not timeseries data table, please remove the config: {isTimeseriesTable : true}.", tableName)); + } + return result; + } + public static StreamDetails getStreamDetails(SyncClientInterface ots, String tableName) { DescribeTableRequest describeTableRequest = new DescribeTableRequest(tableName); DescribeTableResponse result = ots.describeTable(describeTableRequest); return result.getStreamDetails(); } - public static List getOrderedShardList(SyncClientInterface ots, String streamId) { + public static StreamDetails getStreamDetails(SyncClientInterface ots, String tableName, boolean isTimeseriesTable) { + if (!isTimeseriesTable) { + return getStreamDetails(ots, tableName); + } else { + DescribeStreamResponse result = getStreamResponse(ots, tableName, isTimeseriesTable); + //TODO:时序表无法直接获取StreamDetails,需要手动构建。 + // 其中lastEnableTime字段暂时无法获取 + return new StreamDetails(true, result.getStreamId(), result.getExpirationTime(), 0); + } + } + + public static List getOrderedShardList(SyncClientInterface ots, String streamId, boolean isTimeseriesTable) { DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest(streamId); + if (isTimeseriesTable) { + describeStreamRequest.setSupportTimeseriesTable(true); + } DescribeStreamResponse describeStreamResult = ots.describeStream(describeStreamRequest); List shardList = new ArrayList(); shardList.addAll(describeStreamResult.getShards()); @@ -54,10 +103,15 @@ public class OTSHelper { return shardList; } - public static boolean checkTableExists(SyncClientInterface ots, String tableName) { + public static boolean checkTableExists(SyncClientInterface ots, String tableName, boolean isTimeseriesTable) { boolean exist = false; try { - describeTable(ots, tableName); + if (isTimeseriesTable) { + describeTimeseriesTable(ots, tableName); + } else { + describeTable(ots, tableName); + } + exist = true; } catch (TableStoreException ex) { if (!ex.getErrorCode().equals(OBJECT_NOT_EXIST)) { @@ -71,6 +125,10 @@ public class OTSHelper { return ots.describeTable(new DescribeTableRequest(tableName)); } + public static DescribeTimeseriesTableResponse describeTimeseriesTable(SyncClientInterface ots, String tableName) { + return ((SyncClient) ots).asTimeseriesClient().describeTimeseriesTable(new DescribeTimeseriesTableRequest(tableName)); + } + public static void createTable(SyncClientInterface ots, TableMeta tableMeta, TableOptions tableOptions) { CreateTableRequest request = new CreateTableRequest(tableMeta, tableOptions, new ReservedThroughput(CREATE_TABLE_READ_CU, CREATE_TABLE_WRITE_CU)); @@ -109,11 +167,12 @@ public class OTSHelper { return false; } - public static Map toShardMap(List orderedShardList) { + public static Map toShardMap(List orderedShardList) { Map shardsMap = new HashMap(); for (StreamShard shard : orderedShardList) { shardsMap.put(shard.getShardId(), shard); } return shardsMap; } + } diff --git a/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/OTSStreamJobShardUtil.java b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/OTSStreamJobShardUtil.java new file mode 100644 index 00000000..a062b44f --- /dev/null +++ b/otsstreamreader/src/main/java/com/alibaba/datax/plugin/reader/otsstreamreader/internal/utils/OTSStreamJobShardUtil.java @@ -0,0 +1,105 @@ +package com.alibaba.datax.plugin.reader.otsstreamreader.internal.utils; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.RetryUtil; +import com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConfig; +import com.alibaba.datax.plugin.reader.otsstreamreader.internal.core.CheckpointTimeTracker; +import com.alibaba.datax.plugin.reader.otsstreamreader.internal.model.OTSStreamJobShard; +import com.alibaba.datax.plugin.reader.otsstreamreader.internal.model.StreamJob; +import com.alibaba.fastjson.JSON; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.StreamShard; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.stream.Collectors; + +import static com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConstants.DEFAULT_SLEEP_TIME_IN_MILLS; +import static com.alibaba.datax.plugin.reader.otsstreamreader.internal.config.OTSStreamReaderConstants.RETRY_TIMES; + +/** + * @author mingya.wmy (云时) + */ +public class OTSStreamJobShardUtil { + + private static OTSStreamJobShard otsStreamJobShard = null; + + /** + * 获取全局OTS StreamJob 和 allShards ,懒汉单例模式,减少对OTS接口交互频次 + * 备注:config 和 version 所有TASK 均一样 + * + * @param config + * @param version + * @return + * @throws Exception + */ + public static OTSStreamJobShard getOTSStreamJobShard(OTSStreamReaderConfig config, String version) throws Exception { + if (otsStreamJobShard == null) { + synchronized (OTSHelper.class) { + if (otsStreamJobShard == null) { + otsStreamJobShard = RetryUtil.executeWithRetry(new Callable() { + @Override + public OTSStreamJobShard call() throws Exception { + return getOTSStreamJobShardByOtsClient(config, version); + } + }, RETRY_TIMES, DEFAULT_SLEEP_TIME_IN_MILLS, true); + } + } + } + + return otsStreamJobShard; + } + + /** + * 获取OTS StreamJob 和 allShards + * + * @param config OTS CONF + * @param version OTS STREAM VERSION + * @return + */ + private static OTSStreamJobShard getOTSStreamJobShardByOtsClient(OTSStreamReaderConfig config, String version) { + // Init ots,Task阶段从OTS中获取 allShards 和 streamJob + SyncClientInterface ots = null; + try { + ots = OTSHelper.getOTSInstance(config); + String streamId = OTSHelper.getStreamResponse(ots, config.getDataTable(), config.isTimeseriesTable()).getStreamId(); + List allShards = OTSHelper.getOrderedShardList(ots, streamId, config.isTimeseriesTable()); + + CheckpointTimeTracker checkpointInfoTracker = new CheckpointTimeTracker(ots, config.getStatusTable(), streamId); + StreamJob streamJobFromCPT = checkpointInfoTracker.readStreamJob(config.getEndTimestampMillis()); + if (!StringUtils.equals(streamJobFromCPT.getVersion(), version)) { + throw new RuntimeException(String.format("streamJob version (\"%s\") is not equal to \"%s\", streamJob: %s", + streamJobFromCPT.getVersion(), version, JSON.toJSONString(streamJobFromCPT))); + } + + Set shardIdSetsFromTracker = streamJobFromCPT.getShardIds(); + + if (shardIdSetsFromTracker == null || shardIdSetsFromTracker.isEmpty()) { + throw new RuntimeException(String.format("StreamJob [statusTable=%s, streamId=%s] shardIds can't be null!", + config.getStatusTable(), streamId)); + } + + Set currentAllStreamShardIdSets = allShards.stream().map(streamShard -> streamShard.getShardId()).collect(Collectors.toSet()); + + for (String shardId: shardIdSetsFromTracker) { + if (!currentAllStreamShardIdSets.contains(shardId)) { + allShards.add(new StreamShard(shardId)); + } + } + + StreamJob streamJob = new StreamJob(config.getDataTable(), streamId, version, shardIdSetsFromTracker, + config.getStartTimestampMillis(), config.getEndTimestampMillis()); + + return new OTSStreamJobShard(streamJob, allShards); + } catch (Throwable e) { + throw new DataXException(String.format("Get ots shards error: %s", e.getMessage())); + } finally { + if (ots != null) { + ots.shutdown(); + } + } + } + +} diff --git a/otsstreamreader/tools/tablestore_streamreader_console.py b/otsstreamreader/tools/tablestore_streamreader_console.py new file mode 100644 index 00000000..f9379d72 --- /dev/null +++ b/otsstreamreader/tools/tablestore_streamreader_console.py @@ -0,0 +1,179 @@ +#!/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 new file mode 100644 index 00000000..2444dcbf --- /dev/null +++ b/otsstreamreader/tools/tabulate.py @@ -0,0 +1,1237 @@ +# -*- 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/otswriter/doc/otswriter.md b/otswriter/doc/otswriter.md index cbfaf2a8..43697feb 100644 --- a/otswriter/doc/otswriter.md +++ b/otswriter/doc/otswriter.md @@ -7,13 +7,8 @@ ___ ## 1 快速介绍 -OTSWriter插件实现了向OTS写入数据,目前支持三种写入方式: +OTSWriter插件实现了向OTS写入数据,目前支持了多版本数据的写入、主键自增列的写入等功能。 -* PutRow,对应于OTS API PutRow,插入数据到指定的行,如果该行不存在,则新增一行;若该行存在,则覆盖原有行。 - -* UpdateRow,对应于OTS API UpdateRow,更新指定行的数据,如果该行不存在,则新增一行;若该行存在,则根据请求的内容在这一行中新增、修改或者删除指定列的值。 - -* DeleteRow,对应于OTS API DeleteRow,删除指定行的数据。 OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务,提供海量结构化数据的存储和实时访问。OTS 以实例和表的形式组织数据,通过数据分片和负载均衡技术,实现规模上的无缝扩展。 @@ -28,6 +23,7 @@ OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务, * 配置一个写入OTS作业: +`normal模式` ``` { "job": { @@ -37,48 +33,53 @@ OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务, { "reader": {}, "writer": { - "name": "otswriter", + "name": "otswriter", "parameter": { "endpoint":"", "accessId":"", "accessKey":"", "instanceName":"", - // 导出数据表的表名 "table":"", - - // Writer支持不同类型之间进行相互转换 - // 如下类型转换不支持: - // ================================ - // int -> binary - // double -> bool, binary - // bool -> binary - // bytes -> int, double, bool - // ================================ - + + // 可选 multiVersion||normal,可选配置,默认normal + "mode":"normal", + + //newVersion定义是否使用新版本插件 可选值:true || false + "newVersion":"true", + + //是否允许向包含主键自增列的ots表中写入数据 + //与mode:multiVersion的多版本模式不兼容 + "enableAutoIncrement":"true", + // 需要导入的PK列名,区分大小写 - // 类型支持:STRING,INT + // 类型支持:STRING,INT,BINARY + // 必选 // 1. 支持类型转换,注意类型转换时的精度丢失 // 2. 顺序不要求和表的Meta一致 - "primaryKey" : [ - {"name":"pk1", "type":"string"}, - {"name":"pk2", "type":"int"} + // 3. name全局唯一 + "primaryKey":[ + "userid", + "groupid" ], - + // 需要导入的列名,区分大小写 // 类型支持STRING,INT,DOUBLE,BOOL和BINARY - "column" : [ - {"name":"col2", "type":"INT"}, - {"name":"col3", "type":"STRING"}, - {"name":"col4", "type":"STRING"}, - {"name":"col5", "type":"BINARY"}, - {"name":"col6", "type":"DOUBLE"} + // 必选 + // 1.name全局唯一 + "column":[ + {"name":"addr", "type":"string"}, + {"name":"height", "type":"int"} ], - + + // 如果用户配置了时间戳,系统将使用配置的时间戳,如果没有配置,使用OTS的系统时间戳 + // 可选 + "defaultTimestampInMillionSecond": 142722431, + // 写入OTS的方式 // PutRow : 等同于OTS API中PutRow操作,检查条件是ignore // UpdateRow : 等同于OTS API中UpdateRow操作,检查条件是ignore - // DeleteRow: 等同于OTS API中DeleteRow操作,检查条件是ignore - "writeMode" : "PutRow" + "writeMode":"PutRow" + } } } @@ -92,94 +93,168 @@ OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务, * **endpoint** - * 描述:OTS Server的EndPoint(服务地址),例如http://bazhen.cn−hangzhou.ots.aliyuncs.com。 + * 描述:OTS Server的EndPoint(服务地址),例如http://bazhen.cn−hangzhou.ots.aliyuncs.com。 - * 必选:是
    + * 必选:是
    - * 默认值:无
    + * 默认值:无
    * **accessId** - * 描述:OTS的accessId
    + * 描述:OTS的accessId
    - * 必选:是
    + * 必选:是
    - * 默认值:无
    + * 默认值:无
    * **accessKey** - * 描述:OTS的accessKey
    + * 描述:OTS的accessKey
    - * 必选:是
    + * 必选:是
    - * 默认值:无
    + * 默认值:无
    * **instanceName** - * 描述:OTS的实例名称,实例是用户使用和管理 OTS 服务的实体,用户在开通 OTS 服务之后,需要通过管理控制台来创建实例,然后在实例内进行表的创建和管理。实例是 OTS 资源管理的基础单元,OTS 对应用程序的访问控制和资源计量都在实例级别完成。
    + * 描述:OTS的实例名称,实例是用户使用和管理 OTS 服务的实体,用户在开通 OTS 服务之后,需要通过管理控制台来创建实例,然后在实例内进行表的创建和管理。实例是 OTS 资源管理的基础单元,OTS 对应用程序的访问控制和资源计量都在实例级别完成。
    - * 必选:是
    + * 必选:是
    - * 默认值:无
    + * 默认值:无
    * **table** - * 描述:所选取的需要抽取的表名称,这里有且只能填写一张表。在OTS不存在多表同步的需求。
    + * 描述:所选取的需要抽取的表名称,这里有且只能填写一张表。在OTS不存在多表同步的需求。
    - * 必选:是
    + * 必选:是
    + + * 默认值:无
    + +* **newVersion** + + * 描述:version定义了使用的ots SDK版本。
    + * true,新版本插件,使用com.alicloud.openservices.tablestore的依赖(推荐) + * false,旧版本插件,使用com.aliyun.openservices.ots的依赖,**不支持多版本数据的读取** + + * 必选:否
    + + * 默认值:false
    + +* **mode** + + * 描述:是否为多版本数据,目前有两种模式。
    + * normal,对应普通的数据 + * multiVersion,写入数据为多版本格式的数据,多版本模式下,配置参数有所不同,详见3.4节 + + * 必选:否
    + + * 默认值:normal
    + + +* **enableAutoIncrement** + + * 描述:是否允许向包含主键自增列的ots表中写入数据。
    + * true,插件会扫描表中的自增列信息,并在写入数据时自动添加自增列 + * false,写入含主键自增列的表时会报错 + + * 必选:否
    + + * 默认值:false
    + + +* **isTimeseriesTable** + + * 描述:写入的对应表是否为时序表,仅在mode=normal模式下生效。
    + * true,写入的数据表为时序数据表 + * false,写入的数据表为普通的宽表 + + * 必选:否
    + + * 默认值:false
    + + * 在写入时序数据表的模式下,不需要配置`primaryKey`字段,只需要配置`column`字段,配置样例: + ```json + "column": [ + { + "name": "_m_name", // 表示度量名称(measurement)字段 + }, + { + "name": "_data_source", // 表示数据源(dataSource)字段 + }, + { + "name": "_tags", // 表示标签字段,会被解析为Map类型 + }, + { + "name": "_time", // 表示时间戳字段,会被解析为long类型的值 + }, + { + "name": "tag_a", + "isTag":"true" // 表示标签内部字段,该字段会被解析到标签的字典内部 + }, + { + "name": "column1", // 属性列名称 + "type": "string" // 属性列类型,支持 bool string int double binary + }, + { + "name": "column2", + "type": "int" + } + ], + ``` + + - * 默认值:无
    * **primaryKey** - * 描述: OTS的主键信息,使用JSON的数组描述字段信息。OTS本身是NoSQL系统,在OTSWriter导入数据过程中,必须指定相应地字段名称。 + * 描述: OTS的主键信息,使用JSON的数组描述字段信息。OTS本身是NoSQL系统,在OTSWriter导入数据过程中,必须指定相应地字段名称。 - OTS的PrimaryKey只能支持STRING,INT两种类型,因此OTSWriter本身也限定填写上述两种类型。 + OTS的PrimaryKey只能支持STRING,INT两种类型,因此OTSWriter本身也限定填写上述两种类型。 - DataX本身支持类型转换的,因此对于源头数据非String/Int,OTSWriter会进行数据类型转换。 + DataX本身支持类型转换的,因此对于源头数据非String/Int,OTSWriter会进行数据类型转换。 - 配置实例: + 配置实例: - ```json - "primaryKey" : [ - {"name":"pk1", "type":"string"}, - {"name":"pk2", "type":"int"} - ], - ``` - * 必选:是
    + ```json + "primaryKey":[ + "userid", + "groupid" + ] + ``` + * 必选:是
    - * 默认值:无
    + * 默认值:无
    * **column** - * 描述:所配置的表中需要同步的列名集合,使用JSON的数组描述字段信息。使用格式为 + * 描述:所配置的表中需要同步的列名集合,使用JSON的数组描述字段信息。使用格式为 - ```json - {"name":"col2", "type":"INT"}, - ``` + ```json + {"name":"col2", "type":"INT"}, + ``` - 其中的name指定写入的OTS列名,type指定写入的类型。OTS类型支持STRING,INT,DOUBLE,BOOL和BINARY几种类型 。 + 其中的name指定写入的OTS列名,type指定写入的类型。OTS类型支持STRING,INT,DOUBLE,BOOL和BINARY几种类型 。 - 写入过程不支持常量、函数或者自定义表达式。 + 写入过程不支持常量、函数或者自定义表达式。 - * 必选:是
    + * 必选:是
    - * 默认值:无
    + * 默认值:无
    * **writeMode** - * 描述:写入模式,目前支持两种模式, + * 描述:写入模式,目前支持两种模式, - * PutRow,对应于OTS API PutRow,插入数据到指定的行,如果该行不存在,则新增一行;若该行存在,则覆盖原有行。 + * PutRow,对应于OTS API PutRow,插入数据到指定的行,如果该行不存在,则新增一行;若该行存在,则覆盖原有行。 - * UpdateRow,对应于OTS API UpdateRow,更新指定行的数据,如果该行不存在,则新增一行;若该行存在,则根据请求的内容在这一行中新增、修改或者删除指定列的值。 + * UpdateRow,对应于OTS API UpdateRow,更新指定行的数据,如果该行不存在,则新增一行;若该行存在,则根据请求的内容在这一行中新增、修改或者删除指定列的值。 - * DeleteRow,对应于OTS API DeleteRow,删除指定行的数据。 - * 必选:是
    +* 必选:是
    - * 默认值:无
    +* 默认值:无
    ### 3.3 类型转换 @@ -197,43 +272,79 @@ OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务, * 注意,OTS本身不支持日期型类型。应用层一般使用Long报错时间的Unix TimeStamp。 -## 4 性能报告 +### 3.4 multiVersion模式 -### 4.1 环境准备 +#### 3.4.1 模式介绍 -#### 4.1.1 数据特征 +multiVersion模式解决了ots数据库中多版本数据的导入问题。支持Hbase的全量数据迁移到OTS -2列PK(10 + 8),15列String(10 Byte), 2两列Integer(8 Byte),算上Column Name每行大概327Byte,每次BatchWriteRow写入100行数据,所以当个请求的数据大小是32KB。 - -#### 4.1.2 机器参数 - -OTS端:3台前端机,5台后端机 - -DataX运行端: 24核CPU, 98GB内存 - -### 4.2 测试报告 - -#### 4.2.1 测试报告 - -|并发数|DataX CPU|DATAX流量 |OTS 流量 | BatchWrite前端QPS| BatchWriteRow前端延时| -|--------|--------| --------|--------|--------|------| -|40| 1027% |Speed 22.13MB/s, 112640 records/s|65.8M/s |42|153ms | -|50| 1218% |Speed 24.11MB/s, 122700 records/s|73.5M/s |47|174ms| -|60| 1355% |Speed 25.31MB/s, 128854 records/s|78.1M/s |50|190ms| -|70| 1578% |Speed 26.35MB/s, 134121 records/s|80.8M/s |52|210ms| -|80| 1771% |Speed 26.55MB/s, 135161 records/s|82.7M/s |53|230ms| +* 注意:这种模式的数据格式比较特殊,该writer需要reader也提供版本的输出 +* 当前只有hbase reader 与 ots reader提供这种模式,使用时切记注意 +#### 3.4.2 配置样例 +``` +{ + "job": { + "setting": { + }, + "content": [ + { + "reader": {}, + "writer": { + "name": "otswriter", + "parameter": { + "endpoint":"", + "accessId":"", + "accessKey":"", + "instanceName":"", + "table":"", + + // 多版本模式,插件会按照多版本模式去解析所有配置 + "mode":"multiVersion", + + "newVersion":"true", + + // 配置PK信息 + // 考虑到配置成本,并不需要配置PK在Record(Line)中的位置,要求 + // Record的格式固定,PK一定在行首,PK之后是columnName,格式如下: + // 如:{pk0,pk1,pk2,pk3}, {columnName}, {timestamp}, {value} + "primaryKey":[ + "userid", + "groupid" + ], + + // 列名前缀过滤 + // 描述:hbase导入过来的数据,cf和qulifier共同组成columnName, + // OTS并不支持cf,所以需要将cf过滤掉 + // 注意: + // 1.该参数选填,如果没有填写或者值为空字符串,表示不对列名进行过滤。 + // 2.如果datax传入的数据columnName列不是以前缀开始,则将该Record放入脏数据回收器中 + "columnNamePrefixFilter":"cf:" + } + } + } + ] + } +} +``` +## 4 约束限制 - -## 5 约束限制 - -### 5.1 写入幂等性 +### 4.1 写入幂等性 OTS写入本身是支持幂等性的,也就是使用OTS SDK同一条数据写入OTS系统,一次和多次请求的结果可以理解为一致的。因此对于OTSWriter多次尝试写入同一条数据与写入一条数据结果是等同的。 -### 5.2 单任务FailOver +### 4.2 单任务FailOver 由于OTS写入本身是幂等性的,因此可以支持单任务FailOver。即一旦写入Fail,DataX会重新启动相关子任务进行重试。 -## 6 FAQ +## 5 FAQ + +* 1.如果使用多版本模式,value为null应该怎么解释? + * : 表示删除指定的版本 +* 2.如果ts列为空怎么办? + * :插件记录为垃圾数据 +* 3.Record的count和期望不符? + * : 插件异常终止 +* 4.在普通模式下,采用UpdateRow的方式写入数据,如果不指定TS,相同行数的数据怎么写入到OTS中? + * : 后面的覆盖前面的数据 diff --git a/otswriter/pom.xml b/otswriter/pom.xml index cb255e1f..f393d76c 100644 --- a/otswriter/pom.xml +++ b/otswriter/pom.xml @@ -10,17 +10,6 @@ otswriter - - org.apache.logging.log4j - log4j-api - 2.17.1 - - - - org.apache.logging.log4j - log4j-core - 2.17.1 - com.alibaba.datax datax-common @@ -44,18 +33,25 @@ com.aliyun.openservices ots-public - 2.2.4 + 2.2.6 - - log4j-api - org.apache.logging.log4j - log4j-core org.apache.logging.log4j - + + + com.aliyun.openservices + tablestore + 5.13.10 + + + log4j-core + org.apache.logging.log4j + + + com.google.code.gson gson @@ -63,6 +59,14 @@ + + + src/main/java + + **/*.properties + + + diff --git a/otswriter/src/main/assembly/package.xml b/otswriter/src/main/assembly/package.xml index 5ae7a015..91523025 100644 --- a/otswriter/src/main/assembly/package.xml +++ b/otswriter/src/main/assembly/package.xml @@ -12,8 +12,8 @@ src/main/resources plugin.json - plugin_job_template.json - + plugin_job_template.json + plugin/writer/otswriter diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/IOtsWriterMasterProxy.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/IOtsWriterMasterProxy.java new file mode 100644 index 00000000..af364b86 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/IOtsWriterMasterProxy.java @@ -0,0 +1,16 @@ +package com.alibaba.datax.plugin.writer.otswriter; + +import com.alibaba.datax.common.util.Configuration; + +import java.util.List; + +public interface IOtsWriterMasterProxy { + + public void init(Configuration param) throws Exception; + + public void close(); + + public List split(int mandatoryNumber); + + +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/IOtsWriterSlaveProxy.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/IOtsWriterSlaveProxy.java new file mode 100644 index 00000000..1ce78ccb --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/IOtsWriterSlaveProxy.java @@ -0,0 +1,25 @@ +package com.alibaba.datax.plugin.writer.otswriter; + +import com.alibaba.datax.common.plugin.RecordReceiver; +import com.alibaba.datax.common.plugin.TaskPluginCollector; +import com.alibaba.datax.common.util.Configuration; + +public interface IOtsWriterSlaveProxy { + + /** + * Slave的初始化,创建Slave所使用的资源 + */ + public void init(Configuration configuration); + + /** + * 释放Slave的所有资源 + */ + public void close() throws OTSCriticalException; + + /** + * Slave的执行器,将Datax的数据写入到OTS中 + * @param recordReceiver + * @throws OTSCriticalException + */ + public void write(RecordReceiver recordReceiver, TaskPluginCollector taskPluginCollector) throws OTSCriticalException; +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/Key.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/Key.java index 0724b9cf..10dd9cc9 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/Key.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/Key.java @@ -25,7 +25,11 @@ public final class Key { public final static String OTS_ACCESSKEY = "accessKey"; public final static String OTS_INSTANCE_NAME = "instanceName"; - + public final static String ENABLE_AUTO_INCREMENT = "enableAutoIncrement"; + public final static String IS_TIMESERIES_TABLE = "isTimeseriesTable"; + + public final static String TIMEUNIT_FORMAT = "timeunit"; + public final static String TABLE_NAME = "table"; public final static String PRIMARY_KEY = "primaryKey"; @@ -33,4 +37,11 @@ public final class Key { public final static String COLUMN = "column"; public final static String WRITE_MODE = "writeMode"; + + public final static String MODE = "mode"; + public final static String NEW_VERISON = "newVersion"; + + public final static String DEFAULT_TIMESTAMP = "defaultTimestampInMillisecond"; + + public final static String COLUMN_NAME_PREFIX_FILTER = "columnNamePrefixFilter"; } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OTSCriticalException.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OTSCriticalException.java new file mode 100644 index 00000000..b89df008 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OTSCriticalException.java @@ -0,0 +1,24 @@ +package com.alibaba.datax.plugin.writer.otswriter; + +/** + * 插件错误异常,该异常主要用于描述插件的异常退出 + * @author redchen + */ +public class OTSCriticalException extends Exception{ + + private static final long serialVersionUID = 5820460098894295722L; + + public OTSCriticalException() {} + + public OTSCriticalException(String message) { + super(message); + } + + public OTSCriticalException(Throwable a) { + super(a); + } + + public OTSCriticalException(String message, Throwable a) { + super(message, a); + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OTSErrorCode.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OTSErrorCode.java new file mode 100644 index 00000000..86877730 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OTSErrorCode.java @@ -0,0 +1,115 @@ +/** + * Copyright (C) Alibaba Cloud Computing + * All rights reserved. + * + * 版权所有 (C)阿里云计算有限公司 + */ + +package com.alibaba.datax.plugin.writer.otswriter; + +/** + * 表示来自开放结构化数据服务(Open Table Service,OTS)的错误代码。 + * + */ +public interface OTSErrorCode { + /** + * 用户身份验证失败。 + */ + static final String AUTHORIZATION_FAILURE = "OTSAuthFailed"; + + /** + * 服务器内部错误。 + */ + static final String INTERNAL_SERVER_ERROR = "OTSInternalServerError"; + + /** + * 参数错误。 + */ + static final String INVALID_PARAMETER = "OTSParameterInvalid"; + + /** + * 整个请求过大。 + */ + static final String REQUEST_TOO_LARGE = "OTSRequestBodyTooLarge"; + + /** + * 客户端请求超时。 + */ + static final String REQUEST_TIMEOUT = "OTSRequestTimeout"; + + /** + * 用户的配额已经用满。 + */ + static final String QUOTA_EXHAUSTED = "OTSQuotaExhausted"; + + /** + * 内部服务器发生failover,导致表的部分分区不可服务。 + */ + static final String PARTITION_UNAVAILABLE = "OTSPartitionUnavailable"; + + /** + * 表刚被创建还无法立马提供服务。 + */ + static final String TABLE_NOT_READY = "OTSTableNotReady"; + + /** + * 请求的表不存在。 + */ + static final String OBJECT_NOT_EXIST = "OTSObjectNotExist"; + + /** + * 请求创建的表已经存在。 + */ + static final String OBJECT_ALREADY_EXIST = "OTSObjectAlreadyExist"; + + /** + * 多个并发的请求写同一行数据,导致冲突。 + */ + static final String ROW_OPEARTION_CONFLICT = "OTSRowOperationConflict"; + + /** + * 主键不匹配。 + */ + static final String INVALID_PK = "OTSInvalidPK"; + + /** + * 读写能力调整过于频繁。 + */ + static final String TOO_FREQUENT_RESERVED_THROUGHPUT_ADJUSTMENT = "OTSTooFrequentReservedThroughputAdjustment"; + + /** + * 该行总列数超出限制。 + */ + static final String OUT_OF_COLUMN_COUNT_LIMIT = "OTSOutOfColumnCountLimit"; + + /** + * 该行所有列数据大小总和超出限制。 + */ + static final String OUT_OF_ROW_SIZE_LIMIT = "OTSOutOfRowSizeLimit"; + + /** + * 剩余预留读写能力不足。 + */ + static final String NOT_ENOUGH_CAPACITY_UNIT = "OTSNotEnoughCapacityUnit"; + + /** + * 预查条件检查失败。 + */ + static final String CONDITION_CHECK_FAIL = "OTSConditionCheckFail"; + + /** + * 在OTS内部操作超时。 + */ + static final String STORAGE_TIMEOUT = "OTSTimeout"; + + /** + * 在OTS内部有服务器不可访问。 + */ + static final String SERVER_UNAVAILABLE = "OTSServerUnavailable"; + + /** + * OTS内部服务器繁忙。 + */ + static final String SERVER_BUSY = "OTSServerBusy"; + +} \ No newline at end of file diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriter.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriter.java index 4d2ed17b..46227238 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriter.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriter.java @@ -1,41 +1,44 @@ package com.alibaba.datax.plugin.writer.otswriter; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - 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.writer.otswriter.utils.Common; -import com.aliyun.openservices.ots.ClientException; -import com.aliyun.openservices.ots.OTSException; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSConf; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSConst; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSMode; +import com.alibaba.datax.plugin.writer.otswriter.utils.GsonParser; +import com.alicloud.openservices.tablestore.ClientException; +import com.alicloud.openservices.tablestore.TableStoreException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; public class OtsWriter { + public static class Job extends Writer.Job { private static final Logger LOG = LoggerFactory.getLogger(Job.class); - private OtsWriterMasterProxy proxy = new OtsWriterMasterProxy(); - + + private IOtsWriterMasterProxy proxy; + @Override public void init() { LOG.info("init() begin ..."); + proxy = new OtsWriterMasterProxy(); try { this.proxy.init(getPluginJobConf()); - } catch (OTSException e) { - LOG.error("OTSException: {}", e.getMessage(), e); - throw DataXException.asDataXException(new OtsWriterError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); + } catch (TableStoreException e) { + LOG.error("OTSException: {}", e.toString(), e); + throw DataXException.asDataXException(new OtsWriterError(e.getErrorCode(), "OTS Client Error"), e.toString(), e); } catch (ClientException e) { - LOG.error("ClientException: {}", e.getMessage(), e); - throw DataXException.asDataXException(new OtsWriterError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); - } catch (IllegalArgumentException e) { - LOG.error("IllegalArgumentException. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsWriterError.INVALID_PARAM, Common.getDetailMessage(e), e); + LOG.error("ClientException: {}", e.toString(), e); + throw DataXException.asDataXException(OtsWriterError.ERROR, e.toString(), e); } catch (Exception e) { - LOG.error("Exception. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsWriterError.ERROR, Common.getDetailMessage(e), e); + LOG.error("Exception. ErrorMsg:{}", e.toString(), e); + throw DataXException.asDataXException(OtsWriterError.ERROR, e.toString(), e); } + LOG.info("init() end ..."); } @@ -50,42 +53,67 @@ public class OtsWriter { return this.proxy.split(mandatoryNumber); } catch (Exception e) { LOG.error("Exception. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsWriterError.ERROR, Common.getDetailMessage(e), e); + throw DataXException.asDataXException(OtsWriterError.ERROR, e.toString(), e); } } } - + public static class Task extends Writer.Task { private static final Logger LOG = LoggerFactory.getLogger(Task.class); - private OtsWriterSlaveProxy proxy = new OtsWriterSlaveProxy(); - + private IOtsWriterSlaveProxy proxy = null; + + /** + * 基于配置,构建对应的worker代理 + */ @Override - public void init() {} + public void init() { + OTSConf conf = GsonParser.jsonToConf(this.getPluginJobConf().getString(OTSConst.OTS_CONF)); + // 是否使用新接口 + if(conf.isNewVersion()) { + if (conf.getMode() == OTSMode.MULTI_VERSION) { + LOG.info("init OtsWriterSlaveProxyMultiVersion"); + proxy = new OtsWriterSlaveProxyMultiversion(); + } else { + LOG.info("init OtsWriterSlaveProxyNormal"); + proxy = new OtsWriterSlaveProxyNormal(); + } + + } + else{ + proxy = new OtsWriterSlaveProxyOld(); + } + + proxy.init(this.getPluginJobConf()); + + } @Override public void destroy() { - this.proxy.close(); + try { + proxy.close(); + } catch (OTSCriticalException e) { + LOG.error("OTSCriticalException. ErrorMsg:{}", e.getMessage(), e); + throw DataXException.asDataXException(OtsWriterError.ERROR, e.toString(), e); + } } @Override public void startWrite(RecordReceiver lineReceiver) { LOG.info("startWrite() begin ..."); + try { - this.proxy.init(this.getPluginJobConf()); - this.proxy.write(lineReceiver, this.getTaskPluginCollector()); - } catch (OTSException e) { - LOG.error("OTSException: {}", e.getMessage(), e); - throw DataXException.asDataXException(new OtsWriterError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); + proxy.write(lineReceiver, this.getTaskPluginCollector()); + } catch (TableStoreException e) { + LOG.error("OTSException: {}", e.toString(), e); + throw DataXException.asDataXException(new OtsWriterError(e.getErrorCode(), "OTS Client Error"), e.toString(), e); } catch (ClientException e) { - LOG.error("ClientException: {}", e.getMessage(), e); - throw DataXException.asDataXException(new OtsWriterError(e.getErrorCode(), "OTS端的错误"), Common.getDetailMessage(e), e); - } catch (IllegalArgumentException e) { - LOG.error("IllegalArgumentException. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsWriterError.INVALID_PARAM, Common.getDetailMessage(e), e); + LOG.error("ClientException: {}", e.toString(), e); + throw DataXException.asDataXException(OtsWriterError.ERROR, e.toString(), e); } catch (Exception e) { - LOG.error("Exception. ErrorMsg:{}", e.getMessage(), e); - throw DataXException.asDataXException(OtsWriterError.ERROR, Common.getDetailMessage(e), e); + LOG.error("Exception. ErrorMsg:{}", e.toString(), e); + throw DataXException.asDataXException(OtsWriterError.ERROR, e.toString(), e); } + LOG.info("startWrite() end ..."); } } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterError.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterError.java index 67d1ee2b..092a7343 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterError.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterError.java @@ -14,10 +14,10 @@ public class OtsWriterError implements ErrorCode { public final static OtsWriterError ERROR = new OtsWriterError( "OtsWriterError", - "该错误表示插件的内部错误,表示系统没有处理到的异常"); + "This error represents an internal error of the ots writer plugin, which indicates that the system is not processed."); public final static OtsWriterError INVALID_PARAM = new OtsWriterError( "OtsWriterInvalidParameter", - "该错误表示参数错误,表示用户输入了错误的参数格式等"); + "This error represents a parameter error, indicating that the user entered the wrong parameter format."); public OtsWriterError (String code) { this.code = code; @@ -41,6 +41,6 @@ public class OtsWriterError implements ErrorCode { @Override public String toString() { - return this.code; + return "[ code:" + this.code + ", message:" + this.description + "]"; } } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterMasterProxy.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterMasterProxy.java index 91cf9b12..774aca1e 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterMasterProxy.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterMasterProxy.java @@ -1,110 +1,138 @@ package com.alibaba.datax.plugin.writer.otswriter; -import java.util.ArrayList; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.alibaba.datax.common.util.Configuration; import com.alibaba.datax.plugin.writer.otswriter.callable.GetTableMetaCallable; import com.alibaba.datax.plugin.writer.otswriter.model.OTSConf; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSConf.RestrictConf; import com.alibaba.datax.plugin.writer.otswriter.model.OTSConst; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSMode; import com.alibaba.datax.plugin.writer.otswriter.model.OTSOpType; -import com.alibaba.datax.plugin.writer.otswriter.utils.GsonParser; -import com.alibaba.datax.plugin.writer.otswriter.utils.ParamChecker; -import com.alibaba.datax.plugin.writer.otswriter.utils.RetryHelper; -import com.alibaba.datax.plugin.writer.otswriter.utils.WriterModelParser; -import com.aliyun.openservices.ots.OTSClient; -import com.aliyun.openservices.ots.model.TableMeta; +import com.alibaba.datax.plugin.writer.otswriter.utils.*; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.TimeseriesClient; +import com.alicloud.openservices.tablestore.model.TableMeta; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class OtsWriterMasterProxy implements IOtsWriterMasterProxy { -public class OtsWriterMasterProxy { - - private OTSConf conf = new OTSConf(); - - private OTSClient ots = null; - - private TableMeta meta = null; - private static final Logger LOG = LoggerFactory.getLogger(OtsWriterMasterProxy.class); - + private OTSConf conf = new OTSConf(); + private SyncClientInterface ots = null; + private TableMeta meta = null; + /** * @param param * @throws Exception */ + @Override public void init(Configuration param) throws Exception { - + // 默认参数 - conf.setRetry(param.getInt(OTSConst.RETRY, 18)); - conf.setSleepInMillisecond(param.getInt(OTSConst.SLEEP_IN_MILLISECOND, 100)); - conf.setBatchWriteCount(param.getInt(OTSConst.BATCH_WRITE_COUNT, 100)); - conf.setConcurrencyWrite(param.getInt(OTSConst.CONCURRENCY_WRITE, 5)); - conf.setIoThreadCount(param.getInt(OTSConst.IO_THREAD_COUNT, 1)); - conf.setSocketTimeout(param.getInt(OTSConst.SOCKET_TIMEOUT, 20000)); - conf.setConnectTimeout(param.getInt(OTSConst.CONNECT_TIMEOUT, 10000)); - conf.setBufferSize(param.getInt(OTSConst.BUFFER_SIZE, 1024)); - - RestrictConf restrictConf = conf.new RestrictConf(); - restrictConf.setRequestTotalSizeLimition(param.getInt(OTSConst.REQUEST_TOTAL_SIZE_LIMITATION, 1024 * 1024)); - restrictConf.setAttributeColumnSize(param.getInt(OTSConst.ATTRIBUTE_COLUMN_SIZE_LIMITATION, 2 * 1024 * 1024)); - restrictConf.setPrimaryKeyColumnSize(param.getInt(OTSConst.PRIMARY_KEY_COLUMN_SIZE_LIMITATION, 1024)); - restrictConf.setMaxColumnsCount(param.getInt(OTSConst.ATTRIBUTE_COLUMN_MAX_COUNT, 1024)); - conf.setRestrictConf(restrictConf); + setStaticParams(param); + + conf.setTimestamp(param.getInt(Key.DEFAULT_TIMESTAMP, -1)); + conf.setRequestTotalSizeLimitation(param.getInt(OTSConst.REQUEST_TOTAL_SIZE_LIMITATION, 1024 * 1024)); // 必选参数 - conf.setEndpoint(ParamChecker.checkStringAndGet(param, Key.OTS_ENDPOINT)); - conf.setAccessId(ParamChecker.checkStringAndGet(param, Key.OTS_ACCESSID)); - conf.setAccessKey(ParamChecker.checkStringAndGet(param, Key.OTS_ACCESSKEY)); - conf.setInstanceName(ParamChecker.checkStringAndGet(param, Key.OTS_INSTANCE_NAME)); - conf.setTableName(ParamChecker.checkStringAndGet(param, Key.TABLE_NAME)); - - conf.setOperation(WriterModelParser.parseOTSOpType(ParamChecker.checkStringAndGet(param, Key.WRITE_MODE))); - - ots = new OTSClient( - this.conf.getEndpoint(), - this.conf.getAccessId(), - this.conf.getAccessKey(), - this.conf.getInstanceName()); - - meta = getTableMeta(ots, conf.getTableName()); - LOG.info("Table Meta : {}", GsonParser.metaToJson(meta)); - - conf.setPrimaryKeyColumn(WriterModelParser.parseOTSPKColumnList(ParamChecker.checkListAndGet(param, Key.PRIMARY_KEY, true))); - ParamChecker.checkPrimaryKey(meta, conf.getPrimaryKeyColumn()); - - conf.setAttributeColumn(WriterModelParser.parseOTSAttrColumnList(ParamChecker.checkListAndGet(param, Key.COLUMN, conf.getOperation() == OTSOpType.UPDATE_ROW ? true : false))); - ParamChecker.checkAttribute(conf.getAttributeColumn()); + conf.setEndpoint(ParamChecker.checkStringAndGet(param, Key.OTS_ENDPOINT)); + conf.setAccessId(ParamChecker.checkStringAndGet(param, Key.OTS_ACCESSID)); + conf.setAccessKey(ParamChecker.checkStringAndGet(param, Key.OTS_ACCESSKEY)); + conf.setInstanceName(ParamChecker.checkStringAndGet(param, Key.OTS_INSTANCE_NAME)); + conf.setTableName(ParamChecker.checkStringAndGet(param, Key.TABLE_NAME)); + + ots = Common.getOTSInstance(conf); + + conf.setNewVersion(param.getBool(Key.NEW_VERISON, false)); + conf.setMode(WriterModelParser.parseOTSMode(param.getString(Key.MODE, "normal"))); + conf.setEnableAutoIncrement(param.getBool(Key.ENABLE_AUTO_INCREMENT, false)); + conf.setTimeseriesTable(param.getBool(Key.IS_TIMESERIES_TABLE, false)); + ParamChecker.checkVersion(conf); + + if (!conf.isTimeseriesTable()){ + meta = getTableMeta(ots, conf.getTableName()); + LOG.debug("Table Meta : {}", GsonParser.metaToJson(meta)); + conf.setPrimaryKeyColumn(WriterModelParser.parseOTSPKColumnList(meta, ParamChecker.checkListAndGet(param, Key.PRIMARY_KEY, true))); + } + + if (conf.getMode() == OTSMode.MULTI_VERSION) { + conf.setOperation(OTSOpType.UPDATE_ROW);// 多版本只支持Update模式 + conf.setColumnNamePrefixFilter(param.getString(Key.COLUMN_NAME_PREFIX_FILTER, null)); + } else if (!conf.isTimeseriesTable()){ // 普通模式,写入宽表 + conf.setOperation(WriterModelParser.parseOTSOpType(ParamChecker.checkStringAndGet(param, Key.WRITE_MODE), conf.getMode())); + conf.setAttributeColumn(WriterModelParser.parseOTSAttrColumnList(conf.getPrimaryKeyColumn(), ParamChecker.checkListAndGet(param, Key.COLUMN, false), conf.getMode() + ) + ); + ParamChecker.checkAttribute(conf.getAttributeColumn()); + } else { // 普通模式,写入时序表 + conf.setOperation(OTSOpType.PUT_ROW);// 时序表只支持Put模式 + conf.setAttributeColumn(WriterModelParser.parseOTSTimeseriesRowAttrList(ParamChecker.checkListAndGet(param, Key.COLUMN, true))); + conf.setTimeUnit(ParamChecker.checkTimeUnitAndGet(param.getString(Key.TIMEUNIT_FORMAT, "MICROSECONDS"))); + } + + /** + * 如果配置支持主键列自增 + */ + if (conf.getEnableAutoIncrement()) { + ParamChecker.checkPrimaryKeyWithAutoIncrement(meta, conf.getPrimaryKeyColumn()); + conf.setEncodePkColumnMapping(Common.getEncodePkColumnMappingWithAutoIncrement(meta, conf.getPrimaryKeyColumn())); + } + /** + * 如果配置不支持主键列自增 + */ + else if (!conf.isTimeseriesTable()){ + ParamChecker.checkPrimaryKey(meta, conf.getPrimaryKeyColumn()); + conf.setEncodePkColumnMapping(Common.getEncodePkColumnMapping(meta, conf.getPrimaryKeyColumn())); + } + + } - - public List split(int mandatoryNumber){ + + @Override + public List split(int mandatoryNumber) { LOG.info("Begin split and MandatoryNumber : {}", mandatoryNumber); List configurations = new ArrayList(); + String json = GsonParser.confToJson(this.conf); for (int i = 0; i < mandatoryNumber; i++) { Configuration configuration = Configuration.newDefault(); - configuration.set(OTSConst.OTS_CONF, GsonParser.confToJson(this.conf)); + configuration.set(OTSConst.OTS_CONF, json); configurations.add(configuration); } LOG.info("End split."); - assert(mandatoryNumber == configurations.size()); return configurations; } - + + @Override public void close() { ots.shutdown(); } - + public OTSConf getOTSConf() { return conf; } // private function - private TableMeta getTableMeta(OTSClient ots, String tableName) throws Exception { + private TableMeta getTableMeta(SyncClientInterface ots, String tableName) throws Exception { return RetryHelper.executeWithRetry( new GetTableMetaCallable(ots, tableName), conf.getRetry(), conf.getSleepInMillisecond() - ); + ); + } + + public void setStaticParams(Configuration param) { + // 默认参数 + conf.setRetry(param.getInt(OTSConst.RETRY, 18)); + conf.setSleepInMillisecond(param.getInt(OTSConst.SLEEP_IN_MILLISECOND, 100)); + conf.setBatchWriteCount(param.getInt(OTSConst.BATCH_WRITE_COUNT, 100)); + conf.setConcurrencyWrite(param.getInt(OTSConst.CONCURRENCY_WRITE, 5)); + conf.setIoThreadCount(param.getInt(OTSConst.IO_THREAD_COUNT, 1)); + conf.setSocketTimeoutInMillisecond(param.getInt(OTSConst.SOCKET_TIMEOUTIN_MILLISECOND, 10000)); + conf.setConnectTimeoutInMillisecond(param.getInt(OTSConst.CONNECT_TIMEOUT_IN_MILLISECOND, 10000)); + } } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyMultiversion.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyMultiversion.java new file mode 100644 index 00000000..6db75692 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyMultiversion.java @@ -0,0 +1,135 @@ +package com.alibaba.datax.plugin.writer.otswriter; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.plugin.RecordReceiver; +import com.alibaba.datax.common.plugin.TaskPluginCollector; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.writer.otswriter.model.*; +import com.alibaba.datax.plugin.writer.otswriter.utils.CollectorUtil; +import com.alibaba.datax.plugin.writer.otswriter.utils.Common; +import com.alibaba.datax.plugin.writer.otswriter.utils.GsonParser; +import com.alibaba.datax.plugin.writer.otswriter.utils.ParseRecord; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.PrimaryKey; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.alibaba.datax.plugin.writer.otswriter.utils.Common.getOTSInstance; + +public class OtsWriterSlaveProxyMultiversion implements IOtsWriterSlaveProxy { + + private OTSConf conf = null; + private SyncClientInterface ots = null; + private OTSSendBuffer buffer = null; + private Map pkColumnMapping = null; + private static final Logger LOG = LoggerFactory.getLogger(OtsWriterSlaveProxyMultiversion.class); + + @Override + public void init(Configuration configuration) { + LOG.info("OtsWriterSlaveProxyMultiversion init begin"); + this.conf = GsonParser.jsonToConf(configuration.getString(OTSConst.OTS_CONF)); + this.ots = getOTSInstance(conf); + this.pkColumnMapping = Common.getPkColumnMapping(conf.getEncodePkColumnMapping()); + buffer = new OTSSendBuffer(ots, conf); + LOG.info("init end"); + } + + @Override + public void close() throws OTSCriticalException { + LOG.info("close begin"); + ots.shutdown(); + LOG.info("close end"); + } + + @Override + public void write(RecordReceiver recordReceiver, TaskPluginCollector taskPluginCollector) throws OTSCriticalException { + LOG.info("write begin"); + // 初始化全局垃圾回收器 + CollectorUtil.init(taskPluginCollector); + // Record format : {PK1, PK2, ...} {ColumnName} {TimeStamp} {Value} + int expectColumnCount = conf.getPrimaryKeyColumn().size()+ 3;// 3表示{ColumnName} {TimeStamp} {Value} + Record record = null; + PrimaryKey lastCellPk = null; + List rowBuffer = new ArrayList(); + while ((record = recordReceiver.getFromReader()) != null) { + + LOG.debug("Record Raw: {}", record.toString()); + + int columnCount = record.getColumnNumber(); + if (columnCount != expectColumnCount) { + // 如果Column的个数和预期的个数不一致时,认为是系统故障或者用户配置Column错误,异常退出 + throw new OTSCriticalException(String.format( + OTSErrorMessage.RECORD_AND_COLUMN_SIZE_ERROR, + columnCount, + expectColumnCount, + record.toString() + )); + } + + PrimaryKey curPk = null; + if ((curPk = Common.getPKFromRecord(this.pkColumnMapping, record)) == null) { + continue; + } + + // check same row + if (lastCellPk == null) { + lastCellPk = curPk; + } else if (!lastCellPk.equals(curPk)) { + OTSLine line = ParseRecord.parseMultiVersionRecordToOTSLine( + conf.getTableName(), + conf.getOperation(), + pkColumnMapping, + conf.getColumnNamePrefixFilter(), + lastCellPk, + rowBuffer); + if (line != null) { + buffer.write(line); + } + rowBuffer.clear(); + lastCellPk = curPk; + } + rowBuffer.add(record); + } + // Flush剩余数据 + if (!rowBuffer.isEmpty()) { + OTSLine line = ParseRecord.parseMultiVersionRecordToOTSLine( + conf.getTableName(), + conf.getOperation(), + pkColumnMapping, + conf.getColumnNamePrefixFilter(), + lastCellPk, + rowBuffer); + if (line != null) { + buffer.write(line); + } + } + + buffer.close(); + LOG.info("write end"); + } + + public void setOts(SyncClientInterface ots){ + this.ots = ots; + } + + public OTSConf getConf() { + return conf; + } + + public void setConf(OTSConf conf) { + this.conf = conf; + } + + public void setBuffer(OTSSendBuffer buffer) { + this.buffer = buffer; + } + + public void setPkColumnMapping(Map pkColumnMapping) { + this.pkColumnMapping = pkColumnMapping; + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyNormal.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyNormal.java new file mode 100644 index 00000000..aaa0ef04 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyNormal.java @@ -0,0 +1,153 @@ +package com.alibaba.datax.plugin.writer.otswriter; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.plugin.RecordReceiver; +import com.alibaba.datax.common.plugin.TaskPluginCollector; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.writer.otswriter.callable.GetTableMetaCallable; +import com.alibaba.datax.plugin.writer.otswriter.model.*; +import com.alibaba.datax.plugin.writer.otswriter.utils.*; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; +import com.alicloud.openservices.tablestore.model.TableMeta; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +import static com.alibaba.datax.plugin.writer.otswriter.utils.Common.getOTSInstance; + +public class OtsWriterSlaveProxyNormal implements IOtsWriterSlaveProxy { + + private OTSConf conf = null; + private SyncClientInterface ots = null; + private OTSSendBuffer buffer = null; + private Map pkColumnMapping = null; + private static final Logger LOG = LoggerFactory.getLogger(OtsWriterSlaveProxyNormal.class); + private PrimaryKeySchema primaryKeySchema =null; + + + @Override + public void init(Configuration configuration) { + LOG.info("init begin"); + this.conf = GsonParser.jsonToConf(configuration.getString(OTSConst.OTS_CONF)); + this.ots = getOTSInstance(conf); + if (!conf.isTimeseriesTable()){ + this.pkColumnMapping = Common.getPkColumnMapping(conf.getEncodePkColumnMapping()); + } + + buffer = new OTSSendBuffer(ots, conf); + + if(conf.getEnableAutoIncrement()){ + primaryKeySchema = getAutoIncrementKey(); + } + LOG.info("init end"); + } + + @Override + public void close() throws com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException { + LOG.info("close begin"); + ots.shutdown(); + LOG.info("close end"); + } + + @Override + public void write(RecordReceiver recordReceiver, TaskPluginCollector taskPluginCollector) throws com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException { + LOG.info("write begin"); + + // 初始化全局垃圾回收器 + CollectorUtil.init(taskPluginCollector); + int expectColumnCount = conf.getAttributeColumn().size(); + if (!conf.isTimeseriesTable()){ + expectColumnCount += conf.getPrimaryKeyColumn().size(); + } + Record record = null; + + while ((record = recordReceiver.getFromReader()) != null) { + + LOG.debug("Record Raw: {}", record.toString()); + + int columnCount = record.getColumnNumber(); + if (columnCount != expectColumnCount) { + // 如果Column的个数和预期的个数不一致时,认为是系统故障或者用户配置Column错误,异常退出 + throw new OTSCriticalException(String.format( + OTSErrorMessage.RECORD_AND_COLUMN_SIZE_ERROR, + columnCount, + expectColumnCount, + record.toString() + )); + } + OTSLine line; + + if(conf.getEnableAutoIncrement()){ + line = ParseRecord.parseNormalRecordToOTSLineWithAutoIncrement( + conf.getTableName(), + conf.getOperation(), + pkColumnMapping, + conf.getAttributeColumn(), + record, + conf.getTimestamp(), + primaryKeySchema); + } + else if(!conf.isTimeseriesTable()){ + line = ParseRecord.parseNormalRecordToOTSLine( + conf.getTableName(), + conf.getOperation(), + pkColumnMapping, + conf.getAttributeColumn(), + record, + conf.getTimestamp()); + }else{ + line = ParseRecord.parseNormalRecordToOTSLineOfTimeseriesTable(conf.getAttributeColumn(), + record, conf.getTimeUnit()); + } + + + if (line != null) { + buffer.write(line); + } + } + + buffer.close(); + LOG.info("write end"); + } + + private PrimaryKeySchema getAutoIncrementKey() { + TableMeta tableMeta = null; + try { + tableMeta = RetryHelper.executeWithRetry( + new GetTableMetaCallable(ots, conf.getTableName()), + conf.getRetry(), + conf.getSleepInMillisecond() + ); + } catch (Exception e) { + throw new RuntimeException(e); + } + for (PrimaryKeySchema primaryKeySchema : tableMeta.getPrimaryKeyList()) { + if(primaryKeySchema.hasOption()){ + return primaryKeySchema; + } + } + return null; + } + + public void setOts(SyncClientInterface ots){ + this.ots = ots; + } + + public OTSConf getConf() { + return conf; + } + + public void setConf(OTSConf conf) { + this.conf = conf; + } + + public void setBuffer(OTSSendBuffer buffer) { + this.buffer = buffer; + } + + public void setPkColumnMapping(Map pkColumnMapping) { + this.pkColumnMapping = pkColumnMapping; + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxy.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyOld.java similarity index 77% rename from otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxy.java rename to otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyOld.java index 762edfb4..625925f1 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxy.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/OtsWriterSlaveProxyOld.java @@ -1,7 +1,16 @@ package com.alibaba.datax.plugin.writer.otswriter; -import com.alibaba.datax.plugin.writer.otswriter.model.*; -import com.alibaba.datax.plugin.writer.otswriter.utils.Common; +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.plugin.RecordReceiver; +import com.alibaba.datax.common.plugin.TaskPluginCollector; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSConf; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSConst; +import com.alibaba.datax.plugin.writer.otswriter.utils.WriterRetryPolicy; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage; +import com.alibaba.datax.plugin.writer.otswriter.utils.WithRecord; +import com.alibaba.datax.plugin.writer.otswriter.utils.CommonOld; +import com.alibaba.datax.plugin.writer.otswriter.utils.GsonParser; import com.aliyun.openservices.ots.*; import com.aliyun.openservices.ots.internal.OTSCallback; import com.aliyun.openservices.ots.internal.writer.WriterConfig; @@ -10,19 +19,13 @@ import org.apache.commons.math3.util.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alibaba.datax.common.element.Record; -import com.alibaba.datax.common.plugin.RecordReceiver; -import com.alibaba.datax.common.plugin.TaskPluginCollector; -import com.alibaba.datax.common.util.Configuration; -import com.alibaba.datax.plugin.writer.otswriter.utils.GsonParser; - import java.util.List; import java.util.concurrent.Executors; -public class OtsWriterSlaveProxy { +public class OtsWriterSlaveProxyOld implements IOtsWriterSlaveProxy { - private static final Logger LOG = LoggerFactory.getLogger(OtsWriterSlaveProxy.class); + private static final Logger LOG = LoggerFactory.getLogger(OtsWriterSlaveProxyOld.class); private OTSConf conf; private OTSAsync otsAsync; private OTSWriter otsWriter; @@ -54,14 +57,16 @@ public class OtsWriterSlaveProxy { } } + @Override public void init(Configuration configuration) { conf = GsonParser.jsonToConf(configuration.getString(OTSConst.OTS_CONF)); - + ClientConfiguration clientConfigure = new ClientConfiguration(); clientConfigure.setIoThreadCount(conf.getIoThreadCount()); clientConfigure.setMaxConnections(conf.getConcurrencyWrite()); clientConfigure.setSocketTimeoutInMillisecond(conf.getSocketTimeout()); - clientConfigure.setConnectionTimeoutInMillisecond(conf.getConnectTimeout()); + // TODO + clientConfigure.setConnectionTimeoutInMillisecond(10000); OTSServiceConfiguration otsConfigure = new OTSServiceConfiguration(); otsConfigure.setRetryStrategy(new WriterRetryPolicy(conf)); @@ -75,39 +80,44 @@ public class OtsWriterSlaveProxy { otsConfigure); } + @Override public void close() { otsAsync.shutdown(); } - - public void write(RecordReceiver recordReceiver, TaskPluginCollector collector) throws Exception { + + @Override + public void write(RecordReceiver recordReceiver, TaskPluginCollector collector) throws OTSCriticalException { LOG.info("Writer slave started."); WriterConfig writerConfig = new WriterConfig(); writerConfig.setConcurrency(conf.getConcurrencyWrite()); writerConfig.setMaxBatchRowsCount(conf.getBatchWriteCount()); - writerConfig.setMaxBatchSize(conf.getRestrictConf().getRequestTotalSizeLimition()); - writerConfig.setBufferSize(conf.getBufferSize()); - writerConfig.setMaxAttrColumnSize(conf.getRestrictConf().getAttributeColumnSize()); - writerConfig.setMaxColumnsCount(conf.getRestrictConf().getMaxColumnsCount()); - writerConfig.setMaxPKColumnSize(conf.getRestrictConf().getPrimaryKeyColumnSize()); + // TODO + writerConfig.setMaxBatchSize(1024 * 1024); + writerConfig.setBufferSize(1024); + writerConfig.setMaxAttrColumnSize(2 * 1024 * 1024); + writerConfig.setMaxColumnsCount(1024); + writerConfig.setMaxPKColumnSize(1024); + otsWriter = new DefaultOTSWriter(otsAsync, conf.getTableName(), writerConfig, new WriterCallback(collector), Executors.newFixedThreadPool(3)); int expectColumnCount = conf.getPrimaryKeyColumn().size() + conf.getAttributeColumn().size(); Record record; while ((record = recordReceiver.getFromReader()) != null) { LOG.debug("Record Raw: {}", record.toString()); - + int columnCount = record.getColumnNumber(); if (columnCount != expectColumnCount) { // 如果Column的个数和预期的个数不一致时,认为是系统故障或者用户配置Column错误,异常退出 - throw new IllegalArgumentException(String.format(OTSErrorMessage.RECORD_AND_COLUMN_SIZE_ERROR, columnCount, expectColumnCount)); + throw new IllegalArgumentException(String.format(OTSErrorMessage.RECORD_AND_COLUMN_SIZE_ERROR, columnCount, expectColumnCount, record.toString())); } - + + // 类型转换 try { - RowPrimaryKey primaryKey = Common.getPKFromRecord(conf.getPrimaryKeyColumn(), record); - List> attributes = Common.getAttrFromRecord(conf.getPrimaryKeyColumn().size(), conf.getAttributeColumn(), record); - RowChange rowChange = Common.columnValuesToRowChange(conf.getTableName(), conf.getOperation(), primaryKey, attributes); + RowPrimaryKey primaryKey = CommonOld.getPKFromRecord(conf.getPrimaryKeyColumn(), record); + List> attributes = CommonOld.getAttrFromRecord(conf.getPrimaryKeyColumn().size(), conf.getAttributeColumn(), record); + RowChange rowChange = CommonOld.columnValuesToRowChange(conf.getTableName(), conf.getOperation(), primaryKey, attributes); WithRecord withRecord = (WithRecord)rowChange; withRecord.setRecord(record); otsWriter.addRowChange(rowChange); diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/BatchWriteRowCallable.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/BatchWriteRowCallable.java new file mode 100644 index 00000000..f7330937 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/BatchWriteRowCallable.java @@ -0,0 +1,25 @@ +package com.alibaba.datax.plugin.writer.otswriter.callable; + +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.BatchWriteRowRequest; +import com.alicloud.openservices.tablestore.model.BatchWriteRowResponse; + +import java.util.concurrent.Callable; + +public class BatchWriteRowCallable implements Callable{ + + private SyncClientInterface ots = null; + private BatchWriteRowRequest batchWriteRowRequest = null; + + public BatchWriteRowCallable(SyncClientInterface ots, BatchWriteRowRequest batchWriteRowRequest) { + this.ots = ots; + this.batchWriteRowRequest = batchWriteRowRequest; + + } + + @Override + public BatchWriteRowResponse call() throws Exception { + return ots.batchWriteRow(batchWriteRowRequest); + } + +} \ No newline at end of file diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/GetTableMetaCallable.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/GetTableMetaCallable.java index d4128e14..b3b26d76 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/GetTableMetaCallable.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/GetTableMetaCallable.java @@ -1,29 +1,27 @@ package com.alibaba.datax.plugin.writer.otswriter.callable; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.DescribeTableRequest; +import com.alicloud.openservices.tablestore.model.DescribeTableResponse; +import com.alicloud.openservices.tablestore.model.TableMeta; + import java.util.concurrent.Callable; -import com.aliyun.openservices.ots.OTSClient; -import com.aliyun.openservices.ots.model.DescribeTableRequest; -import com.aliyun.openservices.ots.model.DescribeTableResult; -import com.aliyun.openservices.ots.model.TableMeta; - public class GetTableMetaCallable implements Callable{ - private OTSClient ots = null; + private SyncClientInterface ots = null; private String tableName = null; - public GetTableMetaCallable(OTSClient ots, String tableName) { + public GetTableMetaCallable(SyncClientInterface ots, String tableName) { this.ots = ots; this.tableName = tableName; } @Override public TableMeta call() throws Exception { - DescribeTableRequest describeTableRequest = new DescribeTableRequest(); - describeTableRequest.setTableName(tableName); - DescribeTableResult result = ots.describeTable(describeTableRequest); - TableMeta tableMeta = result.getTableMeta(); - return tableMeta; + DescribeTableRequest describeTableRequest = new DescribeTableRequest(tableName); + DescribeTableResponse result = ots.describeTable(describeTableRequest); + return result.getTableMeta(); } } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/GetTableMetaCallableOld.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/GetTableMetaCallableOld.java new file mode 100644 index 00000000..af7d5088 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/GetTableMetaCallableOld.java @@ -0,0 +1,29 @@ +package com.alibaba.datax.plugin.writer.otswriter.callable; + +import com.aliyun.openservices.ots.OTSClient; +import com.aliyun.openservices.ots.model.DescribeTableRequest; +import com.aliyun.openservices.ots.model.DescribeTableResult; +import com.aliyun.openservices.ots.model.TableMeta; + +import java.util.concurrent.Callable; + +public class GetTableMetaCallableOld implements Callable{ + + private OTSClient ots = null; + private String tableName = null; + + public GetTableMetaCallableOld(OTSClient ots, String tableName) { + this.ots = ots; + this.tableName = tableName; + } + + @Override + public TableMeta call() throws Exception { + DescribeTableRequest describeTableRequest = new DescribeTableRequest(); + describeTableRequest.setTableName(tableName); + DescribeTableResult result = ots.describeTable(describeTableRequest); + TableMeta tableMeta = result.getTableMeta(); + return tableMeta; + } + +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/PutRowChangeCallable.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/PutRowChangeCallable.java new file mode 100644 index 00000000..b3857094 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/PutRowChangeCallable.java @@ -0,0 +1,24 @@ +package com.alibaba.datax.plugin.writer.otswriter.callable; + +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.PutRowRequest; +import com.alicloud.openservices.tablestore.model.PutRowResponse; + +import java.util.concurrent.Callable; + +public class PutRowChangeCallable implements Callable{ + + private SyncClientInterface ots = null; + private PutRowRequest putRowRequest = null; + + public PutRowChangeCallable(SyncClientInterface ots, PutRowRequest putRowRequest) { + this.ots = ots; + this.putRowRequest = putRowRequest; + } + + @Override + public PutRowResponse call() throws Exception { + return ots.putRow(putRowRequest); + } + +} \ No newline at end of file diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/PutTimeseriesDataCallable.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/PutTimeseriesDataCallable.java new file mode 100644 index 00000000..664f4b41 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/PutTimeseriesDataCallable.java @@ -0,0 +1,22 @@ +package com.alibaba.datax.plugin.writer.otswriter.callable; + +import com.alicloud.openservices.tablestore.TimeseriesClient; +import com.alicloud.openservices.tablestore.model.timeseries.PutTimeseriesDataRequest; +import com.alicloud.openservices.tablestore.model.timeseries.PutTimeseriesDataResponse; + +import java.util.concurrent.Callable; + +public class PutTimeseriesDataCallable implements Callable { + private TimeseriesClient client = null; + private PutTimeseriesDataRequest putTimeseriesDataRequest = null; + + public PutTimeseriesDataCallable(TimeseriesClient client, PutTimeseriesDataRequest putTimeseriesDataRequest) { + this.client = client; + this.putTimeseriesDataRequest = putTimeseriesDataRequest; + } + + @Override + public PutTimeseriesDataResponse call() throws Exception { + return client.putTimeseriesData(putTimeseriesDataRequest); + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/UpdateRowChangeCallable.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/UpdateRowChangeCallable.java new file mode 100644 index 00000000..c302e3a1 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/callable/UpdateRowChangeCallable.java @@ -0,0 +1,24 @@ +package com.alibaba.datax.plugin.writer.otswriter.callable; + +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.model.UpdateRowRequest; +import com.alicloud.openservices.tablestore.model.UpdateRowResponse; + +import java.util.concurrent.Callable; + +public class UpdateRowChangeCallable implements Callable{ + + private SyncClientInterface ots = null; + private UpdateRowRequest updateRowRequest = null; + + public UpdateRowChangeCallable(SyncClientInterface ots, UpdateRowRequest updateRowRequest ) { + this.ots = ots; + this.updateRowRequest = updateRowRequest; + } + + @Override + public UpdateRowResponse call() throws Exception { + return ots.updateRow(updateRowRequest); + } + +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/LogExceptionManager.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/LogExceptionManager.java deleted file mode 100644 index 93175ddb..00000000 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/LogExceptionManager.java +++ /dev/null @@ -1,58 +0,0 @@ -package com.alibaba.datax.plugin.writer.otswriter.model; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.aliyun.openservices.ots.OTSErrorCode; -import com.aliyun.openservices.ots.OTSException; - -/** - * 添加这个类的主要目的是为了解决当用户遇到CU不够时,打印大量的日志 - * @author redchen - * - */ -public class LogExceptionManager { - - private long count = 0; - private long updateTimestamp = 0; - - private static final Logger LOG = LoggerFactory.getLogger(LogExceptionManager.class); - - private synchronized void countAndReset() { - count++; - long cur = System.currentTimeMillis(); - long interval = cur - updateTimestamp; - if (interval >= 10000) { - LOG.warn("Call callable fail, OTSNotEnoughCapacityUnit, total times:"+ count +", time range:"+ (interval/1000) +"s, times per second:" + ((float)count / (interval/1000))); - count = 0; - updateTimestamp = cur; - } - } - - public synchronized void addException(Exception exception) { - if (exception instanceof OTSException) { - OTSException e = (OTSException)exception; - if (e.getErrorCode().equals(OTSErrorCode.NOT_ENOUGH_CAPACITY_UNIT)) { - countAndReset(); - } else { - LOG.warn( - "Call callable fail, OTSException:ErrorCode:{}, ErrorMsg:{}, RequestId:{}", - new Object[]{e.getErrorCode(), e.getMessage(), e.getRequestId()} - ); - } - } else { - LOG.warn("Call callable fail, {}", exception.getMessage()); - } - } - - public synchronized void addException(com.aliyun.openservices.ots.model.Error error, String requestId) { - if (error.getCode().equals(OTSErrorCode.NOT_ENOUGH_CAPACITY_UNIT)) { - countAndReset(); - } else { - LOG.warn( - "OTSException:ErrorCode:{}, ErrorMsg:{}, RequestId:{}", - new Object[]{error.getCode(), error.getMessage(), requestId} - ); - } - } -} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSAttrColumn.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSAttrColumn.java index d37960e0..7564130a 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSAttrColumn.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSAttrColumn.java @@ -1,16 +1,33 @@ package com.alibaba.datax.plugin.writer.otswriter.model; -import com.aliyun.openservices.ots.model.ColumnType; +import com.alicloud.openservices.tablestore.model.ColumnType; + public class OTSAttrColumn { - private String name; - private ColumnType type; + // 该字段只在多版本中使用,表示多版本中,输入源中columnName的值,由将对应的Cell写入用户配置name的列中 + private String srcName = null; + private String name = null; + private ColumnType type = null; + //该字段只在写入时序表时使用,该字段是否为时序数据的标签内部字段 + private Boolean isTag = false; public OTSAttrColumn(String name, ColumnType type) { this.name = name; this.type = type; } + public OTSAttrColumn(String srcName, String name, ColumnType type) { + this.srcName = srcName; + this.name = name; + this.type = type; + } + + public OTSAttrColumn(String name, ColumnType type, Boolean isTag) { + this.name = name; + this.type = type; + this.isTag = isTag; + } + public String getName() { return name; } @@ -18,4 +35,12 @@ public class OTSAttrColumn { public ColumnType getType() { return type; } + + public String getSrcName() { + return srcName; + } + + public Boolean getTag() { + return isTag; + } } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBatchWriteRowTaskManager.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBatchWriteRowTaskManager.java new file mode 100644 index 00000000..fdeb2825 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBatchWriteRowTaskManager.java @@ -0,0 +1,41 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +import com.alicloud.openservices.tablestore.SyncClientInterface; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * 控制Task的并发数目 + * + */ +public class OTSBatchWriteRowTaskManager implements OTSTaskManagerInterface { + + private SyncClientInterface ots = null; + private OTSBlockingExecutor executorService = null; + private OTSConf conf = null; + + private static final Logger LOG = LoggerFactory.getLogger(OTSBatchWriteRowTaskManager.class); + + public OTSBatchWriteRowTaskManager( + SyncClientInterface ots, + OTSConf conf) { + this.ots = ots; + this.conf = conf; + + executorService = new OTSBlockingExecutor(conf.getConcurrencyWrite()); + } + + public void execute(List lines) throws Exception { + LOG.debug("Begin execute."); + executorService.execute(new OTSBatchWriterRowTask(ots, conf, lines)); + LOG.debug("End execute."); + } + + public void close() throws Exception { + LOG.debug("Begin close."); + executorService.shutdown(); + LOG.debug("End close."); + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBatchWriterRowTask.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBatchWriterRowTask.java new file mode 100644 index 00000000..416526fd --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBatchWriterRowTask.java @@ -0,0 +1,196 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +import com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException; +import com.alibaba.datax.plugin.writer.otswriter.OTSErrorCode; +import com.alibaba.datax.plugin.writer.otswriter.callable.BatchWriteRowCallable; +import com.alibaba.datax.plugin.writer.otswriter.callable.PutRowChangeCallable; +import com.alibaba.datax.plugin.writer.otswriter.callable.UpdateRowChangeCallable; +import com.alibaba.datax.plugin.writer.otswriter.utils.CollectorUtil; +import com.alibaba.datax.plugin.writer.otswriter.utils.Common; +import com.alibaba.datax.plugin.writer.otswriter.utils.LineAndError; +import com.alibaba.datax.plugin.writer.otswriter.utils.RetryHelper; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.TableStoreException; +import com.alicloud.openservices.tablestore.model.*; +import com.alicloud.openservices.tablestore.model.BatchWriteRowResponse.RowResult; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +public class OTSBatchWriterRowTask implements Runnable { + private SyncClientInterface ots = null; + private OTSConf conf = null; + private List otsLines = new ArrayList(); + + private boolean isDone = false; + private int retryTimes = 0; + + private static final Logger LOG = LoggerFactory.getLogger(OTSBatchWriterRowTask.class); + + public OTSBatchWriterRowTask( + final SyncClientInterface ots, + final OTSConf conf, + final List lines + ) { + this.ots = ots; + this.conf = conf; + + this.otsLines.addAll(lines); + } + + @Override + public void run() { + LOG.debug("Begin run"); + sendAll(otsLines); + LOG.debug("End run"); + } + + public boolean isDone() { + return this.isDone; + } + + private boolean isExceptionForSendOneByOne(TableStoreException ee) { + if (ee.getErrorCode().equals(OTSErrorCode.INVALID_PARAMETER)|| + ee.getErrorCode().equals(OTSErrorCode.REQUEST_TOO_LARGE) + ) { + return true; + } + return false; + } + + private BatchWriteRowRequest createRequest(List lines) { + BatchWriteRowRequest newRequest = new BatchWriteRowRequest(); + switch (conf.getOperation()) { + case PUT_ROW: + case UPDATE_ROW: + for (OTSLine l : lines) { + newRequest.addRowChange(l.getRowChange()); + } + break; + default: + throw new RuntimeException(String.format(OTSErrorMessage.OPERATION_PARSE_ERROR, conf.getOperation())); + } + return newRequest; + } + + /** + * 单行发送数据 + * @param line + */ + public void sendLine(OTSLine line) { + try { + switch (conf.getOperation()) { + case PUT_ROW: + PutRowRequest putRowRequest = new PutRowRequest(); + putRowRequest.setRowChange((RowPutChange) line.getRowChange()); + PutRowResponse putResult = RetryHelper.executeWithRetry( + new PutRowChangeCallable(ots, putRowRequest), + conf.getRetry(), + conf.getSleepInMillisecond()); + LOG.debug("Requst ID : {}", putResult.getRequestId()); + break; + case UPDATE_ROW: + UpdateRowRequest updateRowRequest = new UpdateRowRequest(); + updateRowRequest.setRowChange((RowUpdateChange) line.getRowChange()); + UpdateRowResponse updateResult = RetryHelper.executeWithRetry( + new UpdateRowChangeCallable(ots, updateRowRequest), + conf.getRetry(), + conf.getSleepInMillisecond()); + LOG.debug("Requst ID : {}", updateResult.getRequestId()); + break; + } + } catch (Exception e) { + LOG.warn("sendLine fail. ", e); + CollectorUtil.collect(line.getRecords(), e.getMessage()); + } + } + + private void sendAllOneByOne(List lines) { + for (OTSLine l : lines) { + sendLine(l); + } + } + + /** + * 批量发送数据 + * 如果程序发送失败,BatchWriteRow接口可能整体异常返回或者返回每个子行的操作状态 + * 1.在整体异常的情况下:方法会检查这个异常是否能通过把批量数据拆分成单行发送,如果不行, + * 将会把这一批数据记录到脏数据回收器中,如果可以,方法会调用sendAllOneByOne进行单行数据发送。 + * 2.如果BatchWriteRow成功执行,方法会检查每行的返回状态,如果子行操作失败,方法会收集所有失 + * 败的行,重新调用sendAll,发送失败的数据。 + * @param lines + */ + private void sendAll(List lines) { + try { + Thread.sleep(Common.getDelaySendMillinSeconds(retryTimes, conf.getSleepInMillisecond())); + BatchWriteRowRequest batchWriteRowRequest = createRequest(lines); + BatchWriteRowResponse result = RetryHelper.executeWithRetry( + new BatchWriteRowCallable(ots, batchWriteRowRequest), + conf.getRetry(), + conf.getSleepInMillisecond()); + + LOG.debug("Requst ID : {}", result.getRequestId()); + List errors = getLineAndError(result, lines); + if (!errors.isEmpty()){ + if(retryTimes < conf.getRetry()) { + retryTimes++; + LOG.warn("Retry times : {}", retryTimes); + List newLines = new ArrayList(); + for (LineAndError re : errors) { + LOG.warn("Because: {}", re.getError().getMessage()); + if (RetryHelper.canRetry(re.getError().getCode())) { + newLines.add(re.getLine()); + } else { + LOG.warn("Can not retry, record row to collector. {}", re.getError().getMessage()); + CollectorUtil.collect(re.getLine().getRecords(), re.getError().getMessage()); + } + } + if (!newLines.isEmpty()) { + sendAll(newLines); + } + } else { + LOG.warn("Retry times more than limitation. RetryTime : {}", retryTimes); + CollectorUtil.collect(errors); + } + } + } catch (TableStoreException e) { + LOG.warn("Send data fail. {}", e.getMessage()); + if (isExceptionForSendOneByOne(e)) { + if (lines.size() == 1) { + LOG.warn("Can not retry.", e); + CollectorUtil.collect(e.getMessage(), lines); + } else { + // 进入单行发送的分支 + sendAllOneByOne(lines); + } + } else { + LOG.error("Can not send lines to OTS for RuntimeException.", e); + CollectorUtil.collect(e.getMessage(), lines); + } + } catch (Exception e) { + LOG.error("Can not send lines to OTS for Exception.", e); + CollectorUtil.collect(e.getMessage(), lines); + } + } + + private List getLineAndError(BatchWriteRowResponse result, List lines) throws OTSCriticalException { + List errors = new ArrayList(); + + switch(conf.getOperation()) { + case PUT_ROW: + case UPDATE_ROW: { + List status = result.getFailedRows(); + for (RowResult r : status) { + errors.add(new LineAndError(lines.get(r.getIndex()), r.getError())); + } + } + break; + default: + LOG.error("Bug branch."); + throw new OTSCriticalException(String.format(OTSErrorMessage.OPERATION_PARSE_ERROR, conf.getOperation())); + } + return errors; + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBlockingExecutor.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBlockingExecutor.java new file mode 100644 index 00000000..059ba338 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSBlockingExecutor.java @@ -0,0 +1,55 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.*; + +/** + * 单个Channel会多线程并发的写入数据到OTS中,需要使用一个固定的线程池来执行Runnable对象,同时当 + * 线程池满时,阻塞execute方法。原生的Executor并不能做到阻塞execute方法。只是当queue满时, + * 方法抛出默认RejectedExecutionException,或者我们实现RejectedExecutionHandler, + * 这两种方法都无法满足阻塞用户请求的需求,所以我们用信号量来实现了一个阻塞的Executor + * @author redchen + * + */ +public class OTSBlockingExecutor { + private final ExecutorService exec; + private final Semaphore semaphore; + + private static final Logger LOG = LoggerFactory.getLogger(OTSBlockingExecutor.class); + + public OTSBlockingExecutor(int concurrency) { + this.exec = new ThreadPoolExecutor( + concurrency, concurrency, + 0L, TimeUnit.SECONDS, + new LinkedBlockingQueue()); + this.semaphore = new Semaphore(concurrency); + } + + public void execute(final Runnable task) + throws InterruptedException { + LOG.debug("Begin execute"); + try { + semaphore.acquire(); + exec.execute(new Runnable() { + public void run() { + try { + task.run(); + } finally { + semaphore.release(); + } + } + }); + } catch (RejectedExecutionException e) { + semaphore.release(); + throw new RuntimeException(OTSErrorMessage.INSERT_TASK_ERROR); + } + LOG.debug("End execute"); + } + + public void shutdown() throws InterruptedException { + this.exec.shutdown(); + while (!this.exec.awaitTermination(1, TimeUnit.SECONDS)){} + } +} \ No newline at end of file diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSConf.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSConf.java index bd7eccc5..fee9ed55 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSConf.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSConf.java @@ -1,73 +1,51 @@ package com.alibaba.datax.plugin.writer.otswriter.model; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; + import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; public class OTSConf { - private String endpoint; - private String accessId; - private String accessKey; - private String instanceName; - private String tableName; + private String endpoint= null; + private String accessId = null; + private String accessKey = null; + private String instanceName = null; + private String tableName = null; + - private List primaryKeyColumn; - private List attributeColumn; - - private int bufferSize = 1024; - private int retry = 18; - private int sleepInMillisecond = 100; - private int batchWriteCount = 10; - private int concurrencyWrite = 5; - private int ioThreadCount = 1; - private int socketTimeout = 20000; - private int connectTimeout = 10000; + private List primaryKeyColumn = null; + private List attributeColumn = null; + + private int retry = -1; + private int sleepInMillisecond = -1; + private int batchWriteCount = -1; + private int concurrencyWrite = -1; + private int ioThreadCount = -1; + private int socketTimeoutInMillisecond = -1; + private int connectTimeoutInMillisecond = -1; - private OTSOpType operation; - private RestrictConf restrictConf; + private OTSOpType operation = null; - //限制项 - public class RestrictConf { - private int requestTotalSizeLimition = 1024 * 1024; - private int primaryKeyColumnSize = 1024; - private int attributeColumnSize = 2 * 1024 * 1024; - private int maxColumnsCount = 1024; + private int requestTotalSizeLimitation = -1; + + private OTSMode mode = null; + private boolean enableAutoIncrement = false; + private boolean isNewVersion = false; + private boolean isTimeseriesTable = false; + private TimeUnit timeUnit = TimeUnit.MICROSECONDS; + private long timestamp = -1; + private Map encodePkColumnMapping = null; + private String columnNamePrefixFilter = null; - public int getRequestTotalSizeLimition() { - return requestTotalSizeLimition; - } - public void setRequestTotalSizeLimition(int requestTotalSizeLimition) { - this.requestTotalSizeLimition = requestTotalSizeLimition; - } - - public void setPrimaryKeyColumnSize(int primaryKeyColumnSize) { - this.primaryKeyColumnSize = primaryKeyColumnSize; - } - - public void setAttributeColumnSize(int attributeColumnSize) { - this.attributeColumnSize = attributeColumnSize; - } - - public void setMaxColumnsCount(int maxColumnsCount) { - this.maxColumnsCount = maxColumnsCount; - } - - public int getAttributeColumnSize() { - return attributeColumnSize; - } - - public int getMaxColumnsCount() { - return maxColumnsCount; - } - - public int getPrimaryKeyColumnSize() { - return primaryKeyColumnSize; - } + public Map getEncodePkColumnMapping() { + return encodePkColumnMapping; } - - public RestrictConf getRestrictConf() { - return restrictConf; + public void setEncodePkColumnMapping(Map encodePkColumnMapping) { + this.encodePkColumnMapping = encodePkColumnMapping; } - public void setRestrictConf(RestrictConf restrictConf) { - this.restrictConf = restrictConf; + public int getSocketTimeoutInMillisecond() { + return socketTimeoutInMillisecond; } public OTSOpType getOperation() { return operation; @@ -75,10 +53,10 @@ public class OTSConf { public void setOperation(OTSOpType operation) { this.operation = operation; } - public List getPrimaryKeyColumn() { + public List getPrimaryKeyColumn() { return primaryKeyColumn; } - public void setPrimaryKeyColumn(List primaryKeyColumn) { + public void setPrimaryKeyColumn(List primaryKeyColumn) { this.primaryKeyColumn = primaryKeyColumn; } @@ -149,24 +127,72 @@ public class OTSConf { this.ioThreadCount = ioThreadCount; } public int getSocketTimeout() { - return socketTimeout; + return socketTimeoutInMillisecond; } - public void setSocketTimeout(int socketTimeout) { - this.socketTimeout = socketTimeout; + public void setSocketTimeoutInMillisecond(int socketTimeoutInMillisecond) { + this.socketTimeoutInMillisecond = socketTimeoutInMillisecond; } - public int getConnectTimeout() { - return connectTimeout; + public int getConnectTimeoutInMillisecond() { + return connectTimeoutInMillisecond; + } + public void setConnectTimeoutInMillisecond(int connectTimeoutInMillisecond) { + this.connectTimeoutInMillisecond = connectTimeoutInMillisecond; + } + public OTSMode getMode() { + return mode; + } + public void setMode(OTSMode mode) { + this.mode = mode; + } + public long getTimestamp() { + return timestamp; + } + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + public String getColumnNamePrefixFilter() { + return columnNamePrefixFilter; + } + public void setColumnNamePrefixFilter(String columnNamePrefixFilter) { + this.columnNamePrefixFilter = columnNamePrefixFilter; } - public int getBufferSize() { - return bufferSize; + public boolean getEnableAutoIncrement() { + return enableAutoIncrement; } - public void setBufferSize(int bufferSize) { - this.bufferSize = bufferSize; + public void setEnableAutoIncrement(boolean enableAutoIncrement) { + this.enableAutoIncrement = enableAutoIncrement; + } + public boolean isNewVersion() { + return isNewVersion; } - public void setConnectTimeout(int connectTimeout) { - this.connectTimeout = connectTimeout; + public void setNewVersion(boolean newVersion) { + isNewVersion = newVersion; + } + + public boolean isTimeseriesTable() { + return isTimeseriesTable; + } + + public void setTimeseriesTable(boolean timeseriesTable) { + isTimeseriesTable = timeseriesTable; + } + + public TimeUnit getTimeUnit() { + return timeUnit; + } + + public void setTimeUnit(TimeUnit timeUnit) { + this.timeUnit = timeUnit; + } + + public int getRequestTotalSizeLimitation() { + return requestTotalSizeLimitation; + } + + public void setRequestTotalSizeLimitation(int requestTotalSizeLimitation) { + this.requestTotalSizeLimitation = requestTotalSizeLimitation; } } \ No newline at end of file diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSConst.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSConst.java index 1b8f8053..bda736e8 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSConst.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSConst.java @@ -2,20 +2,27 @@ package com.alibaba.datax.plugin.writer.otswriter.model; public class OTSConst { // Reader support type - public final static String TYPE_STRING = "STRING"; + public final static String TYPE_STRING = "STRING"; public final static String TYPE_INTEGER = "INT"; - public final static String TYPE_DOUBLE = "DOUBLE"; + public final static String TYPE_DOUBLE = "DOUBLE"; public final static String TYPE_BOOLEAN = "BOOL"; - public final static String TYPE_BINARY = "BINARY"; - + public final static String TYPE_BINARY = "BINARY"; + // Column public final static String NAME = "name"; + public final static String SRC_NAME = "srcName"; public final static String TYPE = "type"; - + public final static String IS_TAG = "is_timeseries_tag"; + public final static String OTS_CONF = "OTS_CONF"; - + + public final static String OTS_MODE_NORMAL = "normal"; + public final static String OTS_MODE_MULTI_VERSION = "multiVersion"; + public final static String OTS_MODE_TIME_SERIES = "timeseries"; + public final static String OTS_OP_TYPE_PUT = "PutRow"; public final static String OTS_OP_TYPE_UPDATE = "UpdateRow"; + // only support in old version public final static String OTS_OP_TYPE_DELETE = "DeleteRow"; // options @@ -24,13 +31,13 @@ public class OTSConst { public final static String BATCH_WRITE_COUNT = "batchWriteCount"; public final static String CONCURRENCY_WRITE = "concurrencyWrite"; public final static String IO_THREAD_COUNT = "ioThreadCount"; - public final static String SOCKET_TIMEOUT = "socketTimeoutInMillisecond"; - public final static String CONNECT_TIMEOUT = "connectTimeoutInMillisecond"; - public final static String BUFFER_SIZE = "bufferSize"; - - // 限制项 + public final static String MAX_CONNECT_COUNT = "maxConnectCount"; + public final static String SOCKET_TIMEOUTIN_MILLISECOND = "socketTimeoutInMillisecond"; + public final static String CONNECT_TIMEOUT_IN_MILLISECOND = "connectTimeoutInMillisecond"; public final static String REQUEST_TOTAL_SIZE_LIMITATION = "requestTotalSizeLimitation"; - public final static String ATTRIBUTE_COLUMN_SIZE_LIMITATION = "attributeColumnSizeLimitation"; - public final static String PRIMARY_KEY_COLUMN_SIZE_LIMITATION = "primaryKeyColumnSizeLimitation"; - public final static String ATTRIBUTE_COLUMN_MAX_COUNT = "attributeColumnMaxCount"; -} \ No newline at end of file + + public static final String MEASUREMENT_NAME = "_m_name"; + public static final String DATA_SOURCE = "_data_source"; + public static final String TAGS = "_tags"; + public static final String TIME = "_time"; +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSErrorMessage.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSErrorMessage.java index 9523342f..4bde553a 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSErrorMessage.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSErrorMessage.java @@ -2,13 +2,19 @@ package com.alibaba.datax.plugin.writer.otswriter.model; public class OTSErrorMessage { - public static final String OPERATION_PARSE_ERROR = "The 'writeMode' only support 'PutRow', 'UpdateRow' or 'DeleteRow', not '%s'."; + public static final String MODE_PARSE_ERROR = "The 'mode' only support 'normal' and 'multiVersion' not '%s'."; + + public static final String OPERATION_PARSE_ERROR = "The 'writeMode' only support 'PutRow' and 'UpdateRow' not '%s'."; + + public static final String MUTLI_MODE_OPERATION_PARSE_ERROR = "When configurion set mode='MultiVersion', the 'writeMode' only support 'UpdateRow' not '%s'."; public static final String UNSUPPORT_PARSE = "Unsupport parse '%s' to '%s'."; - public static final String RECORD_AND_COLUMN_SIZE_ERROR = "Size of record not equal size of config column. record size : %d, config column size : %d."; + public static final String UNSUPPORT = "Unsupport : '%s'."; - public static final String PK_TYPE_ERROR = "Primary key type only support 'string' and 'int', not support '%s'."; + public static final String RECORD_AND_COLUMN_SIZE_ERROR = "Size of record not equal size of config column. record size : %d, config column size : %d, record data : %s."; + + public static final String PK_TYPE_ERROR = "Primary key type only support 'string', 'int' and 'binary', not support '%s'."; public static final String ATTR_TYPE_ERROR = "Column type only support 'string','int','double','bool' and 'binary', not support '%s'."; @@ -17,7 +23,9 @@ public class OTSErrorMessage { public static final String INPUT_PK_COUNT_NOT_EQUAL_META_ERROR = "The count of 'primaryKey' not equal meta, input count : %d, primary key count : %d in meta."; public static final String INPUT_PK_TYPE_NOT_MATCH_META_ERROR = "The type of 'primaryKey' not match meta, column name : %s, input type: %s, primary key type : %s in meta."; - + + public static final String INPUT_PK_NAME_NOT_EXIST_IN_META_ERROR = "The input primary column '%s' is not exist in meta."; + public static final String ATTR_REPEAT_COLUMN_ERROR = "Repeat column '%s' in 'column'."; public static final String MISSING_PARAMTER_ERROR = "The param '%s' is not exist."; @@ -36,25 +44,49 @@ public class OTSErrorMessage { public static final String ATTR_MAP_NAME_TYPE_ERROR = "The 'name' and 'type only support string in json map of 'column'."; + public static final String ATTR_MAP_SRCNAME_NAME_TYPE_ERROR = "The 'srcName', 'name' and 'type' only support string in json map of 'column'."; + + public static final String PK_MAP_KEY_TYPE_ERROR = "The '%s' only support string in json map of 'primaryKey'."; + + public static final String ATTR_MAP_KEY_TYPE_ERROR = "The '%s' only support string in json map of 'column'."; + public static final String PK_MAP_INCLUDE_NAME_TYPE_ERROR = "The only support 'name' and 'type' fileds in json map of 'primaryKey'."; public static final String ATTR_MAP_INCLUDE_NAME_TYPE_ERROR = "The only support 'name' and 'type' fileds in json map of 'column'."; - public static final String PK_ITEM_IS_NOT_MAP_ERROR = "The item is not map in 'primaryKey'."; + public static final String PK_MAP_FILED_MISSING_ERROR = "The '%s' fileds is missing in json map of 'primaryKey'."; + + public static final String ATTR_MAP_FILED_MISSING_ERROR = "The '%s' fileds is missing in json map of 'column'."; + + public static final String ATTR_MAP_INCLUDE_SRCNAME_NAME_TYPE_ERROR = "The only support 'srcName', 'name' and 'type' fileds in json map of 'column'."; + + public static final String PK_ITEM_IS_ILLEAGAL_ERROR = "The item is not string or map in 'primaryKey'."; + + public static final String PK_IS_NOT_EXIST_AT_OTS_ERROR = "Can not find the pk('%s') at ots in 'primaryKey'."; public static final String ATTR_ITEM_IS_NOT_MAP_ERROR = "The item is not map in 'column'."; public static final String PK_COLUMN_NAME_IS_EMPTY_ERROR = "The name of item can not be a empty string in 'primaryKey'."; + public static final String PK_COLUMN_TYPE_IS_EMPTY_ERROR = "The type of item can not be a empty string in 'primaryKey'."; + public static final String ATTR_COLUMN_NAME_IS_EMPTY_ERROR = "The name of item can not be a empty string in 'column'."; - public static final String MULTI_ATTR_COLUMN_ERROR = "Multi item in 'column', column name : %s ."; + public static final String ATTR_COLUMN_SRC_NAME_IS_EMPTY_ERROR = "The srcName of item can not be a empty string in 'column'."; + + public static final String ATTR_COLUMN_TYPE_IS_EMPTY_ERROR = "The type of item can not be a empty string in 'column'."; + + public static final String MULTI_PK_ATTR_COLUMN_ERROR = "Duplicate item in 'column' and 'primaryKey', column name : %s ."; + + public static final String MULTI_ATTR_COLUMN_ERROR = "Duplicate item in 'column', column name : %s ."; + + public static final String MULTI_ATTR_SRC_COLUMN_ERROR = "Duplicate src name in 'column', src name : %s ."; public static final String COLUMN_CONVERSION_ERROR = "Column coversion error, src type : %s, src value: %s, expect type: %s ."; public static final String PK_COLUMN_VALUE_IS_NULL_ERROR = "The column of record is NULL, primary key name : %s ."; - public static final String PK_STRONG_LENGTH_ERROR = "The length of pk string value is more than configuration, conf: %d, input: %d ."; + public static final String PK_STRING_LENGTH_ERROR = "The length of pk string value is more than configuration, conf: %d, input: %d ."; public static final String ATTR_STRING_LENGTH_ERROR = "The length of attr string value is more than configuration, conf: %d, input: %d ."; @@ -63,4 +95,31 @@ public class OTSErrorMessage { public static final String LINE_LENGTH_ERROR = "The length of row is more than length of request configuration, conf: %d, row: %d ."; public static final String INSERT_TASK_ERROR = "Can not execute the task, becase the ExecutorService is shutdown."; + + public static final String COLUMN_NOT_DEFINE = "The column name : '%s' not define in column."; + + public static final String INPUT_RECORDS_IS_EMPTY = "The input records can not be empty."; + + public static final String MULTI_VERSION_TIMESTAMP_IS_EMPTY = "The input timestamp can not be empty in the multiVersion mode."; + + public static final String MULTI_VERSION_VALUE_IS_EMPTY = "The input value can not be empty in the multiVersion mode."; + + public static final String INPUT_COLUMN_COUNT_LIMIT = "The input count(%d) of column more than max(%d)."; + + public static final String PUBLIC_SDK_NO_SUPPORT_MULTI_VERSION = "The old version do not support multi version function. Please add config in otswriter: \"newVersion\":\"true\" ."; + + public static final String PUBLIC_SDK_NO_SUPPORT_AUTO_INCREMENT = "The old version do not support auto increment primary key function. Please add config in otswriter: \"newVersion\":\"true\" ."; + + public static final String NOT_SUPPORT_MULTI_VERSION_AUTO_INCREMENT = "The multi version mode do not support auto increment primary key function."; + + public static final String PUBLIC_SDK_NO_SUPPORT_TIMESERIES_TABLE = "The old version do not support write timeseries table. Please add config in otswriter: \"newVersion\":\"true\" ."; + + public static final String NOT_SUPPORT_TIMESERIES_TABLE_AUTO_INCREMENT = "The timeseries table do not support auto increment primary key function."; + + public static final String NO_FOUND_M_NAME_FIELD_ERROR = "The '_m_name' field should be set in columns because 'measurement' is required in timeseries data."; + + public static final String NO_FOUND_TIME_FIELD_ERROR = "The '_time' field should be set in columns because 'time' is required in timeseries data."; + + public static final String TIMEUNIT_FORMAT_ERROR = "The value of param 'timeunit' is '%s', which should be in ['NANOSECONDS', 'MICROSECONDS', 'MILLISECONDS', 'SECONDS', 'MINUTES']."; + } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSLine.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSLine.java new file mode 100644 index 00000000..7be4a1a8 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSLine.java @@ -0,0 +1,85 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException; +import com.alibaba.datax.plugin.writer.otswriter.utils.CalculateHelper; +import com.alicloud.openservices.tablestore.model.PrimaryKey; +import com.alicloud.openservices.tablestore.model.RowChange; +import com.alicloud.openservices.tablestore.model.RowPutChange; +import com.alicloud.openservices.tablestore.model.RowUpdateChange; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesRow; + +import java.util.ArrayList; +import java.util.List; + +public class OTSLine { + private int dataSize = 0; + + private PrimaryKey pk = null; + private RowChange change = null; + private TimeseriesRow timeseriesRow = null; + + private List records = new ArrayList(); + + public OTSLine( + PrimaryKey pk, + List records, + RowChange change) throws OTSCriticalException { + this.pk = pk; + this.change = change; + this.records.addAll(records); + setSize(this.change); + } + + public OTSLine( + PrimaryKey pk, + Record record, + RowChange change) throws OTSCriticalException { + this.pk = pk; + this.change = change; + this.records.add(record); + setSize(this.change); + } + + public OTSLine( + Record record, + TimeseriesRow row) throws OTSCriticalException { + this.timeseriesRow = row; + this.records.add(record); + setSize(this.timeseriesRow); + } + + private void setSize(RowChange change) throws OTSCriticalException { + if (change instanceof RowPutChange) { + this.dataSize = CalculateHelper.getRowPutChangeSize((RowPutChange) change); + } else if (change instanceof RowUpdateChange) { + this.dataSize = CalculateHelper.getRowUpdateChangeSize((RowUpdateChange) change); + } else { + throw new RuntimeException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, change.getClass().toString(), "RowPutChange or RowUpdateChange")); + } + } + + private void setSize(TimeseriesRow row) throws OTSCriticalException { + this.dataSize = CalculateHelper.getTimeseriesRowDataSize(row); + } + + public List getRecords() { + return records; + } + + public PrimaryKey getPk() { + return pk; + } + + public int getDataSize() { + return dataSize; + } + + public RowChange getRowChange() { + return change; + } + + public TimeseriesRow getTimeseriesRow() { + return timeseriesRow; + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSMode.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSMode.java new file mode 100644 index 00000000..530ad5de --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSMode.java @@ -0,0 +1,6 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +public enum OTSMode { + NORMAL, // 普通模式 + MULTI_VERSION // 多版本模式 +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSOpType.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSOpType.java index 17b65033..80d70d6d 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSOpType.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSOpType.java @@ -3,5 +3,6 @@ package com.alibaba.datax.plugin.writer.otswriter.model; public enum OTSOpType { PUT_ROW, UPDATE_ROW, + @Deprecated DELETE_ROW } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSPKColumn.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSPKColumn.java deleted file mode 100644 index c873cb96..00000000 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSPKColumn.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.alibaba.datax.plugin.writer.otswriter.model; - -import com.aliyun.openservices.ots.model.PrimaryKeyType; - -public class OTSPKColumn { - private String name; - private PrimaryKeyType type; - - public OTSPKColumn(String name, PrimaryKeyType type) { - this.name = name; - this.type = type; - } - - public PrimaryKeyType getType() { - return type; - } - - public String getName() { - return name; - } - -} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSRowPrimaryKey.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSRowPrimaryKey.java deleted file mode 100644 index d89d5017..00000000 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSRowPrimaryKey.java +++ /dev/null @@ -1,61 +0,0 @@ -package com.alibaba.datax.plugin.writer.otswriter.model; - -import java.util.Map; -import java.util.Map.Entry; - -import com.aliyun.openservices.ots.model.PrimaryKeyValue; - -public class OTSRowPrimaryKey { - - private Map columns; - - public OTSRowPrimaryKey(Map columns) { - if (null == columns) { - throw new IllegalArgumentException("Input columns can not be null."); - } - this.columns = columns; - } - - public Map getColumns() { - return columns; - } - - @Override - public int hashCode() { - int result = 31; - for (Entry entry : columns.entrySet()) { - result = result ^ entry.getKey().hashCode() ^ entry.getValue().hashCode(); - } - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (!(obj instanceof OTSRowPrimaryKey)) { - return false; - } - OTSRowPrimaryKey other = (OTSRowPrimaryKey) obj; - - if (columns.size() != other.columns.size()) { - return false; - } - - for (Entry entry : columns.entrySet()) { - PrimaryKeyValue otherValue = other.columns.get(entry.getKey()); - - if (otherValue == null) { - return false; - } - if (!otherValue.equals(entry.getValue())) { - return false; - } - } - return true; - } -} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSSendBuffer.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSSendBuffer.java new file mode 100644 index 00000000..f85b2c16 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSSendBuffer.java @@ -0,0 +1,82 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +import com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +public class OTSSendBuffer { + + private OTSConf conf = null; + private OTSTaskManagerInterface manager = null; + + private int totalSize = 0; + private List buffer = new ArrayList(); + + + private static final Logger LOG = LoggerFactory.getLogger(OTSSendBuffer.class); + + public OTSSendBuffer( + SyncClientInterface ots, + OTSConf conf) { + this.conf = conf; + if (conf.isTimeseriesTable()){ + this.manager = new OTSTimeseriesRowTaskManager(ots, conf); + } + else { + this.manager = new OTSBatchWriteRowTaskManager(ots, conf); + } + + } + + public void write(OTSLine line) throws OTSCriticalException { + LOG.debug("write begin"); + // 检查是否满足发送条件 + if (buffer.size() >= conf.getBatchWriteCount() || + ((totalSize + line.getDataSize()) > conf.getRequestTotalSizeLimitation() && totalSize > 0) + ) { + try { + manager.execute(new ArrayList(buffer)); + } catch (Exception e) { + LOG.error("OTSBatchWriteRowTaskManager execute fail : {}", e.getMessage(), e); + throw new OTSCriticalException(e); + } + buffer.clear(); + totalSize = 0; + } + buffer.add(line); + totalSize += line.getDataSize(); + LOG.debug("write end"); + } + + public void flush() throws OTSCriticalException { + LOG.debug("flush begin"); + if (!buffer.isEmpty()) { + try { + manager.execute(new ArrayList(buffer)); + } catch (Exception e) { + LOG.error("OTSBatchWriteRowTaskManager flush fail : {}", e.getMessage(), e); + throw new OTSCriticalException(e); + } + } + LOG.debug("flush end"); + } + + public void close() throws OTSCriticalException { + LOG.debug("close begin"); + try { + flush(); + } finally { + try { + manager.close(); + } catch (Exception e) { + LOG.error("OTSBatchWriteRowTaskManager close fail : {}", e.getMessage(), e); + throw new OTSCriticalException(e); + } + } + LOG.debug("close end"); + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTaskManagerInterface.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTaskManagerInterface.java new file mode 100644 index 00000000..5db85d7d --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTaskManagerInterface.java @@ -0,0 +1,9 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +import java.util.List; + +public interface OTSTaskManagerInterface { + public void execute(List lines) throws Exception; + + public void close() throws Exception; +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTimeseriesRowTask.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTimeseriesRowTask.java new file mode 100644 index 00000000..7cda8e33 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTimeseriesRowTask.java @@ -0,0 +1,167 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +import com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException; +import com.alibaba.datax.plugin.writer.otswriter.OTSErrorCode; +import com.alibaba.datax.plugin.writer.otswriter.callable.PutTimeseriesDataCallable; +import com.alibaba.datax.plugin.writer.otswriter.utils.CollectorUtil; +import com.alibaba.datax.plugin.writer.otswriter.utils.Common; +import com.alibaba.datax.plugin.writer.otswriter.utils.LineAndError; +import com.alibaba.datax.plugin.writer.otswriter.utils.RetryHelper; +import com.alicloud.openservices.tablestore.TableStoreException; +import com.alicloud.openservices.tablestore.TimeseriesClient; +import com.alicloud.openservices.tablestore.model.PutRowRequest; +import com.alicloud.openservices.tablestore.model.timeseries.PutTimeseriesDataRequest; +import com.alicloud.openservices.tablestore.model.timeseries.PutTimeseriesDataResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +public class OTSTimeseriesRowTask implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(OTSTimeseriesRowTask.class); + private TimeseriesClient client = null; + private OTSConf conf = null; + private List otsLines = new ArrayList(); + private boolean isDone = false; + private int retryTimes = 0; + + public OTSTimeseriesRowTask( + final TimeseriesClient client, + final OTSConf conf, + final List lines + ) { + this.client = client; + this.conf = conf; + + this.otsLines.addAll(lines); + } + + @Override + public void run() { + LOG.debug("Begin run"); + sendAll(otsLines); + LOG.debug("End run"); + } + + public boolean isDone() { + return this.isDone; + } + + private boolean isExceptionForSendOneByOne(TableStoreException ee) { + if (ee.getErrorCode().equals(OTSErrorCode.INVALID_PARAMETER) || + ee.getErrorCode().equals(OTSErrorCode.REQUEST_TOO_LARGE) + ) { + return true; + } + return false; + } + + private PutTimeseriesDataRequest createRequest(List lines) { + PutTimeseriesDataRequest newRequest = new PutTimeseriesDataRequest(conf.getTableName()); + for (OTSLine l : lines) { + newRequest.addRow(l.getTimeseriesRow()); + } + return newRequest; + } + + /** + * 单行发送数据 + * + * @param line + */ + public void sendLine(OTSLine line) { + try { + PutTimeseriesDataRequest putTimeseriesDataRequest = new PutTimeseriesDataRequest(conf.getTableName()); + putTimeseriesDataRequest.addRow(line.getTimeseriesRow()); + PutTimeseriesDataResponse result = RetryHelper.executeWithRetry( + new PutTimeseriesDataCallable(client, putTimeseriesDataRequest), + conf.getRetry(), + conf.getSleepInMillisecond()); + + + if (!result.isAllSuccess()){ + String errMsg = result.getFailedRows().get(0).getError().getMessage(); + LOG.warn("sendLine fail. " + errMsg); + CollectorUtil.collect(line.getRecords(), errMsg); + + }else { + LOG.debug("Request ID : {}", result.getRequestId()); + } + + } catch (Exception e) { + LOG.warn("sendLine fail. ", e); + CollectorUtil.collect(line.getRecords(), e.getMessage()); + } + } + + private void sendAllOneByOne(List lines) { + for (OTSLine l : lines) { + sendLine(l); + } + } + + private void sendAll(List lines) { + try { + Thread.sleep(Common.getDelaySendMillinSeconds(retryTimes, conf.getSleepInMillisecond())); + PutTimeseriesDataRequest putTimeseriesDataRequest = createRequest(lines); + PutTimeseriesDataResponse result = RetryHelper.executeWithRetry( + new PutTimeseriesDataCallable(client, putTimeseriesDataRequest), + conf.getRetry(), + conf.getSleepInMillisecond()); + + LOG.debug("Request ID : {}", result.getRequestId()); + List errors = getLineAndError(result, lines); + if (!errors.isEmpty()) { + if (retryTimes < conf.getRetry()) { + retryTimes++; + LOG.warn("Retry times : {}", retryTimes); + List newLines = new ArrayList(); + for (LineAndError re : errors) { + LOG.warn("Because: {}", re.getError().getMessage()); + if (RetryHelper.canRetry(re.getError().getCode())) { + newLines.add(re.getLine()); + } else { + LOG.warn("Can not retry, record row to collector. {}", re.getError().getMessage()); + CollectorUtil.collect(re.getLine().getRecords(), re.getError().getMessage()); + } + } + if (!newLines.isEmpty()) { + sendAll(newLines); + } + } else { + LOG.warn("Retry times more than limitation. RetryTime : {}", retryTimes); + CollectorUtil.collect(errors); + } + } + } catch (TableStoreException e) { + LOG.warn("Send data fail. {}", e.getMessage()); + if (isExceptionForSendOneByOne(e)) { + if (lines.size() == 1) { + LOG.warn("Can not retry.", e); + CollectorUtil.collect(e.getMessage(), lines); + } else { + // 进入单行发送的分支 + sendAllOneByOne(lines); + } + } else { + LOG.error("Can not send lines to OTS for RuntimeException.", e); + CollectorUtil.collect(e.getMessage(), lines); + } + } catch (Exception e) { + LOG.error("Can not send lines to OTS for Exception.", e); + CollectorUtil.collect(e.getMessage(), lines); + } + } + + private List getLineAndError(PutTimeseriesDataResponse result, List lines) throws OTSCriticalException { + List errors = new ArrayList(); + + List status = result.getFailedRows(); + for (PutTimeseriesDataResponse.FailedRowResult r : status) { + errors.add(new LineAndError(lines.get(r.getIndex()), r.getError())); + } + + return errors; + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTimeseriesRowTaskManager.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTimeseriesRowTaskManager.java new file mode 100644 index 00000000..2816d955 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/OTSTimeseriesRowTaskManager.java @@ -0,0 +1,41 @@ +package com.alibaba.datax.plugin.writer.otswriter.model; + +import com.alicloud.openservices.tablestore.SyncClient; +import com.alicloud.openservices.tablestore.SyncClientInterface; +import com.alicloud.openservices.tablestore.TimeseriesClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +public class OTSTimeseriesRowTaskManager implements OTSTaskManagerInterface{ + + private TimeseriesClient client = null; + private OTSBlockingExecutor executorService = null; + private OTSConf conf = null; + + private static final Logger LOG = LoggerFactory.getLogger(OTSTimeseriesRowTaskManager.class); + + public OTSTimeseriesRowTaskManager( + SyncClientInterface ots, + OTSConf conf) { + this.client = ((SyncClient)ots).asTimeseriesClient(); + this.conf = conf; + + executorService = new OTSBlockingExecutor(conf.getConcurrencyWrite()); + } + + @Override + public void execute(List lines) throws Exception { + LOG.debug("Begin execute."); + executorService.execute(new OTSTimeseriesRowTask(client, conf, lines)); + LOG.debug("End execute."); + } + + @Override + public void close() throws Exception { + LOG.debug("Begin close."); + executorService.shutdown(); + LOG.debug("End close."); + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowDeleteChangeWithRecord.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowDeleteChangeWithRecord.java index 5d77ad87..1986100a 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowDeleteChangeWithRecord.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowDeleteChangeWithRecord.java @@ -1,6 +1,7 @@ package com.alibaba.datax.plugin.writer.otswriter.model; import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.plugin.writer.otswriter.utils.WithRecord; public class RowDeleteChangeWithRecord extends com.aliyun.openservices.ots.model.RowDeleteChange implements WithRecord { diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowPutChangeWithRecord.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowPutChangeWithRecord.java index e97a7d63..2e19dd77 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowPutChangeWithRecord.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowPutChangeWithRecord.java @@ -1,6 +1,7 @@ package com.alibaba.datax.plugin.writer.otswriter.model; import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.plugin.writer.otswriter.utils.WithRecord; public class RowPutChangeWithRecord extends com.aliyun.openservices.ots.model.RowPutChange implements WithRecord { diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowUpdateChangeWithRecord.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowUpdateChangeWithRecord.java index f47ca1d2..63f27d65 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowUpdateChangeWithRecord.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/RowUpdateChangeWithRecord.java @@ -1,6 +1,7 @@ package com.alibaba.datax.plugin.writer.otswriter.model; import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.plugin.writer.otswriter.utils.WithRecord; public class RowUpdateChangeWithRecord extends com.aliyun.openservices.ots.model.RowUpdateChange implements WithRecord { diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CalculateHelper.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CalculateHelper.java new file mode 100644 index 00000000..f0d8347d --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CalculateHelper.java @@ -0,0 +1,171 @@ +package com.alibaba.datax.plugin.writer.otswriter.utils; + +import com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException; +import com.alicloud.openservices.tablestore.core.utils.Pair; +import com.alicloud.openservices.tablestore.model.*; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesKey; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesRow; + +import java.util.List; +import java.util.Map; + +import static com.alicloud.openservices.tablestore.model.PrimaryKeyValue.AUTO_INCREMENT; + +public class CalculateHelper { + private static int getPrimaryKeyValueSize(PrimaryKeyValue primaryKeyValue) throws OTSCriticalException { + int primaryKeySize = 0; + if(primaryKeyValue == AUTO_INCREMENT){ + return primaryKeySize; + } + switch (primaryKeyValue.getType()) { + case INTEGER: + primaryKeySize = 8; + break; + case STRING: + primaryKeySize = primaryKeyValue.asStringInBytes().length; + break; + case BINARY: + primaryKeySize = primaryKeyValue.asBinary().length; + break; + default: + throw new OTSCriticalException("Bug: not support the type : " + primaryKeyValue.getType() + " in getPrimaryKeyValueSize"); + } + return primaryKeySize; + } + + private static int getColumnValueSize(ColumnValue columnValue) throws OTSCriticalException { + int columnSize = 0; + switch (columnValue.getType()) { + case INTEGER: + columnSize += 8; + break; + case DOUBLE: + columnSize += 8; + break; + case STRING: + columnSize += columnValue.asStringInBytes().length; + break; + case BINARY: + columnSize += columnValue.asBinary().length; + break; + case BOOLEAN: + columnSize += 1; + break; + default: + throw new OTSCriticalException("Bug: not support the type : " + columnValue.getType() + " in getColumnValueSize"); + } + return columnSize; + } + + public static int getRowPutChangeSize(RowPutChange change) throws OTSCriticalException { + int primaryKeyTotalSize = 0; + int columnTotalSize = 0; + + // PrimaryKeys Total Size + PrimaryKey primaryKey = change.getPrimaryKey(); + PrimaryKeyColumn[] primaryKeyColumnArray = primaryKey.getPrimaryKeyColumns(); + PrimaryKeyColumn primaryKeyColumn; + byte[] primaryKeyName; + PrimaryKeyValue primaryKeyValue; + for (int i = 0; i < primaryKeyColumnArray.length; i++) { + primaryKeyColumn = primaryKeyColumnArray[i]; + primaryKeyName = primaryKeyColumn.getNameRawData(); + primaryKeyValue = primaryKeyColumn.getValue(); + + // += PrimaryKey Name Data + primaryKeyTotalSize += primaryKeyName.length; + + // += PrimaryKey Value Data + primaryKeyTotalSize += getPrimaryKeyValueSize(primaryKeyValue); + } + + // Columns Total Size + List columnList = change.getColumnsToPut(); + for (Column column : columnList) { + // += Column Name + columnTotalSize += column.getNameRawData().length; + + // += Column Value + ColumnValue columnValue = column.getValue(); + + columnTotalSize += getColumnValueSize(columnValue); + + // += Timestamp + if (column.hasSetTimestamp()) { + columnTotalSize += 8; + } + } + + return primaryKeyTotalSize + columnTotalSize; + } + + public static int getRowUpdateChangeSize(RowUpdateChange change) throws OTSCriticalException { + int primaryKeyTotalSize = 0; + int columnPutSize = 0; + int columnDeleteSize = 0; + + // PrimaryKeys Total Size + PrimaryKey primaryKey = change.getPrimaryKey(); + PrimaryKeyColumn[] primaryKeyColumnArray = primaryKey.getPrimaryKeyColumns(); + PrimaryKeyColumn primaryKeyColumn; + byte[] primaryKeyName; + PrimaryKeyValue primaryKeyValue; + for (int i = 0; i < primaryKeyColumnArray.length; i++) { + primaryKeyColumn = primaryKeyColumnArray[i]; + primaryKeyName = primaryKeyColumn.getNameRawData(); + primaryKeyValue = primaryKeyColumn.getValue(); + + // += PrimaryKey Name Data + primaryKeyTotalSize += primaryKeyName.length; + + // += PrimaryKey Value Data + primaryKeyTotalSize += getPrimaryKeyValueSize(primaryKeyValue); + } + + // Column Total Size + List> updatePairList = change.getColumnsToUpdate(); + Column column; + ColumnValue columnValue; + RowUpdateChange.Type type; + for (Pair updatePair : updatePairList) { + column = updatePair.getFirst(); + type = updatePair.getSecond(); + + switch (type) { + case DELETE: + columnDeleteSize += column.getNameRawData().length; + columnDeleteSize += 8;// Timestamp + break; + case DELETE_ALL: + columnDeleteSize += column.getNameRawData().length; + break; + case PUT: + // Name + columnPutSize += column.getNameRawData().length; + + // Value + columnValue = column.getValue(); + columnPutSize += getColumnValueSize(columnValue); + break; + default: + throw new OTSCriticalException("Bug: not support the type : " + type); + } + } + + return primaryKeyTotalSize + columnPutSize + columnDeleteSize; + } + + public static int getTimeseriesRowDataSize(TimeseriesRow row) { + TimeseriesKey timeseriesKey = row.getTimeseriesKey(); + Map fields = row.getFields(); + int totalSize = 0; + totalSize += 8; // time size + totalSize += com.alicloud.openservices.tablestore.core.utils.CalculateHelper.calcStringSizeInBytes(timeseriesKey.getMeasurementName()); + totalSize += com.alicloud.openservices.tablestore.core.utils.CalculateHelper.calcStringSizeInBytes(timeseriesKey.getDataSource()); + totalSize += com.alicloud.openservices.tablestore.core.utils.CalculateHelper.calcStringSizeInBytes(timeseriesKey.buildTagsString()); + for (Map.Entry entry : fields.entrySet()) { + totalSize += entry.getValue().getDataSize() + com.alicloud.openservices.tablestore.core.utils.CalculateHelper.calcStringSizeInBytes(entry.getKey()); + } + return totalSize; + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CollectorUtil.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CollectorUtil.java new file mode 100644 index 00000000..432ac37f --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CollectorUtil.java @@ -0,0 +1,40 @@ +package com.alibaba.datax.plugin.writer.otswriter.utils; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.plugin.TaskPluginCollector; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSLine; + +import java.util.List; + +public class CollectorUtil { + + private static TaskPluginCollector taskPluginCollector = null; + + public static void init(TaskPluginCollector collector) { + taskPluginCollector = collector; + } + + public static void collect(Record dirtyRecord, String errorMessage) { + if (taskPluginCollector != null) { + taskPluginCollector.collectDirtyRecord(dirtyRecord, errorMessage); + } + } + + public static void collect(List dirtyRecords, String errorMessage) { + for (Record r:dirtyRecords) { + collect(r, errorMessage); + } + } + + public static void collect(List errors) { + for (LineAndError e:errors) { + collect(e.getLine().getRecords(), e.getError().getMessage()); + } + } + + public static void collect(String errorMessage, List lines) { + for (OTSLine l:lines) { + collect(l.getRecords(), errorMessage); + } + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ColumnConversion.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ColumnConversion.java index 51162b84..5f7c91a5 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ColumnConversion.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ColumnConversion.java @@ -2,11 +2,12 @@ package com.alibaba.datax.plugin.writer.otswriter.utils; import com.alibaba.datax.common.element.Column; import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException; import com.alibaba.datax.plugin.writer.otswriter.model.OTSAttrColumn; import com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSPKColumn; -import com.aliyun.openservices.ots.model.ColumnValue; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; +import com.alicloud.openservices.tablestore.model.ColumnValue; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; +import com.alicloud.openservices.tablestore.model.PrimaryKeyValue; /** @@ -17,45 +18,66 @@ import com.aliyun.openservices.ots.model.PrimaryKeyValue; * 4. long -> binary */ public class ColumnConversion { - public static PrimaryKeyValue columnToPrimaryKeyValue(Column c, OTSPKColumn col) { + public static PrimaryKeyValue columnToPrimaryKeyValue(Column c, PrimaryKeySchema col) throws OTSCriticalException { try { switch (col.getType()) { - case STRING: - return PrimaryKeyValue.fromString(c.asString()); - case INTEGER: - return PrimaryKeyValue.fromLong(c.asLong()); - default: - throw new IllegalArgumentException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, col.getType(), "PrimaryKeyValue")); + case STRING: + return PrimaryKeyValue.fromString(c.asString()); + case INTEGER: + return PrimaryKeyValue.fromLong(c.asLong()); + case BINARY: + return PrimaryKeyValue.fromBinary(c.asBytes()); + default: + throw new OTSCriticalException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, col.getType(), "PrimaryKeyValue")); } } catch (DataXException e) { throw new IllegalArgumentException(String.format( OTSErrorMessage.COLUMN_CONVERSION_ERROR, c.getType(), c.asString(), col.getType().toString() - )); + ), + e); } } - public static ColumnValue columnToColumnValue(Column c, OTSAttrColumn col) { - try { - switch (col.getType()) { + public static ColumnValue columnToColumnValue(Column c) throws OTSCriticalException { + switch (c.getType()) { case STRING: return ColumnValue.fromString(c.asString()); - case INTEGER: + case LONG: return ColumnValue.fromLong(c.asLong()); - case BOOLEAN: + case BOOL: return ColumnValue.fromBoolean(c.asBoolean()); case DOUBLE: return ColumnValue.fromDouble(c.asDouble()); - case BINARY: + case BYTES: return ColumnValue.fromBinary(c.asBytes()); default: - throw new IllegalArgumentException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, col.getType(), "ColumnValue")); + throw new OTSCriticalException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, c.getType(), "ColumnValue")); + } + } + + public static ColumnValue columnToColumnValue(Column c, OTSAttrColumn col) throws OTSCriticalException { + try { + switch (col.getType()) { + case STRING: + return ColumnValue.fromString(c.asString()); + case INTEGER: + return ColumnValue.fromLong(c.asLong()); + case BOOLEAN: + return ColumnValue.fromBoolean(c.asBoolean()); + case DOUBLE: + return ColumnValue.fromDouble(c.asDouble()); + case BINARY: + return ColumnValue.fromBinary(c.asBytes()); + default: + throw new OTSCriticalException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, col.getType(), "ColumnValue")); } } catch (DataXException e) { throw new IllegalArgumentException(String.format( OTSErrorMessage.COLUMN_CONVERSION_ERROR, c.getType(), c.asString(), col.getType().toString() - )); + ), + e); } } } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ColumnConversionOld.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ColumnConversionOld.java new file mode 100644 index 00000000..a2920b91 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ColumnConversionOld.java @@ -0,0 +1,61 @@ +package com.alibaba.datax.plugin.writer.otswriter.utils; + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSAttrColumn; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; +import com.aliyun.openservices.ots.model.ColumnValue; +import com.aliyun.openservices.ots.model.PrimaryKeyValue; + + +/** + * 备注:datax提供的转换机制有如下限制,如下规则是不能转换的 + * 1. bool -> binary + * 2. binary -> long, double, bool + * 3. double -> bool, binary + * 4. long -> binary + */ +public class ColumnConversionOld { + public static PrimaryKeyValue columnToPrimaryKeyValue(Column c, PrimaryKeySchema col) { + try { + switch (col.getType()) { + case STRING: + return PrimaryKeyValue.fromString(c.asString()); + case INTEGER: + return PrimaryKeyValue.fromLong(c.asLong()); + default: + throw new IllegalArgumentException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, col.getType(), "PrimaryKeyValue")); + } + } catch (DataXException e) { + throw new IllegalArgumentException(String.format( + OTSErrorMessage.COLUMN_CONVERSION_ERROR, + c.getType(), c.asString(), col.getType().toString() + )); + } + } + + public static ColumnValue columnToColumnValue(Column c, OTSAttrColumn col) { + try { + switch (col.getType()) { + case STRING: + return ColumnValue.fromString(c.asString()); + case INTEGER: + return ColumnValue.fromLong(c.asLong()); + case BOOLEAN: + return ColumnValue.fromBoolean(c.asBoolean()); + case DOUBLE: + return ColumnValue.fromDouble(c.asDouble()); + case BINARY: + return ColumnValue.fromBinary(c.asBytes()); + default: + throw new IllegalArgumentException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, col.getType(), "ColumnValue")); + } + } catch (DataXException e) { + throw new IllegalArgumentException(String.format( + OTSErrorMessage.COLUMN_CONVERSION_ERROR, + c.getType(), c.asString(), col.getType().toString() + )); + } + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/Common.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/Common.java index 26eb9329..a48efa69 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/Common.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/Common.java @@ -1,108 +1,124 @@ package com.alibaba.datax.plugin.writer.otswriter.utils; -import java.util.ArrayList; -import java.util.List; - import com.alibaba.datax.common.element.Column; import com.alibaba.datax.common.element.Record; -import com.alibaba.datax.common.plugin.TaskPluginCollector; -import com.alibaba.datax.plugin.writer.otswriter.model.*; -import com.aliyun.openservices.ots.ClientException; -import com.aliyun.openservices.ots.OTSException; -import com.aliyun.openservices.ots.model.ColumnValue; -import com.aliyun.openservices.ots.model.PrimaryKeyValue; -import com.aliyun.openservices.ots.model.RowChange; -import com.aliyun.openservices.ots.model.RowPrimaryKey; -import com.aliyun.openservices.ots.model.RowPutChange; -import com.aliyun.openservices.ots.model.RowUpdateChange; -import org.apache.commons.math3.util.Pair; +import com.alibaba.datax.plugin.writer.otswriter.OTSCriticalException; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSAttrColumn; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSConf; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage; +import com.alicloud.openservices.tablestore.ClientConfiguration; +import com.alicloud.openservices.tablestore.SyncClient; +import com.alicloud.openservices.tablestore.core.utils.Pair; +import com.alicloud.openservices.tablestore.model.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.Map.Entry; public class Common { + + private static final Logger LOG = LoggerFactory.getLogger(Common.class); - public static String getDetailMessage(Exception exception) { - if (exception instanceof OTSException) { - OTSException e = (OTSException) exception; - return "OTSException[ErrorCode:" + e.getErrorCode() + ", ErrorMessage:" + e.getMessage() + ", RequestId:" + e.getRequestId() + "]"; - } else if (exception instanceof ClientException) { - ClientException e = (ClientException) exception; - return "ClientException[ErrorCode:" + e.getErrorCode() + ", ErrorMessage:" + e.getMessage() + "]"; - } else if (exception instanceof IllegalArgumentException) { - IllegalArgumentException e = (IllegalArgumentException) exception; - return "IllegalArgumentException[ErrorMessage:" + e.getMessage() + "]"; - } else { - return "Exception[ErrorMessage:" + exception.getMessage() + "]"; + /** + * 从record中分析出PK,如果分析成功,则返回PK,如果分析失败,则返回null,并记录数据到脏数据回收器中 + * @param pkColumns + * @param r + * @return + * @throws OTSCriticalException + */ + public static PrimaryKey getPKFromRecord(Map pkColumns, Record r) throws OTSCriticalException { + if (r.getColumnNumber() < pkColumns.size()) { + throw new OTSCriticalException(String.format("Bug branch, the count(%d) of record < count(%d) of (pk) from config.", r.getColumnNumber(), pkColumns.size())); } - } - - public static RowPrimaryKey getPKFromRecord(List pkColumns, Record r) { - RowPrimaryKey primaryKey = new RowPrimaryKey(); - int pkCount = pkColumns.size(); - for (int i = 0; i < pkCount; i++) { - Column col = r.getColumn(i); - OTSPKColumn expect = pkColumns.get(i); - - if (col.getRawData() == null) { - throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_COLUMN_VALUE_IS_NULL_ERROR, expect.getName())); - } - - PrimaryKeyValue pk = ColumnConversion.columnToPrimaryKeyValue(col, expect); - primaryKey.addPrimaryKeyColumn(expect.getName(), pk); - } - return primaryKey; - } - - public static List> getAttrFromRecord(int pkCount, List attrColumns, Record r) { - List> attr = new ArrayList>(r.getColumnNumber()); - for (int i = 0; i < attrColumns.size(); i++) { - Column col = r.getColumn(i + pkCount); - OTSAttrColumn expect = attrColumns.get(i); - - if (col.getRawData() == null) { - attr.add(new Pair(expect.getName(), null)); - continue; - } - - ColumnValue cv = ColumnConversion.columnToColumnValue(col, expect); - attr.add(new Pair(expect.getName(), cv)); - } - return attr; - } - - public static RowChange columnValuesToRowChange(String tableName, OTSOpType type, RowPrimaryKey pk, List> values) { - switch (type) { - case PUT_ROW: - RowPutChangeWithRecord rowPutChange = new RowPutChangeWithRecord(tableName); - rowPutChange.setPrimaryKey(pk); - - for (Pair en : values) { - if (en.getValue() != null) { - rowPutChange.addAttributeColumn(en.getKey(), en.getValue()); - } + try { + PrimaryKeyBuilder builder = PrimaryKeyBuilder.createPrimaryKeyBuilder(); + for (Entry en : pkColumns.entrySet()) { + Column col = r.getColumn(en.getValue()); + PrimaryKeySchema expect = en.getKey(); + + if (col.getRawData() == null) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_COLUMN_VALUE_IS_NULL_ERROR, expect.getName())); } - return rowPutChange; - case UPDATE_ROW: - RowUpdateChangeWithRecord rowUpdateChange = new RowUpdateChangeWithRecord(tableName); - rowUpdateChange.setPrimaryKey(pk); - - for (Pair en : values) { - if (en.getValue() != null) { - rowUpdateChange.addAttributeColumn(en.getKey(), en.getValue()); - } else { - rowUpdateChange.deleteAttributeColumn(en.getKey()); - } - } - return rowUpdateChange; - case DELETE_ROW: - RowDeleteChangeWithRecord rowDeleteChange = new RowDeleteChangeWithRecord(tableName); - rowDeleteChange.setPrimaryKey(pk); - return rowDeleteChange; - default: - throw new IllegalArgumentException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, type, "RowChange")); + PrimaryKeyValue pk = ColumnConversion.columnToPrimaryKeyValue(col, expect); + builder.addPrimaryKeyColumn(new PrimaryKeyColumn(expect.getName(), pk)); + } + return builder.build(); + } catch (IllegalArgumentException e) { + LOG.warn("getPKFromRecord fail : {}", e.getMessage(), e); + CollectorUtil.collect(r, e.getMessage()); + return null; } } - public static long getDelaySendMilliseconds(int hadRetryTimes, int initSleepInMilliSecond) { + public static PrimaryKey getPKFromRecordWithAutoIncrement(Map pkColumns, Record r, PrimaryKeySchema autoIncrementPrimaryKey) throws OTSCriticalException { + if (r.getColumnNumber() < pkColumns.size()) { + throw new OTSCriticalException(String.format("Bug branch, the count(%d) of record < count(%d) of (pk) from config.", r.getColumnNumber(), pkColumns.size())); + } + try { + PrimaryKeyBuilder builder = PrimaryKeyBuilder.createPrimaryKeyBuilder(); + for (Entry en : pkColumns.entrySet()) { + Column col = r.getColumn(en.getValue()); + PrimaryKeySchema expect = en.getKey(); + + if (col.getRawData() == null) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_COLUMN_VALUE_IS_NULL_ERROR, expect.getName())); + } + + PrimaryKeyValue pk = ColumnConversion.columnToPrimaryKeyValue(col, expect); + builder.addPrimaryKeyColumn(new PrimaryKeyColumn(expect.getName(), pk)); + } + if(autoIncrementPrimaryKey != null){ + if(autoIncrementPrimaryKey.getOption()!= PrimaryKeyOption.AUTO_INCREMENT){ + throw new OTSCriticalException(String.format("The auto Increment PrimaryKey [(%s)] option should be PrimaryKeyOption.AUTO_INCREMENT.", autoIncrementPrimaryKey.getName())); + } + builder.addPrimaryKeyColumn(autoIncrementPrimaryKey.getName(),PrimaryKeyValue.AUTO_INCREMENT); + + } + return builder.build(); + } catch (IllegalArgumentException e) { + LOG.warn("getPKFromRecord fail : {}", e.getMessage(), e); + CollectorUtil.collect(r, e.getMessage()); + return null; + } + } + + /** + * 从Record中解析ColumnValue,如果Record转换为ColumnValue失败,方法会返回null + * @param pkCount + * @param attrColumns + * @param r + * @return + * @throws OTSCriticalException + */ + public static List> getAttrFromRecord(int pkCount, List attrColumns, Record r) throws OTSCriticalException { + if (pkCount + attrColumns.size() != r.getColumnNumber()) { + throw new OTSCriticalException(String.format("Bug branch, the count(%d) of record != count(%d) of (pk + column) from config.", r.getColumnNumber(), (pkCount + attrColumns.size()))); + } + try { + List> attr = new ArrayList>(r.getColumnNumber()); + for (int i = 0; i < attrColumns.size(); i++) { + Column col = r.getColumn(i + pkCount); + OTSAttrColumn expect = attrColumns.get(i); + + if (col.getRawData() == null) { + attr.add(new Pair(expect.getName(), null)); + continue; + } + + ColumnValue cv = ColumnConversion.columnToColumnValue(col, expect); + attr.add(new Pair(expect.getName(), cv)); + } + return attr; + } catch (IllegalArgumentException e) { + LOG.warn("getAttrFromRecord fail : {}", e.getMessage(), e); + CollectorUtil.collect(r, e.getMessage()); + return null; + } + } + + public static long getDelaySendMillinSeconds(int hadRetryTimes, int initSleepInMilliSecond) { if (hadRetryTimes <= 0) { return 0; @@ -118,4 +134,83 @@ public class Common { } return sleepTime; } + + public static SyncClient getOTSInstance(OTSConf conf) { + ClientConfiguration clientConfigure = new ClientConfiguration(); + clientConfigure.setIoThreadCount(conf.getIoThreadCount()); + clientConfigure.setMaxConnections(conf.getConcurrencyWrite()); + clientConfigure.setSocketTimeoutInMillisecond(conf.getSocketTimeout()); + clientConfigure.setConnectionTimeoutInMillisecond(conf.getConnectTimeoutInMillisecond()); + clientConfigure.setRetryStrategy(new DefaultNoRetry()); + + SyncClient ots = new SyncClient( + conf.getEndpoint(), + conf.getAccessId(), + conf.getAccessKey(), + conf.getInstanceName(), + clientConfigure); + Map extraHeaders = new HashMap(); + extraHeaders.put("x-ots-sdk-type", "public"); + extraHeaders.put("x-ots-request-source", "datax-otswriter"); + ots.setExtraHeaders(extraHeaders); + return ots; + } + + public static LinkedHashMap getEncodePkColumnMapping(TableMeta meta, List attrColumns) throws OTSCriticalException { + LinkedHashMap attrColumnMapping = new LinkedHashMap(); + for (Entry en : meta.getPrimaryKeyMap().entrySet()) { + // don't care performance + int i = 0; + for (; i < attrColumns.size(); i++) { + if (attrColumns.get(i).getName().equals(en.getKey())) { + attrColumnMapping.put(GsonParser.primaryKeySchemaToJson(attrColumns.get(i)), i); + break; + } + } + if (i == attrColumns.size()) { + // exception branch + throw new OTSCriticalException(String.format(OTSErrorMessage.INPUT_PK_NAME_NOT_EXIST_IN_META_ERROR, en.getKey())); + } + } + return attrColumnMapping; + } + + public static LinkedHashMap getEncodePkColumnMappingWithAutoIncrement(TableMeta meta, List attrColumns) throws OTSCriticalException { + LinkedHashMap attrColumnMapping = new LinkedHashMap(); + for (Entry en : meta.getPrimaryKeySchemaMap().entrySet()) { + // don't care performance + if(en.getValue().hasOption()){ + continue; + } + + int i = 0; + for (; i < attrColumns.size(); i++) { + if (attrColumns.get(i).getName().equals(en.getKey())) { + attrColumnMapping.put(GsonParser.primaryKeySchemaToJson(attrColumns.get(i)), i); + break; + } + } + if (i == attrColumns.size()) { + // exception branch + throw new OTSCriticalException(String.format(OTSErrorMessage.INPUT_PK_NAME_NOT_EXIST_IN_META_ERROR, en.getKey())); + } + } + return attrColumnMapping; + } + + public static Map getPkColumnMapping(Map mapping) { + Map target = new LinkedHashMap(); + for (Entry en : mapping.entrySet()) { + target.put(GsonParser.jsonToPrimaryKeySchema(en.getKey()), en.getValue()); + } + return target; + } + + public static Map getAttrColumnMapping(List attrColumns) { + Map attrColumnMapping = new LinkedHashMap(); + for (OTSAttrColumn c : attrColumns) { + attrColumnMapping.put(c.getSrcName(), c); + } + return attrColumnMapping; + } } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CommonOld.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CommonOld.java new file mode 100644 index 00000000..a62711cc --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/CommonOld.java @@ -0,0 +1,93 @@ +package com.alibaba.datax.plugin.writer.otswriter.utils; + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage; +import com.alibaba.datax.plugin.writer.otswriter.model.RowDeleteChangeWithRecord; +import com.alibaba.datax.plugin.writer.otswriter.model.RowPutChangeWithRecord; +import com.alibaba.datax.plugin.writer.otswriter.model.RowUpdateChangeWithRecord; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; +import com.aliyun.openservices.ots.ClientException; +import com.aliyun.openservices.ots.OTSException; +import com.aliyun.openservices.ots.model.ColumnValue; +import com.aliyun.openservices.ots.model.PrimaryKeyValue; +import com.aliyun.openservices.ots.model.RowChange; +import com.aliyun.openservices.ots.model.RowPrimaryKey; +import org.apache.commons.math3.util.Pair; + +import java.util.ArrayList; +import java.util.List; + +public class CommonOld { + + public static RowPrimaryKey getPKFromRecord(List pkColumns, Record r) { + RowPrimaryKey primaryKey = new RowPrimaryKey(); + int pkCount = pkColumns.size(); + for (int i = 0; i < pkCount; i++) { + Column col = r.getColumn(i); + PrimaryKeySchema expect = pkColumns.get(i); + + if (col.getRawData() == null) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_COLUMN_VALUE_IS_NULL_ERROR, expect.getName())); + } + + PrimaryKeyValue pk = ColumnConversionOld.columnToPrimaryKeyValue(col, expect); + primaryKey.addPrimaryKeyColumn(expect.getName(), pk); + } + return primaryKey; + } + + public static List> getAttrFromRecord(int pkCount, List attrColumns, Record r) { + List> attr = new ArrayList>(r.getColumnNumber()); + for (int i = 0; i < attrColumns.size(); i++) { + Column col = r.getColumn(i + pkCount); + com.alibaba.datax.plugin.writer.otswriter.model.OTSAttrColumn expect = attrColumns.get(i); + + if (col.getRawData() == null) { + attr.add(new Pair(expect.getName(), null)); + continue; + } + + ColumnValue cv = ColumnConversionOld.columnToColumnValue(col, expect); + attr.add(new Pair(expect.getName(), cv)); + } + return attr; + } + + public static RowChange columnValuesToRowChange(String tableName, + com.alibaba.datax.plugin.writer.otswriter.model.OTSOpType type, + RowPrimaryKey pk, + List> values) { + switch (type) { + case PUT_ROW: + RowPutChangeWithRecord rowPutChange = new RowPutChangeWithRecord(tableName); + rowPutChange.setPrimaryKey(pk); + + for (Pair en : values) { + if (en.getValue() != null) { + rowPutChange.addAttributeColumn(en.getKey(), en.getValue()); + } + } + + return rowPutChange; + case UPDATE_ROW: + RowUpdateChangeWithRecord rowUpdateChange = new RowUpdateChangeWithRecord(tableName); + rowUpdateChange.setPrimaryKey(pk); + + for (Pair en : values) { + if (en.getValue() != null) { + rowUpdateChange.addAttributeColumn(en.getKey(), en.getValue()); + } else { + rowUpdateChange.deleteAttributeColumn(en.getKey()); + } + } + return rowUpdateChange; + case DELETE_ROW: + RowDeleteChangeWithRecord rowDeleteChange = new RowDeleteChangeWithRecord(tableName); + rowDeleteChange.setPrimaryKey(pk); + return rowDeleteChange; + default: + throw new IllegalArgumentException(String.format(OTSErrorMessage.UNSUPPORT_PARSE, type, "RowChange")); + } + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/DefaultNoRetry.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/DefaultNoRetry.java new file mode 100644 index 00000000..ec000566 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/DefaultNoRetry.java @@ -0,0 +1,33 @@ +package com.alibaba.datax.plugin.writer.otswriter.utils; + + + +import com.alicloud.openservices.tablestore.model.DefaultRetryStrategy; +import com.alicloud.openservices.tablestore.model.RetryStrategy; + +public class DefaultNoRetry extends DefaultRetryStrategy { + + public DefaultNoRetry() { + super(); + } + + @Override + public RetryStrategy clone() { + return super.clone(); + } + + @Override + public int getRetries() { + return super.getRetries(); + } + + @Override + public boolean shouldRetry(String action, Exception ex) { + return false; + } + + @Override + public long nextPause(String action, Exception ex) { + return super.nextPause(action, ex); + } +} \ No newline at end of file diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/GsonParser.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/GsonParser.java index 0cae91f2..4e13a327 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/GsonParser.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/GsonParser.java @@ -1,9 +1,10 @@ package com.alibaba.datax.plugin.writer.otswriter.utils; import com.alibaba.datax.plugin.writer.otswriter.model.OTSConf; -import com.aliyun.openservices.ots.model.Direction; -import com.aliyun.openservices.ots.model.RowPrimaryKey; -import com.aliyun.openservices.ots.model.TableMeta; +import com.alicloud.openservices.tablestore.model.Direction; +import com.alicloud.openservices.tablestore.model.PrimaryKey; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; +import com.alicloud.openservices.tablestore.model.TableMeta; import com.google.gson.Gson; import com.google.gson.GsonBuilder; @@ -39,8 +40,18 @@ public class GsonParser { return g.toJson(meta); } - public static String rowPrimaryKeyToJson (RowPrimaryKey row) { + public static String primaryKeyToJson (PrimaryKey row) { Gson g = gsonBuilder(); return g.toJson(row); } + + public static String primaryKeySchemaToJson (PrimaryKeySchema schema) { + Gson g = gsonBuilder(); + return g.toJson(schema); + } + + public static PrimaryKeySchema jsonToPrimaryKeySchema (String jsonStr) { + Gson g = gsonBuilder(); + return g.fromJson(jsonStr, PrimaryKeySchema.class); + } } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/LineAndError.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/LineAndError.java new file mode 100644 index 00000000..f4e8833e --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/LineAndError.java @@ -0,0 +1,21 @@ +package com.alibaba.datax.plugin.writer.otswriter.utils; + +import com.alibaba.datax.plugin.writer.otswriter.model.OTSLine; + +public class LineAndError { + private OTSLine line; + private com.alicloud.openservices.tablestore.model.Error error; + + public LineAndError(OTSLine record, com.alicloud.openservices.tablestore.model.Error error) { + this.line = record; + this.error = error; + } + + public OTSLine getLine() { + return line; + } + + public com.alicloud.openservices.tablestore.model.Error getError() { + return error; + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ParamChecker.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ParamChecker.java index f9e17af5..b04f8878 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ParamChecker.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ParamChecker.java @@ -1,18 +1,24 @@ package com.alibaba.datax.plugin.writer.otswriter.utils; -import java.util.HashMap; +import com.alibaba.datax.common.exception.CommonErrorCode; +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSAttrColumn; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSConf; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage; +import com.alibaba.datax.plugin.writer.otswriter.model.OTSMode; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; +import com.alicloud.openservices.tablestore.model.PrimaryKeyType; +import com.alicloud.openservices.tablestore.model.TableMeta; + import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage.*; -import com.alibaba.datax.common.util.Configuration; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSAttrColumn; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSPKColumn; -import com.aliyun.openservices.ots.model.PrimaryKeyType; -import com.aliyun.openservices.ots.model.TableMeta; public class ParamChecker { @@ -28,16 +34,13 @@ public class ParamChecker { throw new IllegalArgumentException(String.format(OTSErrorMessage.PARAMETER_LIST_IS_EMPTY_ERROR, key)); } - private static void throwNotListException(String key) { - throw new IllegalArgumentException(String.format(OTSErrorMessage.PARAMETER_IS_NOT_ARRAY_ERROR, key)); - } - - private static void throwNotMapException(String key) { - throw new IllegalArgumentException(String.format(OTSErrorMessage.PARAMETER_IS_NOT_MAP_ERROR, key)); + private static void throwNotListException(String key, Throwable t) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.PARAMETER_IS_NOT_ARRAY_ERROR, key), t); } public static String checkStringAndGet(Configuration param, String key) { String value = param.getString(key); + value = value != null ? value.trim() : null; if (null == value) { throwNotExistException(key); } else if (value.length() == 0) { @@ -51,61 +54,7 @@ public class ParamChecker { try { value = param.getList(key); } catch (ClassCastException e) { - throwNotListException(key); - } - if (null == value) { - throwNotExistException(key); - } else if (isCheckEmpty && value.isEmpty()) { - throwEmptyListException(key); - } - return value; - } - - public static List checkListAndGet(Map range, String key) { - Object obj = range.get(key); - if (null == obj) { - return null; - } - return checkListAndGet(range, key, false); - } - - public static List checkListAndGet(Map range, String key, boolean isCheckEmpty) { - Object obj = range.get(key); - if (null == obj) { - throwNotExistException(key); - } - if (obj instanceof List) { - @SuppressWarnings("unchecked") - List value = (List)obj; - if (isCheckEmpty && value.isEmpty()) { - throwEmptyListException(key); - } - return value; - } else { - throw new IllegalArgumentException(String.format(OTSErrorMessage.PARSE_TO_LIST_ERROR, key)); - } - } - - public static List checkListAndGet(Map range, String key, List defaultList) { - Object obj = range.get(key); - if (null == obj) { - return defaultList; - } - if (obj instanceof List) { - @SuppressWarnings("unchecked") - List value = (List)obj; - return value; - } else { - throw new IllegalArgumentException(String.format(OTSErrorMessage.PARSE_TO_LIST_ERROR, key)); - } - } - - public static Map checkMapAndGet(Configuration param, String key, boolean isCheckEmpty) { - Map value = null; - try { - value = param.getMap(key); - } catch (ClassCastException e) { - throwNotMapException(key); + throwNotListException(key, e); } if (null == value) { throwNotExistException(key); @@ -115,26 +64,75 @@ public class ParamChecker { return value; } - public static void checkPrimaryKey(TableMeta meta, List pk) { - Map types = meta.getPrimaryKey(); + public static void checkPrimaryKey(TableMeta meta, List pk) { + Map pkNameAndTypeMapping = meta.getPrimaryKeyMap(); // 个数是否相等 - if (types.size() != pk.size()) { - throw new IllegalArgumentException(String.format(OTSErrorMessage.INPUT_PK_COUNT_NOT_EQUAL_META_ERROR, pk.size(), types.size())); + if (pkNameAndTypeMapping.size() != pk.size()) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.INPUT_PK_COUNT_NOT_EQUAL_META_ERROR, pk.size(), pkNameAndTypeMapping.size())); } // 名字类型是否相等 - Map inputTypes = new HashMap(); - for (OTSPKColumn col : pk) { - inputTypes.put(col.getName(), col.getType()); - } - - for (Entry e : types.entrySet()) { - if (!inputTypes.containsKey(e.getKey())) { - throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_COLUMN_MISSING_ERROR, e.getKey())); + for (PrimaryKeySchema col : pk) { + PrimaryKeyType type = pkNameAndTypeMapping.get(col.getName()); + if (type == null) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_COLUMN_MISSING_ERROR, col.getName())); } - PrimaryKeyType type = inputTypes.get(e.getKey()); - if (type != e.getValue()) { - throw new IllegalArgumentException(String.format(OTSErrorMessage.INPUT_PK_TYPE_NOT_MATCH_META_ERROR, e.getKey(), type, e.getValue())); + if (type != col.getType()) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.INPUT_PK_TYPE_NOT_MATCH_META_ERROR, col.getName(), type, col.getType())); + } + } + } + + public static void checkVersion(OTSConf conf) { + /** + * conf检查遵循以下规则 + * 1. 旧版本插件 不支持 主键自增列 + * 2. 旧版本插件 不支持 多版本模式 + * 3. 多版本模式 不支持 主键自增列 + * 4. 旧版本插件 不支持 时序数据表 + * 5. 时序数据表 不支持 主键自增列 + */ + if (!conf.isNewVersion() && conf.getEnableAutoIncrement()) { + throw new IllegalArgumentException(PUBLIC_SDK_NO_SUPPORT_AUTO_INCREMENT); + } + if (!conf.isNewVersion() && conf.getMode() == OTSMode.MULTI_VERSION) { + throw new IllegalArgumentException(PUBLIC_SDK_NO_SUPPORT_MULTI_VERSION); + } + if (conf.getMode() == OTSMode.MULTI_VERSION && conf.getEnableAutoIncrement()) { + throw new IllegalArgumentException(NOT_SUPPORT_MULTI_VERSION_AUTO_INCREMENT); + } + if (!conf.isNewVersion() && conf.isTimeseriesTable()) { + throw new IllegalArgumentException(PUBLIC_SDK_NO_SUPPORT_TIMESERIES_TABLE); + } + if (conf.isTimeseriesTable() && conf.getEnableAutoIncrement()) { + throw new IllegalArgumentException(NOT_SUPPORT_TIMESERIES_TABLE_AUTO_INCREMENT); + } + } + + public static void checkPrimaryKeyWithAutoIncrement(TableMeta meta, List pk) { + Map pkNameAndTypeMapping = meta.getPrimaryKeyMap(); + int autoIncrementKeySize = 0; + for(PrimaryKeySchema p : meta.getPrimaryKeyList()){ + if(p.hasOption()){ + autoIncrementKeySize++; + } + } + // 个数是否相等 + if (pkNameAndTypeMapping.size() != pk.size() + autoIncrementKeySize) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.INPUT_PK_COUNT_NOT_EQUAL_META_ERROR, pk.size() + autoIncrementKeySize, pkNameAndTypeMapping.size())); + } + + // 名字类型是否相等 + for (PrimaryKeySchema col : pk) { + if(col.hasOption()){ + continue; + } + PrimaryKeyType type = pkNameAndTypeMapping.get(col.getName()); + if (type == null) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_COLUMN_MISSING_ERROR, col.getName())); + } + if (type != col.getType()) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.INPUT_PK_TYPE_NOT_MATCH_META_ERROR, col.getName(), type, col.getType())); } } } @@ -150,4 +148,23 @@ public class ParamChecker { } } } + + public static TimeUnit checkTimeUnitAndGet(String str) { + if (null == str) { + return null; + } else if ("NANOSECONDS".equalsIgnoreCase(str)) { + return TimeUnit.NANOSECONDS; + } else if ("MICROSECONDS".equalsIgnoreCase(str)) { + return TimeUnit.MICROSECONDS; + } else if ("MILLISECONDS".equalsIgnoreCase(str)) { + return TimeUnit.MILLISECONDS; + } else if ("SECONDS".equalsIgnoreCase(str)) { + return TimeUnit.SECONDS; + } else if ("MINUTES".equalsIgnoreCase(str)) { + return TimeUnit.MINUTES; + } else { + throw new IllegalArgumentException(String.format(OTSErrorMessage.TIMEUNIT_FORMAT_ERROR, str)); + } + } + } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ParseRecord.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ParseRecord.java new file mode 100644 index 00000000..1f157131 --- /dev/null +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/ParseRecord.java @@ -0,0 +1,326 @@ +package com.alibaba.datax.plugin.writer.otswriter.utils; + +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.plugin.writer.otswriter.OTSCriticalException; +import com.alibaba.datax.plugin.writer.otswriter.model.*; +import com.alicloud.openservices.tablestore.core.protocol.timeseries.TimeseriesResponseFactory; +import com.alicloud.openservices.tablestore.core.utils.Pair; +import com.alicloud.openservices.tablestore.model.*; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesKey; +import com.alicloud.openservices.tablestore.model.timeseries.TimeseriesRow; +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; + + +public class ParseRecord { + + private static final Logger LOG = LoggerFactory.getLogger(ParseRecord.class); + + private static com.alicloud.openservices.tablestore.model.Column buildColumn(String name, ColumnValue value, long timestamp) { + if (timestamp > 0) { + return new com.alicloud.openservices.tablestore.model.Column( + name, + value, + timestamp + ); + } else { + return new com.alicloud.openservices.tablestore.model.Column( + name, + value + ); + } + } + /** + * 基于普通方式处理Record + * 当PK或者Attr解析失败时,方法会返回null + * @param tableName + * @param type + * @param pkColumns + * @param attrColumns + * @param record + * @param timestamp + * @return + * @throws OTSCriticalException + */ + public static OTSLine parseNormalRecordToOTSLine( + String tableName, + OTSOpType type, + Map pkColumns, + List attrColumns, + Record record, + long timestamp) throws OTSCriticalException { + + PrimaryKey pk = Common.getPKFromRecord(pkColumns, record); + if (pk == null) { + return null; + } + List> values = Common.getAttrFromRecord(pkColumns.size(), attrColumns, record); + if (values == null) { + return null; + } + + switch (type) { + case PUT_ROW: + RowPutChange rowPutChange = new RowPutChange(tableName, pk); + for (Pair en : values) { + if (en.getSecond() != null) { + rowPutChange.addColumn(buildColumn(en.getFirst(), en.getSecond(), timestamp)); + } + } + if (rowPutChange.getColumnsToPut().isEmpty()) { + return null; + } + return new OTSLine(pk, record, rowPutChange); + case UPDATE_ROW: + RowUpdateChange rowUpdateChange = new RowUpdateChange(tableName, pk); + for (Pair en : values) { + if (en.getSecond() != null) { + rowUpdateChange.put(buildColumn(en.getFirst(), en.getSecond(), timestamp)); + } else { + rowUpdateChange.deleteColumns(en.getFirst()); // 删除整列 + } + } + return new OTSLine(pk, record, rowUpdateChange); + default: + LOG.error("Bug branch, can not support : {}(OTSOpType)", type); + throw new OTSCriticalException(String.format(OTSErrorMessage.UNSUPPORT, type)); + } + } + + + public static OTSLine parseNormalRecordToOTSLineWithAutoIncrement( + String tableName, + OTSOpType type, + Map pkColumns, + List attrColumns, + Record record, + long timestamp, + PrimaryKeySchema autoIncrementPrimaryKey) throws OTSCriticalException { + + PrimaryKey pk = Common.getPKFromRecordWithAutoIncrement(pkColumns, record, autoIncrementPrimaryKey); + if (pk == null) { + return null; + } + List> values = Common.getAttrFromRecord(pkColumns.size(), attrColumns, record); + if (values == null) { + return null; + } + + switch (type) { + case PUT_ROW: + RowPutChange rowPutChange = new RowPutChange(tableName, pk); + for (Pair en : values) { + if (en.getSecond() != null) { + rowPutChange.addColumn(buildColumn(en.getFirst(), en.getSecond(), timestamp)); + } + } + if (rowPutChange.getColumnsToPut().isEmpty()) { + return null; + } + return new OTSLine(pk, record, rowPutChange); + case UPDATE_ROW: + RowUpdateChange rowUpdateChange = new RowUpdateChange(tableName, pk); + for (Pair en : values) { + if (en.getSecond() != null) { + rowUpdateChange.put(buildColumn(en.getFirst(), en.getSecond(), timestamp)); + } else { + rowUpdateChange.deleteColumns(en.getFirst()); // 删除整列 + } + } + return new OTSLine(pk, record, rowUpdateChange); + default: + LOG.error("Bug branch, can not support : {}(OTSOpType)", type); + throw new OTSCriticalException(String.format(OTSErrorMessage.UNSUPPORT, type)); + } + } + + public static OTSLine parseNormalRecordToOTSLineOfTimeseriesTable( + List attrColumns, + Record record, + TimeUnit timeUnit + ) throws OTSCriticalException { + + if (attrColumns.size() != record.getColumnNumber()){ + throw new OTSCriticalException(String.format("Bug branch, the count(%d) of record != count(%d) of column from config.", record.getColumnNumber(), (attrColumns.size()))); + } + + Map tags = new HashMap<>(); + String measurementName = null; + String dataSource = null; + Long timeInUs = null; + Map columnsValues = new HashMap<>(); + + try { + for (int i = 0; i < attrColumns.size(); i++) { + // 如果是tags内部字段 + if (attrColumns.get(i).getTag()){ + tags.put(attrColumns.get(i).getName(), record.getColumn(i).asString()); + } + else if (attrColumns.get(i).getName().equals(OTSConst.MEASUREMENT_NAME)){ + measurementName = record.getColumn(i).asString(); + } + else if (attrColumns.get(i).getName().equals(OTSConst.DATA_SOURCE)){ + dataSource = record.getColumn(i).asString(); + } + else if (attrColumns.get(i).getName().equals(OTSConst.TAGS)){ + String tagString = record.getColumn(i).asString(); + tags.putAll(TimeseriesResponseFactory.parseTagsOrAttrs(tagString)); + } + else if (attrColumns.get(i).getName().equals(OTSConst.TIME)){ + timeInUs = record.getColumn(i).asLong(); + } + else{ + switch (attrColumns.get(i).getType()){ + case INTEGER: + columnsValues.put(attrColumns.get(i).getName(), ColumnValue.fromLong(record.getColumn(i).asLong())); + break; + case BOOLEAN: + columnsValues.put(attrColumns.get(i).getName(), ColumnValue.fromBoolean(record.getColumn(i).asBoolean())); + break; + case DOUBLE: + columnsValues.put(attrColumns.get(i).getName(), ColumnValue.fromDouble(record.getColumn(i).asDouble())); + break; + case BINARY: + columnsValues.put(attrColumns.get(i).getName(), ColumnValue.fromBinary(record.getColumn(i).asBytes())); + break; + case STRING: + default: + columnsValues.put(attrColumns.get(i).getName(), ColumnValue.fromString(record.getColumn(i).asString())); + break; + } + } + } + // 度量名称与时间戳字段值不能为空,否则报错 + if (measurementName == null){ + throw new IllegalArgumentException("The value of the '_m_name' (measurement) field cannot be empty. Please check the input of writer"); + } + else if (timeInUs == null){ + throw new IllegalArgumentException("The value of the '_time' field cannot be empty. Please check the input of writer"); + } + } catch (IllegalArgumentException e) { + LOG.warn("getAttrFromRecord fail : {}", e.getMessage(), e); + CollectorUtil.collect(record, e.getMessage()); + return null; + } + TimeseriesKey key = new TimeseriesKey(measurementName, dataSource, tags); + TimeseriesRow row = new TimeseriesRow(key); + switch (timeUnit){ + case NANOSECONDS: + timeInUs = timeInUs / 1000; + break; + case MILLISECONDS: + timeInUs = timeInUs * 1000; + break; + case SECONDS: + timeInUs = timeInUs * 1000 * 1000; + break; + case MINUTES: + timeInUs = timeInUs * 1000 * 1000 * 60; + break; + case MICROSECONDS: + default: + break; + } + row.setTimeInUs(timeInUs); + + for (Map.Entry entry : columnsValues.entrySet()){ + row.addField(entry.getKey(), entry.getValue()); + } + + return new OTSLine(record, row); + } + + public static String getDefineCoumnName(String attrColumnNamePrefixFilter, int columnNameIndex, Record r) { + String columnName = r.getColumn(columnNameIndex).asString(); + if (attrColumnNamePrefixFilter != null) { + if (columnName.startsWith(attrColumnNamePrefixFilter) && columnName.length() > attrColumnNamePrefixFilter.length()) { + columnName = columnName.substring(attrColumnNamePrefixFilter.length()); + } else { + throw new IllegalArgumentException(String.format(OTSErrorMessage.COLUMN_NOT_DEFINE, columnName)); + } + } + return columnName; + } + + private static void appendCellToRowUpdateChange( + Map pkColumns, + String attrColumnNamePrefixFilter, + Record r, + RowUpdateChange updateChange + ) throws OTSCriticalException { + try { + String columnName = getDefineCoumnName(attrColumnNamePrefixFilter, pkColumns.size(), r); + Column timestamp = r.getColumn(pkColumns.size() + 1); + Column value = r.getColumn(pkColumns.size() + 2); + + if (timestamp.getRawData() == null) { + throw new IllegalArgumentException(OTSErrorMessage.MULTI_VERSION_TIMESTAMP_IS_EMPTY); + } + + if (value.getRawData() == null) { + updateChange.deleteColumn(columnName, timestamp.asLong()); + return; + } + + ColumnValue otsValue = ColumnConversion.columnToColumnValue(value); + + com.alicloud.openservices.tablestore.model.Column c = new com.alicloud.openservices.tablestore.model.Column( + columnName, + otsValue, + timestamp.asLong() + ); + updateChange.put(c); + return; + } catch (IllegalArgumentException e) { + LOG.warn("parseToColumn fail : {}", e.getMessage(), e); + CollectorUtil.collect(r, e.getMessage()); + return; + } catch (DataXException e) { + LOG.warn("parseToColumn fail : {}", e.getMessage(), e); + CollectorUtil.collect(r, e.getMessage()); + return; + } + } + + /** + * 基于特殊模式处理Record + * 当所有Record转换为Column失败时,方法会返回null + * @param tableName + * @param type + * @param pkColumns + * @param records + * @return + * @throws Exception + */ + public static OTSLine parseMultiVersionRecordToOTSLine( + String tableName, + OTSOpType type, + Map pkColumns, + String attrColumnNamePrefixFilter, + PrimaryKey pk, + List records) throws OTSCriticalException { + + switch(type) { + case UPDATE_ROW: + RowUpdateChange updateChange = new RowUpdateChange(tableName, pk); + for (Record r : records) { + appendCellToRowUpdateChange(pkColumns, attrColumnNamePrefixFilter, r, updateChange); + } + if (updateChange.getColumnsToUpdate().isEmpty()) { + return null; + } else { + return new OTSLine(pk, records, updateChange); + } + default: + LOG.error("Bug branch, can not support : {}(OTSOpType)", type); + throw new OTSCriticalException(String.format(OTSErrorMessage.UNSUPPORT, type)); + } + } +} diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/RetryHelper.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/RetryHelper.java index a863b908..5f353777 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/RetryHelper.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/RetryHelper.java @@ -1,34 +1,40 @@ package com.alibaba.datax.plugin.writer.otswriter.utils; +import com.alibaba.datax.plugin.writer.otswriter.OTSErrorCode; +import com.alicloud.openservices.tablestore.ClientException; +import com.alicloud.openservices.tablestore.TableStoreException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.HashSet; import java.util.Set; import java.util.concurrent.Callable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.alibaba.datax.plugin.writer.otswriter.model.LogExceptionManager; -import com.aliyun.openservices.ots.ClientException; -import com.aliyun.openservices.ots.OTSErrorCode; -import com.aliyun.openservices.ots.OTSException; - public class RetryHelper { private static final Logger LOG = LoggerFactory.getLogger(RetryHelper.class); private static final Set noRetryErrorCode = prepareNoRetryErrorCode(); - public static LogExceptionManager logManager = new LogExceptionManager(); - + /** + * 对重试的封装,方法需要用户传入最大重试次数,最大的重试时间。 + * 如果方法执行失败,方法会进入重试,每次重试之前,方法会sleep一段时间(sleep机制请参见 + * Common.getDelaySendMillinSeconds方法),直到重试次数达到上限,系统会抛出异常。 + * @param callable + * @param maxRetryTimes + * @param sleepInMilliSecond + * @return + * @throws Exception + */ public static V executeWithRetry(Callable callable, int maxRetryTimes, int sleepInMilliSecond) throws Exception { int retryTimes = 0; while (true){ - Thread.sleep(Common.getDelaySendMilliseconds(retryTimes, sleepInMilliSecond)); + Thread.sleep(Common.getDelaySendMillinSeconds(retryTimes, sleepInMilliSecond)); try { return callable.call(); } catch (Exception e) { - logManager.addException(e); + LOG.warn("Call callable fail.", e); if (!canRetry(e)){ - LOG.error("Can not retry for Exception.", e); + LOG.error("Can not retry for Exception : {}", e.getMessage()); throw e; } else if (retryTimes >= maxRetryTimes) { LOG.error("Retry times more than limition. maxRetryTimes : {}", maxRetryTimes); @@ -41,7 +47,7 @@ public class RetryHelper { } private static Set prepareNoRetryErrorCode() { - Set pool = new HashSet(); + final Set pool = new HashSet(); pool.add(OTSErrorCode.AUTHORIZATION_FAILURE); pool.add(OTSErrorCode.INVALID_PARAMETER); pool.add(OTSErrorCode.REQUEST_TOO_LARGE); @@ -63,11 +69,21 @@ public class RetryHelper { } public static boolean canRetry(Exception exception) { - OTSException e = null; - if (exception instanceof OTSException) { - e = (OTSException) exception; + TableStoreException e = null; + if (exception instanceof TableStoreException) { + e = (TableStoreException) exception; + LOG.warn( + "OTSException:ErrorCode:{}, ErrorMsg:{}, RequestId:{}", + new Object[]{e.getErrorCode(), e.getMessage(), e.getRequestId()} + ); return canRetry(e.getErrorCode()); + } else if (exception instanceof ClientException) { + ClientException ce = (ClientException) exception; + LOG.warn( + "ClientException:ErrorMsg:{}", + ce.getMessage() + ); return true; } else { return false; diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/WithRecord.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WithRecord.java similarity index 71% rename from otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/WithRecord.java rename to otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WithRecord.java index 2e1672a7..9bb4d4e3 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/model/WithRecord.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WithRecord.java @@ -1,4 +1,4 @@ -package com.alibaba.datax.plugin.writer.otswriter.model; +package com.alibaba.datax.plugin.writer.otswriter.utils; import com.alibaba.datax.common.element.Record; diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WriterModelParser.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WriterModelParser.java index c81587b6..76d6c843 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WriterModelParser.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WriterModelParser.java @@ -1,18 +1,12 @@ package com.alibaba.datax.plugin.writer.otswriter.utils; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import com.alibaba.datax.plugin.writer.otswriter.model.*; +import com.alicloud.openservices.tablestore.model.ColumnType; +import com.alicloud.openservices.tablestore.model.PrimaryKeySchema; +import com.alicloud.openservices.tablestore.model.PrimaryKeyType; +import com.alicloud.openservices.tablestore.model.TableMeta; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSAttrColumn; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSPKColumn; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSConst; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSErrorMessage; -import com.alibaba.datax.plugin.writer.otswriter.model.OTSOpType; -import com.aliyun.openservices.ots.model.ColumnType; -import com.aliyun.openservices.ots.model.PrimaryKeyType; +import java.util.*; /** * 解析配置中参数 @@ -26,39 +20,92 @@ public class WriterModelParser { return PrimaryKeyType.STRING; } else if (type.equalsIgnoreCase(OTSConst.TYPE_INTEGER)) { return PrimaryKeyType.INTEGER; + } else if (type.equalsIgnoreCase(OTSConst.TYPE_BINARY)) { + return PrimaryKeyType.BINARY; } else { throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_TYPE_ERROR, type)); } } - public static OTSPKColumn parseOTSPKColumn(Map column) { - if (column.containsKey(OTSConst.NAME) && column.containsKey(OTSConst.TYPE) && column.size() == 2) { - Object type = column.get(OTSConst.TYPE); - Object name = column.get(OTSConst.NAME); - if (type instanceof String && name instanceof String) { - String typeStr = (String) type; - String nameStr = (String) name; - if (nameStr.isEmpty()) { - throw new IllegalArgumentException(OTSErrorMessage.PK_COLUMN_NAME_IS_EMPTY_ERROR); - } - return new OTSPKColumn(nameStr, parsePrimaryKeyType(typeStr)); - } else { - throw new IllegalArgumentException(OTSErrorMessage.PK_MAP_NAME_TYPE_ERROR); - } + private static Object columnGetObject(Map column, String key, String error) { + Object value = column.get(key); + + if (value == null) { + throw new IllegalArgumentException(error); + } + + return value; + } + + private static String checkString(Object value, String error) { + if (!(value instanceof String)) { + throw new IllegalArgumentException(error); + } + return (String)value; + } + + private static void checkStringEmpty(String value, String error) { + if (value.isEmpty()) { + throw new IllegalArgumentException(error); + } + } + + public static PrimaryKeySchema parseOTSPKColumn(Map column) { + String typeStr = checkString( + columnGetObject(column, OTSConst.TYPE, String.format(OTSErrorMessage.PK_MAP_FILED_MISSING_ERROR, OTSConst.TYPE)), + String.format(OTSErrorMessage.PK_MAP_KEY_TYPE_ERROR, OTSConst.TYPE) + ); + String nameStr = checkString( + columnGetObject(column, OTSConst.NAME, String.format(OTSErrorMessage.PK_MAP_FILED_MISSING_ERROR, OTSConst.NAME)), + String.format(OTSErrorMessage.PK_MAP_KEY_TYPE_ERROR, OTSConst.NAME) + ); + + checkStringEmpty(typeStr, OTSErrorMessage.PK_COLUMN_TYPE_IS_EMPTY_ERROR); + checkStringEmpty(nameStr, OTSErrorMessage.PK_COLUMN_NAME_IS_EMPTY_ERROR); + + if (column.size() == 2) { + return new PrimaryKeySchema(nameStr, parsePrimaryKeyType(typeStr)); } else { throw new IllegalArgumentException(OTSErrorMessage.PK_MAP_INCLUDE_NAME_TYPE_ERROR); } } - public static List parseOTSPKColumnList(List values) { - List pks = new ArrayList(); + public static List parseOTSPKColumnList(TableMeta meta, List values) { + + Map pkMapping = meta.getPrimaryKeyMap(); + + List pks = new ArrayList(); for (Object obj : values) { - if (obj instanceof Map) { + /** + * json 中primary key格式为: + * "primaryKey":[ + * "userid", + * "groupid" + *] + */ + if (obj instanceof String) { + String name = (String) obj; + PrimaryKeyType type = pkMapping.get(name); + if (null == type) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.PK_IS_NOT_EXIST_AT_OTS_ERROR, name)); + } else { + pks.add(new PrimaryKeySchema(name, type)); + } + } + /** + * json 中primary key格式为: + * "primaryKey" : [ + * {"name":"pk1", "type":"string"}, + * {"name":"pk2", "type":"int"} + *], + */ + else if (obj instanceof Map) { @SuppressWarnings("unchecked") Map column = (Map) obj; pks.add(parseOTSPKColumn(column)); - } else { - throw new IllegalArgumentException(OTSErrorMessage.PK_ITEM_IS_NOT_MAP_ERROR); + } + else { + throw new IllegalArgumentException(OTSErrorMessage.PK_ITEM_IS_ILLEAGAL_ERROR); } } return pks; @@ -80,60 +127,154 @@ public class WriterModelParser { } } - public static OTSAttrColumn parseOTSAttrColumn(Map column) { - if (column.containsKey(OTSConst.NAME) && column.containsKey(OTSConst.TYPE) && column.size() == 2) { - Object type = column.get(OTSConst.TYPE); - Object name = column.get(OTSConst.NAME); - if (type instanceof String && name instanceof String) { - String typeStr = (String) type; - String nameStr = (String) name; - if (nameStr.isEmpty()) { - throw new IllegalArgumentException(OTSErrorMessage.ATTR_COLUMN_NAME_IS_EMPTY_ERROR); - } - return new OTSAttrColumn(nameStr, parseColumnType(typeStr)); + public static OTSAttrColumn parseOTSAttrColumn(Map column, OTSMode mode) { + String typeStr = checkString( + columnGetObject(column, OTSConst.TYPE, String.format(OTSErrorMessage.ATTR_MAP_FILED_MISSING_ERROR, OTSConst.TYPE)), + String.format(OTSErrorMessage.ATTR_MAP_KEY_TYPE_ERROR, OTSConst.TYPE) + ); + String nameStr = checkString( + columnGetObject(column, OTSConst.NAME, String.format(OTSErrorMessage.ATTR_MAP_FILED_MISSING_ERROR, OTSConst.NAME)), + String.format(OTSErrorMessage.ATTR_MAP_KEY_TYPE_ERROR, OTSConst.NAME) + ); + + checkStringEmpty(typeStr, OTSErrorMessage.ATTR_COLUMN_TYPE_IS_EMPTY_ERROR); + checkStringEmpty(nameStr, OTSErrorMessage.ATTR_COLUMN_NAME_IS_EMPTY_ERROR); + + if (mode == OTSMode.MULTI_VERSION) { + String srcNameStr = checkString( + columnGetObject(column, OTSConst.SRC_NAME, String.format(OTSErrorMessage.ATTR_MAP_FILED_MISSING_ERROR, OTSConst.SRC_NAME)), + String.format(OTSErrorMessage.ATTR_MAP_KEY_TYPE_ERROR, OTSConst.SRC_NAME) + ); + checkStringEmpty(srcNameStr, OTSErrorMessage.ATTR_COLUMN_SRC_NAME_IS_EMPTY_ERROR); + if (column.size() == 3) { + return new OTSAttrColumn(srcNameStr, nameStr, parseColumnType(typeStr)); } else { - throw new IllegalArgumentException(OTSErrorMessage.ATTR_MAP_NAME_TYPE_ERROR); + throw new IllegalArgumentException(OTSErrorMessage.ATTR_MAP_INCLUDE_SRCNAME_NAME_TYPE_ERROR); } } else { - throw new IllegalArgumentException(OTSErrorMessage.ATTR_MAP_INCLUDE_NAME_TYPE_ERROR); + if (column.size() == 2) { + return new OTSAttrColumn(nameStr, parseColumnType(typeStr)); + } else { + throw new IllegalArgumentException(OTSErrorMessage.ATTR_MAP_INCLUDE_NAME_TYPE_ERROR); + } } } - - private static void checkMultiAttrColumn(List attrs) { - Set pool = new HashSet(); - for (OTSAttrColumn col : attrs) { - if (pool.contains(col.getName())) { - throw new IllegalArgumentException(String.format(OTSErrorMessage.MULTI_ATTR_COLUMN_ERROR, col.getName())); + + public static List parseOTSTimeseriesRowAttrList(List values) { + List attrs = new ArrayList(); + // columns内部必须配置_m_name与_time字段,否则报错 + boolean getMeasurementField = false; + boolean getTimeField = false; + for (Object obj : values) { + if (obj instanceof Map) { + @SuppressWarnings("unchecked") + Map column = (Map) obj; + + + String nameStr = checkString( + columnGetObject(column, OTSConst.NAME, String.format(OTSErrorMessage.ATTR_MAP_FILED_MISSING_ERROR, OTSConst.NAME)), + String.format(OTSErrorMessage.ATTR_MAP_KEY_TYPE_ERROR, OTSConst.NAME) + ); + boolean isTag = column.get(OTSConst.IS_TAG) != null && Boolean.parseBoolean((String) column.get(OTSConst.IS_TAG)); + String typeStr = "String"; + if (column.get(OTSConst.TYPE) != null){ + typeStr = (String) column.get(OTSConst.TYPE); + } + + checkStringEmpty(nameStr, OTSErrorMessage.ATTR_COLUMN_NAME_IS_EMPTY_ERROR); + + if (nameStr.equals(OTSConst.MEASUREMENT_NAME)){ + getMeasurementField = true; + } else if (nameStr.equals(OTSConst.TIME)) { + getTimeField = true; + } + + attrs.add(new OTSAttrColumn(nameStr, parseColumnType(typeStr), isTag)); } else { - pool.add(col.getName()); + throw new IllegalArgumentException(OTSErrorMessage.ATTR_ITEM_IS_NOT_MAP_ERROR); + } + } + if (!getMeasurementField){ + throw new IllegalArgumentException(OTSErrorMessage.NO_FOUND_M_NAME_FIELD_ERROR); + } else if (!getTimeField) { + throw new IllegalArgumentException(OTSErrorMessage.NO_FOUND_TIME_FIELD_ERROR); + } + return attrs; + } + + private static void checkMultiAttrColumn(List pk, List attrs, OTSMode mode) { + // duplicate column name + { + Set pool = new HashSet(); + for (OTSAttrColumn col : attrs) { + if (pool.contains(col.getName())) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.MULTI_ATTR_COLUMN_ERROR, col.getName())); + } else { + pool.add(col.getName()); + } + } + for (PrimaryKeySchema col : pk) { + if (pool.contains(col.getName())) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.MULTI_PK_ATTR_COLUMN_ERROR, col.getName())); + } else { + pool.add(col.getName()); + } + } + } + // duplicate src column name + if (mode == OTSMode.MULTI_VERSION) { + Set pool = new HashSet(); + for (OTSAttrColumn col : attrs) { + if (pool.contains(col.getSrcName())) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.MULTI_ATTR_SRC_COLUMN_ERROR, col.getSrcName())); + } else { + pool.add(col.getSrcName()); + } } } } - public static List parseOTSAttrColumnList(List values) { + public static List parseOTSAttrColumnList(List pk, List values, OTSMode mode) { List attrs = new ArrayList(); for (Object obj : values) { if (obj instanceof Map) { @SuppressWarnings("unchecked") Map column = (Map) obj; - attrs.add(parseOTSAttrColumn(column)); + attrs.add(parseOTSAttrColumn(column, mode)); } else { throw new IllegalArgumentException(OTSErrorMessage.ATTR_ITEM_IS_NOT_MAP_ERROR); } } - checkMultiAttrColumn(attrs); + checkMultiAttrColumn(pk, attrs, mode); return attrs; } - - public static OTSOpType parseOTSOpType(String value) { + + public static OTSOpType parseOTSOpType(String value, OTSMode mode) { + OTSOpType type = null; if (value.equalsIgnoreCase(OTSConst.OTS_OP_TYPE_PUT)) { - return OTSOpType.PUT_ROW; + type = OTSOpType.PUT_ROW; } else if (value.equalsIgnoreCase(OTSConst.OTS_OP_TYPE_UPDATE)) { - return OTSOpType.UPDATE_ROW; - } else if (value.equalsIgnoreCase(OTSConst.OTS_OP_TYPE_DELETE)) { - return OTSOpType.DELETE_ROW; - } else { + type = OTSOpType.UPDATE_ROW; + }else if (value.equalsIgnoreCase(OTSConst.OTS_OP_TYPE_DELETE)) { + type = OTSOpType.DELETE_ROW; + }else { throw new IllegalArgumentException(String.format(OTSErrorMessage.OPERATION_PARSE_ERROR, value)); } + + if (mode == OTSMode.MULTI_VERSION && type == OTSOpType.PUT_ROW) { + throw new IllegalArgumentException(String.format(OTSErrorMessage.MUTLI_MODE_OPERATION_PARSE_ERROR, value)); + } + return type; } + + public static OTSMode parseOTSMode(String value) { + if (value.equalsIgnoreCase(OTSConst.OTS_MODE_NORMAL)) { + return OTSMode.NORMAL; + } else if (value.equalsIgnoreCase(OTSConst.OTS_MODE_MULTI_VERSION)) { + return OTSMode.MULTI_VERSION; + } else { + throw new IllegalArgumentException(String.format(OTSErrorMessage.MODE_PARSE_ERROR, value)); + } + } + } diff --git a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/WriterRetryPolicy.java b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WriterRetryPolicy.java similarity index 92% rename from otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/WriterRetryPolicy.java rename to otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WriterRetryPolicy.java index 3aa61a68..18d12dde 100644 --- a/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/WriterRetryPolicy.java +++ b/otswriter/src/main/java/com/alibaba/datax/plugin/writer/otswriter/utils/WriterRetryPolicy.java @@ -1,4 +1,4 @@ -package com.alibaba.datax.plugin.writer.otswriter; +package com.alibaba.datax.plugin.writer.otswriter.utils; import com.alibaba.datax.plugin.writer.otswriter.model.OTSConf; import com.aliyun.openservices.ots.internal.OTSRetryStrategy; diff --git a/otswriter/src/main/resources/plugin.json b/otswriter/src/main/resources/plugin.json index 315e96cc..5151b15d 100644 --- a/otswriter/src/main/resources/plugin.json +++ b/otswriter/src/main/resources/plugin.json @@ -3,4 +3,4 @@ "class": "com.alibaba.datax.plugin.writer.otswriter.OtsWriter", "description": "", "developer": "alibaba" -} \ No newline at end of file +} diff --git a/package.xml b/package.xml old mode 100755 new mode 100644 index 4c1aff04..41cd9c55 --- a/package.xml +++ b/package.xml @@ -145,6 +145,13 @@ datax + + clickhousereader/target/datax/ + + **/*.* + + datax + hdfsreader/target/datax/ @@ -497,5 +504,12 @@ datax + + neo4jwriter/target/datax/ + + **/*.* + + datax + diff --git a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObVersion.java b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/reader/util/ObVersion.java similarity index 97% rename from oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObVersion.java rename to plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/reader/util/ObVersion.java index 2fc414ce..0eb34feb 100644 --- a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObVersion.java +++ b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/reader/util/ObVersion.java @@ -1,4 +1,4 @@ -package com.alibaba.datax.plugin.reader.oceanbasev10reader.util; +package com.alibaba.datax.plugin.rdbms.reader.util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/reader/util/SingleTableSplitUtil.java b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/reader/util/SingleTableSplitUtil.java old mode 100755 new mode 100644 index 10cfe795..844b6cfd --- a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/reader/util/SingleTableSplitUtil.java +++ b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/reader/util/SingleTableSplitUtil.java @@ -7,6 +7,7 @@ import com.alibaba.datax.plugin.rdbms.reader.Key; import com.alibaba.datax.plugin.rdbms.util.*; import com.alibaba.fastjson2.JSON; +import java.text.MessageFormat; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -20,6 +21,7 @@ import java.sql.ResultSetMetaData; import java.sql.Types; import java.util.ArrayList; import java.util.List; +import static org.apache.commons.lang3.StringUtils.EMPTY; public class SingleTableSplitUtil { private static final Logger LOG = LoggerFactory @@ -277,7 +279,24 @@ public class SingleTableSplitUtil { String splitPK = configuration.getString(Key.SPLIT_PK).trim(); String table = configuration.getString(Key.TABLE).trim(); String where = configuration.getString(Key.WHERE, null); - return genPKSql(splitPK,table,where); + String obMode = configuration.getString("obCompatibilityMode"); + // OceanBase对SELECT MIN(%s),MAX(%s) FROM %s这条sql没有做查询改写,会进行全表扫描,在数据量的时候查询耗时很大甚至超时; + // 所以对于OceanBase数据库,查询模板需要改写为分别查询最大值和最小值。这样可以提升查询数量级的性能。 + if (DATABASE_TYPE == DataBaseType.OceanBase && StringUtils.isNotEmpty(obMode)) { + boolean isOracleMode = "ORACLE".equalsIgnoreCase(obMode); + + String minMaxTemplate = isOracleMode ? "select v2.id as min_a, v1.id as max_a from (" + + "select * from (select %s as id from %s {0} order by id desc) where rownum =1 ) v1," + + "(select * from (select %s as id from %s order by id asc) where rownum =1 ) v2;" : + "select v2.id as min_a, v1.id as max_a from (select %s as id from %s {0} order by id desc limit 1) v1," + + "(select %s as id from %s order by id asc limit 1) v2;"; + + String pkRangeSQL = String.format(minMaxTemplate, splitPK, table, splitPK, table); + String whereString = StringUtils.isNotBlank(where) ? String.format("WHERE (%s AND %s IS NOT NULL)", where, splitPK) : EMPTY; + pkRangeSQL = MessageFormat.format(pkRangeSQL, whereString); + return pkRangeSQL; + } + return genPKSql(splitPK, table, where); } public static String genPKSql(String splitPK, String table, String where){ 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 1b46a8bc..dfb39d85 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 @@ -25,6 +25,7 @@ public enum DataBaseType { Oscar("oscar", "com.oscar.Driver"), OceanBase("oceanbase", "com.alipay.oceanbase.jdbc.Driver"), StarRocks("starrocks", "com.mysql.jdbc.Driver"), + GaussDB("gaussdb", "org.opengauss.Driver"), Databend("databend", "com.databend.jdbc.DatabendDriver"); private String typeName; @@ -71,6 +72,8 @@ public enum DataBaseType { break; case StarRocks: break; + case GaussDB: + break; default: throw DataXException.asDataXException(DBUtilErrorCode.UNSUPPORTED_TYPE, "unsupported database type."); } @@ -132,6 +135,8 @@ public enum DataBaseType { result = jdbc + "?" + suffix; } break; + case GaussDB: + break; default: throw DataXException.asDataXException(DBUtilErrorCode.UNSUPPORTED_TYPE, "unsupported database type."); } @@ -159,6 +164,8 @@ public enum DataBaseType { case KingbaseES: case Oscar: break; + case GaussDB: + break; default: throw DataXException.asDataXException(DBUtilErrorCode.UNSUPPORTED_TYPE, "unsupported database type."); } @@ -184,6 +191,8 @@ public enum DataBaseType { case KingbaseES: case Oscar: break; + case GaussDB: + break; default: throw DataXException.asDataXException(DBUtilErrorCode.UNSUPPORTED_TYPE, "unsupported database type"); } @@ -210,6 +219,8 @@ public enum DataBaseType { break; case Oscar: break; + case GaussDB: + break; default: throw DataXException.asDataXException(DBUtilErrorCode.UNSUPPORTED_TYPE, "unsupported database type"); } 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 bec3c683..7b84c320 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 @@ -12,6 +12,7 @@ import com.alibaba.datax.plugin.rdbms.util.DataBaseType; import com.alibaba.datax.plugin.rdbms.util.RdbmsException; import com.alibaba.datax.plugin.rdbms.writer.util.OriginalConfPretreatmentUtil; import com.alibaba.datax.plugin.rdbms.writer.util.WriterUtil; +import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Triple; import org.slf4j.Logger; @@ -199,6 +200,9 @@ public class CommonRdbmsWriter { protected boolean emptyAsNull; protected Triple, List, List> resultSetMetaData; + private int dumpRecordLimit = Constant.DEFAULT_DUMP_RECORD_LIMIT; + private AtomicLong dumpRecordCount = new AtomicLong(0); + public Task(DataBaseType dataBaseType) { this.dataBaseType = dataBaseType; } @@ -209,7 +213,7 @@ public class CommonRdbmsWriter { this.jdbcUrl = writerSliceConfig.getString(Key.JDBC_URL); //ob10的处理 - if (this.jdbcUrl.startsWith(Constant.OB10_SPLIT_STRING) && this.dataBaseType == DataBaseType.MySql) { + if (this.jdbcUrl.startsWith(Constant.OB10_SPLIT_STRING)) { String[] ss = this.jdbcUrl.split(Constant.OB10_SPLIT_STRING_PATTERN); if (ss.length != 3) { throw DataXException @@ -368,7 +372,11 @@ public class CommonRdbmsWriter { } } - protected void doOneInsert(Connection connection, List buffer) { + public boolean needToDumpRecord() { + return dumpRecordCount.incrementAndGet() <= dumpRecordLimit; + } + + public void doOneInsert(Connection connection, List buffer) { PreparedStatement preparedStatement = null; try { connection.setAutoCommit(true); @@ -381,7 +389,10 @@ public class CommonRdbmsWriter { preparedStatement, record); preparedStatement.execute(); } catch (SQLException e) { - LOG.debug(e.toString()); + if (needToDumpRecord()) { + LOG.warn("ERROR : record {}", record); + LOG.warn("Insert fatal error SqlState ={}, errorCode = {}, {}", e.getSQLState(), e.getErrorCode(), e); + } this.taskPluginCollector.collectDirtyRecord(record, e); } finally { diff --git a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/Constant.java b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/Constant.java index 0e4692e2..9510fd14 100755 --- a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/Constant.java +++ b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/Constant.java @@ -19,4 +19,5 @@ public final class Constant { public static final String OB10_SPLIT_STRING = "||_dsc_ob10_dsc_||"; public static final String OB10_SPLIT_STRING_PATTERN = "\\|\\|_dsc_ob10_dsc_\\|\\|"; + public static final int DEFAULT_DUMP_RECORD_LIMIT = 10; } diff --git a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/Key.java b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/Key.java index 25a2ab52..3c282d5d 100755 --- a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/Key.java +++ b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/Key.java @@ -11,6 +11,8 @@ public final class Key { public final static String COLUMN = "column"; + public final static String ONCONFLICT_COLUMN = "onConflictColumn"; + //可选值为:insert,replace,默认为 insert (mysql 支持,oracle 没用 replace 机制,只能 insert,oracle 可以不暴露这个参数) public final static String WRITE_MODE = "writeMode"; diff --git a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/ColumnEntry.java b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/ColumnEntry.java index c86bd206..6bfc1bb9 100644 --- a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/ColumnEntry.java +++ b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/ColumnEntry.java @@ -1,11 +1,11 @@ package com.alibaba.datax.plugin.unstructuredstorage.reader; +import com.alibaba.fastjson2.JSON; +import org.apache.commons.lang3.StringUtils; + import java.text.DateFormat; import java.text.SimpleDateFormat; -import org.apache.commons.lang3.StringUtils; - -import com.alibaba.fastjson2.JSON; public class ColumnEntry { private Integer index; @@ -13,6 +13,15 @@ public class ColumnEntry { private String value; private String format; private DateFormat dateParse; + private String name; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } public Integer getIndex() { return index; diff --git a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/Key.java b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/Key.java index 71e13ad2..0945779b 100755 --- a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/Key.java +++ b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/Key.java @@ -87,4 +87,7 @@ public class Key { public static final String TAR_FILE_FILTER_PATTERN = "tarFileFilterPattern"; public static final String ENABLE_INNER_SPLIT = "enableInnerSplit"; + public static final String HIVE_PARTION_COLUMN = "hivePartitionColumn"; + + } diff --git a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/UnstructuredStorageReaderUtil.java b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/UnstructuredStorageReaderUtil.java index afcad851..27f4c48a 100755 --- a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/UnstructuredStorageReaderUtil.java +++ b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/reader/UnstructuredStorageReaderUtil.java @@ -715,4 +715,70 @@ public class UnstructuredStorageReaderUtil { public static void setSourceFile(Configuration configuration, List sourceFiles){ configuration.set(Constant.SOURCE_FILE, sourceFiles); } + + public static ArrayList getHivePartitionColumns(String filePath, List hivePartitionColumnEntrys) { + ArrayList hivePartitionColumns = new ArrayList<>(); + + if (null == hivePartitionColumnEntrys) { + return hivePartitionColumns; + } + + // 对于分区列pt,则从path中找/pt=xxx/,xxx即分区列的值,另外确认在path中只有一次出现 + + for (ColumnEntry columnEntry : hivePartitionColumnEntrys) { + String parColName = columnEntry.getValue(); + String patten = String.format("/%s=", parColName); + int index = filePath.indexOf(patten); + if (index != filePath.lastIndexOf(patten)) { + throw new DataXException(String.format("Found multiple partition folder in filePath %s, partition: %s", filePath, parColName)); + } + + String subPath = filePath.substring(index + 1); + int firstSeparatorIndex = subPath.indexOf(File.separator); + if (firstSeparatorIndex > 0) { + subPath = subPath.substring(0, firstSeparatorIndex); + } + + if (subPath.split("=").length != 2) { + throw new DataXException(String.format("Found partition column value in filePath %s failed, partition: %s", filePath, parColName)); + } + String parColVal = subPath.split("=")[1]; + + String colType = columnEntry.getType().toUpperCase(); + Type type = Type.valueOf(colType); + + Column generateColumn; + switch (type) { + case STRING: + generateColumn = new StringColumn(parColVal); + break; + + case DOUBLE: + generateColumn = new DoubleColumn(parColVal); + break; + + case LONG: + generateColumn = new LongColumn(parColVal); + break; + + case BOOLEAN: + generateColumn = new BoolColumn(parColVal); + break; + + case DATE: + generateColumn = new DateColumn(new StringColumn(parColVal.toString()).asDate()); + break; + + default: + String errorMessage = String.format("The column type you configured is not currently supported: %s", parColVal); + LOG.error(errorMessage); + throw DataXException.asDataXException(UnstructuredStorageReaderErrorCode.NOT_SUPPORT_TYPE, errorMessage); + } + + hivePartitionColumns.add(generateColumn); + } + + return hivePartitionColumns; + } + } diff --git a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/Constant.java b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/Constant.java index 092fbfd7..a485c124 100755 --- a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/Constant.java +++ b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/Constant.java @@ -12,9 +12,13 @@ public class Constant { public static final String FILE_FORMAT_TEXT = "text"; + public static final String FILE_FORMAT_SQL = "sql"; + //每个分块10MB,最大10000个分块, MAX_FILE_SIZE 单位: MB public static final Long MAX_FILE_SIZE = 10 * 10000L; + public static final int DEFAULT_COMMIT_SIZE = 2000; + public static final String DEFAULT_SUFFIX = ""; public static final String TRUNCATE = "truncate"; diff --git a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/Key.java b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/Key.java index 125957f1..ee97abd8 100755 --- a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/Key.java +++ b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/Key.java @@ -5,12 +5,16 @@ public class Key { // must have public static final String FILE_NAME = "fileName"; + public static final String TABLE_NAME = "table"; + // must have public static final String WRITE_MODE = "writeMode"; // not must , not default , public static final String FIELD_DELIMITER = "fieldDelimiter"; + public static final String QUOTE_CHARACTER = "quoteChar"; + // not must , default os's line delimiter public static final String LINE_DELIMITER = "lineDelimiter"; @@ -38,6 +42,8 @@ public class Key { // writer maxFileSize public static final String MAX_FILE_SIZE = "maxFileSize"; + + public static final String COMMIT_SIZE = "commitSize"; // writer file type suffix, like .txt .csv public static final String SUFFIX = "suffix"; diff --git a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/SqlWriter.java b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/SqlWriter.java new file mode 100644 index 00000000..4d6ff713 --- /dev/null +++ b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/SqlWriter.java @@ -0,0 +1,69 @@ +package com.alibaba.datax.plugin.unstructuredstorage.writer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Writer; +import java.util.List; +import java.util.stream.Collectors; + +public class SqlWriter implements UnstructuredWriter { + private static final Logger LOG = LoggerFactory.getLogger(SqlWriter.class); + + private Writer sqlWriter; + private String quoteChar; + private String lineSeparator; + private String tableName; + private StringBuilder insertPrefix; + + public SqlWriter(Writer writer, String quoteChar, String tableName, String lineSeparator, List columnNames) { + this.sqlWriter = writer; + this.quoteChar = quoteChar; + this.lineSeparator = lineSeparator; + this.tableName = tableName; + buildInsertPrefix(columnNames); + } + + @Override + public void writeOneRecord(List splitedRows) throws IOException { + if (splitedRows.isEmpty()) { + LOG.info("Found one record line which is empty."); + return; + } + + StringBuilder sqlPatten = new StringBuilder(4096).append(insertPrefix); + sqlPatten.append(splitedRows.stream().map(e -> "'" + DataXCsvWriter.replace(e, "'", "''") + "'").collect(Collectors.joining(","))); + sqlPatten.append(");").append(lineSeparator); + this.sqlWriter.write(sqlPatten.toString()); + } + + private void buildInsertPrefix(List columnNames) { + StringBuilder sb = new StringBuilder(columnNames.size() * 32); + + for (String columnName : columnNames) { + if (sb.length() > 0) { + sb.append(","); + } + sb.append(quoteChar).append(columnName).append(quoteChar); + } + + int capacity = 16 + tableName.length() + sb.length(); + this.insertPrefix = new StringBuilder(capacity); + this.insertPrefix.append("INSERT INTO ").append(tableName).append(" (").append(sb).append(")").append(" VALUES("); + } + + public void appendCommit() throws IOException { + this.sqlWriter.write("commit;" + lineSeparator); + } + + @Override + public void flush() throws IOException { + this.sqlWriter.flush(); + } + + @Override + public void close() throws IOException { + this.sqlWriter.close(); + } +} diff --git a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/UnstructuredStorageWriterUtil.java b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/UnstructuredStorageWriterUtil.java index e9040662..4ce6461c 100755 --- a/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/UnstructuredStorageWriterUtil.java +++ b/plugin-unstructured-storage-util/src/main/java/com/alibaba/datax/plugin/unstructuredstorage/writer/UnstructuredStorageWriterUtil.java @@ -10,7 +10,10 @@ import java.util.Set; import java.util.UUID; import com.alibaba.datax.common.element.BytesColumn; + +import com.google.common.base.Preconditions; import org.apache.commons.codec.binary.Base64; +import org.apache.commons.collections.CollectionUtils; import org.apache.commons.compress.compressors.CompressorOutputStream; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; @@ -90,7 +93,8 @@ public class UnstructuredStorageWriterUtil { writerConfiguration.set(Key.FILE_FORMAT, fileFormat); } if (!Constant.FILE_FORMAT_CSV.equals(fileFormat) - && !Constant.FILE_FORMAT_TEXT.equals(fileFormat)) { + && !Constant.FILE_FORMAT_TEXT.equals(fileFormat) + && !Constant.FILE_FORMAT_SQL.equals(fileFormat)) { throw DataXException.asDataXException( UnstructuredStorageWriterErrorCode.ILLEGAL_VALUE, String.format("unsupported fileFormat %s ", fileFormat)); } @@ -232,22 +236,31 @@ public class UnstructuredStorageWriterUtil { // warn: default false String fileFormat = config.getString(Key.FILE_FORMAT, Constant.FILE_FORMAT_TEXT); - + boolean isSqlFormat = Constant.FILE_FORMAT_SQL.equalsIgnoreCase(fileFormat); + int commitSize = config.getInt(Key.COMMIT_SIZE, Constant.DEFAULT_COMMIT_SIZE); UnstructuredWriter unstructuredWriter = produceUnstructuredWriter(fileFormat, config, writer); List headers = config.getList(Key.HEADER, String.class); - if (null != headers && !headers.isEmpty()) { + if (null != headers && !headers.isEmpty() && !isSqlFormat) { unstructuredWriter.writeOneRecord(headers); } Record record = null; + int receivedCount = 0; String byteEncoding = config.getString(Key.BYTE_ENCODING); while ((record = lineReceiver.getFromReader()) != null) { UnstructuredStorageWriterUtil.transportOneRecord(record, nullFormat, dateParse, taskPluginCollector, unstructuredWriter, byteEncoding); + receivedCount++; + if (isSqlFormat && receivedCount % commitSize == 0) { + ((SqlWriter) unstructuredWriter).appendCommit(); + } } + if (isSqlFormat) { + ((SqlWriter)unstructuredWriter).appendCommit(); + } // warn:由调用方控制流的关闭 // IOUtils.closeQuietly(unstructuredWriter); } @@ -262,6 +275,15 @@ public class UnstructuredStorageWriterUtil { String fieldDelimiter = config.getString(Key.FIELD_DELIMITER, String.valueOf(Constant.DEFAULT_FIELD_DELIMITER)); unstructuredWriter = TextCsvWriterManager.produceTextWriter(writer, fieldDelimiter, config); + } else if (StringUtils.equalsIgnoreCase(fileFormat, Constant.FILE_FORMAT_SQL)) { + String tableName = config.getString(Key.TABLE_NAME); + Preconditions.checkArgument(StringUtils.isNotEmpty(tableName), "table name is empty"); + String quoteChar = config.getString(Key.QUOTE_CHARACTER); + Preconditions.checkArgument(StringUtils.isNotEmpty(quoteChar), "quote character is empty"); + String lineSeparator = config.getString(Key.LINE_DELIMITER, IOUtils.LINE_SEPARATOR); + List headers = config.getList(Key.HEADER, String.class); + Preconditions.checkArgument(CollectionUtils.isNotEmpty(headers), "column names are empty"); + unstructuredWriter = new SqlWriter(writer, quoteChar, tableName, lineSeparator, headers); } return unstructuredWriter; diff --git a/pom.xml b/pom.xml index 957c60ee..7847ee5b 100644 --- a/pom.xml +++ b/pom.xml @@ -70,6 +70,7 @@ ftpreader txtfilereader streamreader + clickhousereader mongodbreader tdenginereader @@ -123,11 +124,15 @@ doriswriter selectdbwriter adbmysqlwriter + neo4jwriter plugin-rdbms-util plugin-unstructured-storage-util - + gaussdbreader + gaussdbwriter + datax-example + 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 index 75b2df3a..d5f2887a 100755 --- 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 @@ -78,7 +78,7 @@ public class StarRocksWriter extends Writer { 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); + LOG.info("Begin to execute postSqls:[{}]. context info:{}.", String.join(";", renderedPostSqls), jdbcUrl); StarRocksWriterUtil.executeSqls(conn, renderedPostSqls); DBUtil.closeDBResources(null, null, conn); } diff --git a/streamreader/pom.xml b/streamreader/pom.xml index dc754d9a..7d186076 100755 --- a/streamreader/pom.xml +++ b/streamreader/pom.xml @@ -39,6 +39,16 @@ + + + + src/main/resources + + **/*.* + + true + + diff --git a/streamwriter/pom.xml b/streamwriter/pom.xml index 4a987fac..2fa95d7b 100755 --- a/streamwriter/pom.xml +++ b/streamwriter/pom.xml @@ -34,6 +34,16 @@ + + + + src/main/resources + + **/*.* + + true + + diff --git a/tdenginewriter/doc/tdenginewriter.md b/tdenginewriter/doc/tdenginewriter.md index ba20fdb7..d23a2b4f 100644 --- a/tdenginewriter/doc/tdenginewriter.md +++ b/tdenginewriter/doc/tdenginewriter.md @@ -14,7 +14,7 @@ TDengineWriter can be used as a data migration tool for DBAs to import data from TDengineWriter obtains the protocol data generated by Reader through DataX framework, connects to TDengine through JDBC Driver, executes insert statement /schemaless statement, and writes the data to TDengine. -In TDengine, table can be divided into super table, sub-table and ordinary table. Super table and sub-table include Colum and Tag. The value of tag column of sub-table is fixed value. (details please refer to: [data model](https://www.taosdata.com/docs/cn/v2.0/architecture#model)) +In TDengine, table can be divided into super table, sub-table and ordinary table. Super table and sub-table include Column and Tag. The value of tag column of sub-table is fixed value. (details please refer to: [data model](https://www.taosdata.com/docs/cn/v2.0/architecture#model)) The TDengineWriter can write data to super tables, sub-tables, and ordinary tables using the following methods based on the type of the table and whether the column parameter contains TBName: diff --git a/transformer/doc/transformer.md b/transformer/doc/transformer.md index 260c0fb6..0a00dbaa 100644 --- a/transformer/doc/transformer.md +++ b/transformer/doc/transformer.md @@ -42,7 +42,7 @@ dx_substr(1,"5","10") column 1的value为“dataxTest”=>"Test" * 举例: ``` dx_replace(1,"2","4","****") column 1的value为“dataxTest”=>"da****est" -dx_replace(1,"5","10","****") column 1的value为“dataxTest”=>"data****" +dx_replace(1,"5","10","****") column 1的value为“dataxTest”=>"datax****" ``` 4. dx_filter (关联filter暂不支持,即多个字段的联合判断,函参太过复杂,用户难以使用。) * 参数: diff --git a/userGuid.md b/userGuid.md index 876bae99..badb1b4e 100644 --- a/userGuid.md +++ b/userGuid.md @@ -17,7 +17,7 @@ DataX本身作为数据同步框架,将不同数据源的同步抽象为从源 * 工具部署 - * 方法一、直接下载DataX工具包:[DataX下载地址](https://datax-opensource.oss-cn-hangzhou.aliyuncs.com/202210/datax.tar.gz) + * 方法一、直接下载DataX工具包:[DataX下载地址](https://datax-opensource.oss-cn-hangzhou.aliyuncs.com/202309/datax.tar.gz) 下载后解压至本地某个目录,进入bin目录,即可运行同步作业: