mirror of
https://github.com/alibaba/DataX.git
synced 2025-05-02 19:50:29 +08:00
Merge pull request #1848 from yunjinlxp/yunjin/updateOTS
feat: update ots plugins
This commit is contained in:
commit
82680c4c63
@ -13,7 +13,7 @@ OTSReader插件实现了从OTS读取数据,并可以通过用户指定抽取
|
||||
* 范围抽取
|
||||
* 指定分片抽取
|
||||
|
||||
OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务,提供海量结构化数据的存储和实时访问。OTS 以实例和表的形式组织数据,通过数据分片和负载均衡技术,实现规模上的无缝扩展。
|
||||
本版本的OTSReader新增了支持多版本数据的读取功能,同时兼容旧版本的配置文件
|
||||
|
||||
## 2 实现原理
|
||||
|
||||
@ -25,125 +25,329 @@ OTSReader会根据OTS的表范围,按照Datax并发的数目N,将范围等
|
||||
|
||||
### 3.1 配置样例
|
||||
|
||||
* 配置一个从OTS全表同步抽取数据到本地的作业:
|
||||
|
||||
```
|
||||
{
|
||||
"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:
|
||||
#### 3.1.1
|
||||
* 配置一个从OTS表读取单版本数据的reader:
|
||||
|
||||
```
|
||||
{
|
||||
"job": {
|
||||
"setting": {
|
||||
"speed": {
|
||||
"byte":10485760
|
||||
},
|
||||
"errorLimit":0.0
|
||||
//设置传输速度,单位为byte/s,DataX运行会尽可能达到该速度但是不超过它.
|
||||
"byte": 1048576
|
||||
}
|
||||
//出错限制
|
||||
"errorLimit": {
|
||||
//出错的record条数上限,当大于该值即报错。
|
||||
"record": 0,
|
||||
//出错的record百分比上限 1.0表示100%,0.02表示2%
|
||||
"percentage": 0.02
|
||||
}
|
||||
},
|
||||
"content": [
|
||||
{
|
||||
"reader": {
|
||||
"name": "otsreader",
|
||||
"name": "otsreader-internal",
|
||||
"parameter": {
|
||||
"endpoint":"",
|
||||
"accessId":"",
|
||||
"accessKey":"",
|
||||
"instanceName":"",
|
||||
"table": "",
|
||||
//version定义了是否使用新版本插件 可选值:false || true
|
||||
"newVersion":"false",
|
||||
//mode定义了读取数据的格式(普通数据/多版本数据),可选值:normal || multiversion
|
||||
"mode": "normal",
|
||||
|
||||
// 导出数据表的表名
|
||||
"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)"} // 常量列(二进制)
|
||||
],
|
||||
// 导出的范围,读取的范围是[begin,end),左闭右开的区间
|
||||
// begin小于end,表示正序读取数据
|
||||
// begin大于end,表示反序读取数据
|
||||
// begin和end不能相等
|
||||
// type支持的类型有如下几类:
|
||||
// string、int、binary
|
||||
// binary输入的方式采用二进制的Base64字符串形式传入
|
||||
// INF_MIN 表示无限小
|
||||
// INF_MAX 表示无限大
|
||||
"range":{
|
||||
// 导出数据的起始范围
|
||||
// 支持INF_MIN, INF_MAX, STRING, INT
|
||||
"begin":[
|
||||
{"type":"INF_MIN"},
|
||||
{"type":"INF_MAX"},
|
||||
{"type":"STRING", "value":"hello"},
|
||||
{"type":"INT", "value":"2999"},
|
||||
// 可选,默认表示从无限小开始读取
|
||||
// 这个值的输入可以填写空数组,或者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"}]
|
||||
},
|
||||
|
||||
|
||||
// 指定要导出的列,支持普通列和常量列
|
||||
// 格式
|
||||
// 普通列格式:{"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="}
|
||||
],
|
||||
// 导出数据的结束范围
|
||||
// 支持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 参数说明
|
||||
|
||||
@ -188,6 +392,26 @@ OTSReader会根据OTS的表范围,按照Datax并发的数目N,将范围等
|
||||
|
||||
* 默认值:无 <br />
|
||||
|
||||
* **newVersion**
|
||||
|
||||
* 描述:version定义了使用的ots SDK版本。<br />
|
||||
* true,新版本插件,使用com.alicloud.openservices.tablestore的依赖(推荐)
|
||||
* false,旧版本插件,使用com.aliyun.openservices.ots的依赖,**不支持多版本数据的读取**
|
||||
|
||||
* 必选:否 <br />
|
||||
|
||||
* 默认值:false <br />
|
||||
|
||||
* **mode**
|
||||
|
||||
* 描述:读取为多版本格式的数据,目前有两种模式。<br />
|
||||
* normal,对应普通的数据
|
||||
* multiVersion,写入数据为多版本格式的数据,多版本模式下,配置参数有所不同,详见3.1.2
|
||||
|
||||
* 必选:否 <br />
|
||||
|
||||
* 默认值:normal <br />
|
||||
|
||||
* **column**
|
||||
|
||||
* 描述:所配置的表中需要同步的列名集合,使用JSON的数组描述字段信息。由于OTS本身是NoSQL系统,在OTSReader抽取数据过程中,必须指定相应地字段名称。
|
||||
@ -237,9 +461,9 @@ OTSReader会根据OTS的表范围,按照Datax并发的数目N,将范围等
|
||||
}
|
||||
```
|
||||
|
||||
* 必选:是 <br />
|
||||
* 必选:否 <br />
|
||||
|
||||
* 默认值:空 <br />
|
||||
* 默认值:读取全部值 <br />
|
||||
|
||||
* **split**
|
||||
|
||||
@ -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
|
||||
|
@ -10,17 +10,6 @@
|
||||
<name>otsreader</name>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-api</artifactId>
|
||||
<version>2.17.1</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<version>2.17.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.alibaba.datax</groupId>
|
||||
<artifactId>datax-common</artifactId>
|
||||
@ -47,9 +36,16 @@
|
||||
<version>2.2.4</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>log4j-api</artifactId>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.aliyun.openservices</groupId>
|
||||
<artifactId>tablestore</artifactId>
|
||||
<version>5.13.13</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
@ -61,8 +57,22 @@
|
||||
<artifactId>gson</artifactId>
|
||||
<version>2.2.4</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.alibaba</groupId>
|
||||
<artifactId>fastjson</artifactId>
|
||||
<version>1.2.83_noneautotype</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<build>
|
||||
<resources>
|
||||
<resource>
|
||||
<directory>src/main/java</directory>
|
||||
<includes>
|
||||
<include>**/*.properties</include>
|
||||
</includes>
|
||||
</resource>
|
||||
</resources>
|
||||
<plugins>
|
||||
<!-- compiler plugin -->
|
||||
<plugin>
|
||||
@ -98,10 +108,6 @@
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<version>2.5</version>
|
||||
<configuration>
|
||||
<parallel>all</parallel>
|
||||
<forkCount>10</forkCount>
|
||||
<reuseForks>true</reuseForks>
|
||||
<argLine>-Xmx1024m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=.</argLine>
|
||||
<includes>
|
||||
<include>**/unittest/*.java</include>
|
||||
<include>**/functiontest/*.java</include>
|
||||
@ -111,4 +117,3 @@
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
||||
|
||||
|
@ -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<Configuration> split(int num) throws Exception;
|
||||
|
||||
public void close();
|
||||
|
||||
}
|
@ -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;
|
||||
}
|
@ -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() {
|
||||
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 ...");
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,188 +1,186 @@
|
||||
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;
|
||||
|
||||
|
||||
public OTSConf getConf() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
public TableMeta getMeta() {
|
||||
return meta;
|
||||
}
|
||||
|
||||
public SyncClientInterface getOts() {
|
||||
return ots;
|
||||
}
|
||||
|
||||
public void setOts(SyncClientInterface ots) {
|
||||
this.ots = ots;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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));
|
||||
// 基于预定义的Json格式,检查传入参数是否符合Conf定义规范
|
||||
conf = OTSConf.load(param);
|
||||
|
||||
// 必选参数
|
||||
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));
|
||||
// Init ots
|
||||
ots = OtsHelper.getOTSInstance(conf);
|
||||
|
||||
ots = new OTSClient(
|
||||
this.conf.getEndpoint(),
|
||||
this.conf.getAccessId(),
|
||||
this.conf.getAccesskey(),
|
||||
this.conf.getInstanceName());
|
||||
// 宽行表init
|
||||
if (!conf.isTimeseriesTable()) {
|
||||
// 获取TableMeta
|
||||
meta = OtsHelper.getTableMeta(
|
||||
ots,
|
||||
conf.getTableName(),
|
||||
conf.getRetry(),
|
||||
conf.getRetryPauseInMillisecond());
|
||||
|
||||
meta = getTableMeta(ots, conf.getTableName());
|
||||
LOG.info("Table Meta : {}", GsonParser.metaToJson(meta));
|
||||
|
||||
conf.setColumns(ReaderModelParser.parseOTSColumnList(ParamChecker.checkListAndGet(param, Key.COLUMN, true)));
|
||||
|
||||
Map<String, Object> 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<PrimaryKeyValue> points = ReaderModelParser.parsePrimaryKey(ParamChecker.checkListAndGet(rangeMap, Key.RANGE_SPLIT));
|
||||
ParamChecker.checkInputSplitPoints(meta, range, direction, points);
|
||||
conf.setRangeSplit(points);
|
||||
// 基于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<Configuration> split(int num) throws Exception {
|
||||
LOG.info("Expect split num : " + num);
|
||||
|
||||
}
|
||||
|
||||
public List<Configuration> split(int mandatoryNumber) throws Exception {
|
||||
LOG.info("Expect split num : " + mandatoryNumber);
|
||||
|
||||
List<Configuration> configurations = new ArrayList<Configuration>();
|
||||
|
||||
List<OTSRange> ranges = null;
|
||||
|
||||
if (this.conf.getRangeSplit() != null) { // 用户显示指定了拆分范围
|
||||
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");
|
||||
ranges = userDefinedRangeSplit(meta, range, this.conf.getRangeSplit());
|
||||
configurations = getNormalConfigurationBySplit();
|
||||
LOG.info("End userDefinedRangeSplit");
|
||||
} else { // 采用默认的切分算法
|
||||
LOG.info("Begin defaultRangeSplit");
|
||||
ranges = defaultRangeSplit(ots, meta, range, num);
|
||||
configurations = getDefaultConfiguration(mandatoryNumber);
|
||||
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());
|
||||
|
||||
LOG.info("Expect split num: "+ mandatoryNumber +", and final configuration list count : " + configurations.size());
|
||||
return configurations;
|
||||
}
|
||||
|
||||
public OTSConf getConf() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
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<Configuration> getTimeseriesConfigurationBySplit(int mandatoryNumber) throws Exception {
|
||||
List<TimeseriesScanSplitInfo> timeseriesScanSplitInfoList = OtsHelper.splitTimeseriesScan(
|
||||
ots,
|
||||
conf.getTableName(),
|
||||
conf.getMeasurementName(),
|
||||
mandatoryNumber,
|
||||
conf.getRetry(),
|
||||
conf.getSleepInMilliSecond()
|
||||
);
|
||||
conf.getRetryPauseInMillisecond());
|
||||
List<Configuration> 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<Configuration> getNormalConfigurationBySplit() {
|
||||
List<List<PrimaryKeyColumn>> primaryKeys = new ArrayList<List<PrimaryKeyColumn>>();
|
||||
primaryKeys.add(conf.getRange().getBegin());
|
||||
for (PrimaryKeyColumn column : conf.getRange().getSplit()) {
|
||||
List<PrimaryKeyColumn> point = new ArrayList<PrimaryKeyColumn>();
|
||||
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<Configuration> configurations = new ArrayList<Configuration>(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<OTSRange> defaultRangeSplit(OTSClient ots, TableMeta meta, OTSRange range, int num) throws Exception {
|
||||
private List<Configuration> getDefaultConfiguration(int num) throws Exception {
|
||||
if (num == 1) {
|
||||
List<OTSRange> ranges = new ArrayList<OTSRange>();
|
||||
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<PrimaryKeyColumn> realBegin = getPKOfFirstRow(conf.getRange(), direction);
|
||||
List<PrimaryKeyColumn> realEnd = getPKOfFirstRow(reverseRange, reverseDirection);
|
||||
|
||||
// 因为如果其中一行为空,表示这个范围内至多有一行数据
|
||||
// 所以不再细分,直接使用用户定义的范围
|
||||
if (realBegin == null || realEnd == null) {
|
||||
List<OTSRange> ranges = new ArrayList<OTSRange>();
|
||||
ranges.add(range);
|
||||
return ranges;
|
||||
ranges.add(conf.getRange());
|
||||
return getConfigurationsFromRanges(ranges);
|
||||
}
|
||||
|
||||
// 如果出现realBegin,realEnd的方向和direction不一致的情况,直接返回range
|
||||
@ -191,31 +189,55 @@ public class OtsReaderMasterProxy {
|
||||
if (realDirection != direction) {
|
||||
LOG.warn("Expect '" + direction + "', but direction of realBegin and readlEnd is '" + realDirection + "'");
|
||||
List<OTSRange> ranges = new ArrayList<OTSRange>();
|
||||
ranges.add(range);
|
||||
return ranges;
|
||||
ranges.add(conf.getRange());
|
||||
return getConfigurationsFromRanges(ranges);
|
||||
}
|
||||
|
||||
List<OTSRange> 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<OTSRange> userDefinedRangeSplit(TableMeta meta, OTSRange range, List<PrimaryKeyValue> points) {
|
||||
List<OTSRange> ranges = RangeSplit.rangeSplitByPoint(meta, range.getBegin(), range.getEnd(), points);
|
||||
if (ranges.isEmpty()) { // 当PartitionKey相等时,工具内部不会切分Range
|
||||
ranges.add(range);
|
||||
private List<Configuration> getConfigurationsFromRanges(List<OTSRange> ranges){
|
||||
List<Configuration> 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<PrimaryKeyColumn> 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()
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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<OTSColumn> 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<String> 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<String, PrimaryKeyType> 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<String, String> allData = new HashMap<String, String>();
|
||||
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<Record> primaryKeyRecords = new ArrayList<Record>();
|
||||
for (Entry<String, PrimaryKeyType> 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);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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<String> 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<Pair<String, ColumnType>> 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<String, ColumnType> 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;
|
||||
}
|
||||
}
|
@ -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<GetRangeResult> future;
|
||||
|
||||
RequestItem(RangeRowQueryCriteria criteria, OTSFuture<GetRangeResult> future) {
|
||||
this.criteria = criteria;
|
||||
this.future = future;
|
||||
}
|
||||
|
||||
public RangeRowQueryCriteria getCriteria() {
|
||||
return criteria;
|
||||
}
|
||||
|
||||
public OTSFuture<GetRangeResult> getFuture() {
|
||||
return future;
|
||||
}
|
||||
}
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(OtsReaderSlaveProxy.class);
|
||||
|
||||
private void rowsToSender(List<Row> rows, RecordSender sender, List<OTSColumn> 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<String> 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<String> columns) throws Exception {
|
||||
RangeRowQueryCriteria criteria = generateRangeRowQueryCriteria(conf.getTableName(), begin, end, direction, columns);
|
||||
|
||||
GetRangeRequest request = new GetRangeRequest();
|
||||
request.setRangeRowQueryCriteria(criteria);
|
||||
OTSFuture<GetRangeResult> 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<String> 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.");
|
||||
}
|
||||
}
|
@ -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<GetRangeResult> future;
|
||||
|
||||
RequestItem(RangeRowQueryCriteria criteria, OTSFuture<GetRangeResult> future) {
|
||||
this.criteria = criteria;
|
||||
this.future = future;
|
||||
}
|
||||
|
||||
public RangeRowQueryCriteria getCriteria() {
|
||||
return criteria;
|
||||
}
|
||||
|
||||
public OTSFuture<GetRangeResult> getFuture() {
|
||||
return future;
|
||||
}
|
||||
}
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(OtsReaderSlaveProxyOld.class);
|
||||
|
||||
private void rowsToSender(List<Row> rows, RecordSender sender, List<OTSColumn> 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<String> 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<String> columns) throws Exception {
|
||||
RangeRowQueryCriteria criteria = generateRangeRowQueryCriteria(conf.getTableName(), begin, end, direction, columns);
|
||||
|
||||
GetRangeRequest request = new GetRangeRequest();
|
||||
request.setRangeRowQueryCriteria(criteria);
|
||||
OTSFuture<GetRangeResult> 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<String> 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<PrimaryKeyColumn> 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;
|
||||
}
|
||||
}
|
@ -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<Column>, JsonSerializer<Column>{
|
||||
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());
|
||||
|
||||
}
|
||||
}
|
||||
}
|
@ -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<OTSColumn>, JsonSerializer<OTSColumn>{
|
||||
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 + "");
|
||||
}
|
||||
}
|
||||
}
|
@ -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,15 +25,13 @@ public class PrimaryKeyValueAdaptor implements JsonDeserializer<PrimaryKeyValue>
|
||||
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;
|
||||
}
|
||||
|
||||
@ -52,6 +44,10 @@ public class PrimaryKeyValueAdaptor implements JsonDeserializer<PrimaryKeyValue>
|
||||
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<PrimaryKeyValue>
|
||||
|
||||
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<PrimaryKeyValue>
|
||||
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 + "");
|
||||
}
|
||||
|
@ -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<List<PrimaryKeyColumn>> {
|
||||
|
||||
public class GetFirstRowPrimaryKeyCallable implements Callable<RowPrimaryKey>{
|
||||
|
||||
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<PrimaryKeyColumn> call() throws Exception {
|
||||
List<PrimaryKeyColumn> ret = new ArrayList<>();
|
||||
GetRangeRequest request = new GetRangeRequest();
|
||||
request.setRangeRowQueryCriteria(criteria);
|
||||
GetRangeResult result = ots.getRange(request);
|
||||
List<Row> rows = result.getRows();
|
||||
if(rows.isEmpty()) {
|
||||
GetRangeResponse response = ots.getRange(request);
|
||||
List<Row> rows = response.getRows();
|
||||
if (rows.isEmpty()) {
|
||||
return null;// no data
|
||||
}
|
||||
Row row = rows.get(0);
|
||||
|
||||
Map<String, PrimaryKeyType> 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<String, PrimaryKeyType> pk = meta.getPrimaryKeyMap();
|
||||
|
||||
for (String key : pk.keySet()) {
|
||||
PrimaryKeyColumn v = row.getPrimaryKey().getPrimaryKeyColumnsMap().get(key);
|
||||
ret.add(v);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -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<GetRangeResponse> {
|
||||
|
||||
public class GetRangeCallable implements Callable<GetRangeResult> {
|
||||
|
||||
private OTSClientAsync ots;
|
||||
private SyncClientInterface ots;
|
||||
private RangeRowQueryCriteria criteria;
|
||||
private OTSFuture<GetRangeResult> future;
|
||||
|
||||
public GetRangeCallable(OTSClientAsync ots, RangeRowQueryCriteria criteria, OTSFuture<GetRangeResult> 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) {
|
||||
public GetRangeResponse call() throws Exception {
|
||||
GetRangeRequest request = new GetRangeRequest();
|
||||
request.setRangeRowQueryCriteria(criteria);
|
||||
future = ots.getRange(request);
|
||||
throw e;
|
||||
return ots.getRange(request);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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<GetRangeResult> {
|
||||
|
||||
private OTSClientAsync ots;
|
||||
private RangeRowQueryCriteria criteria;
|
||||
private OTSFuture<GetRangeResult> future;
|
||||
|
||||
public GetRangeCallableOld(OTSClientAsync ots, RangeRowQueryCriteria criteria, OTSFuture<GetRangeResult> 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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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<TableMeta>{
|
||||
|
||||
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,7 +22,7 @@ public class GetTableMetaCallable implements Callable<TableMeta>{
|
||||
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;
|
||||
}
|
||||
|
@ -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<List<TimeseriesScanSplitInfo>> {
|
||||
|
||||
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<TimeseriesScanSplitInfo> call() throws Exception {
|
||||
SplitTimeseriesScanTaskRequest request = new SplitTimeseriesScanTaskRequest(timeseriesTableName, splitCountHint);
|
||||
if (measurementName.length() != 0) {
|
||||
request.setMeasurementName(measurementName);
|
||||
}
|
||||
|
||||
SplitTimeseriesScanTaskResponse response = client.splitTimeseriesScanTask(request);
|
||||
return response.getSplitInfos();
|
||||
}
|
||||
}
|
@ -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<ScanTimeseriesDataResponse> {
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -1,18 +1,17 @@
|
||||
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, // 普通列
|
||||
@ -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() {
|
||||
@ -66,11 +64,24 @@ public class OTSColumn {
|
||||
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;
|
||||
}
|
||||
}
|
@ -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<OTSColumn> column = null;
|
||||
private OTSMode mode = null;
|
||||
|
||||
private List<PrimaryKeyValue> rangeBegin = null;
|
||||
private List<PrimaryKeyValue> rangeEnd = null;
|
||||
private List<PrimaryKeyValue> rangeSplit = null;
|
||||
@Deprecated
|
||||
private String metaMode = "";
|
||||
|
||||
private List<OTSColumn> columns = null;
|
||||
private boolean newVersion = false;
|
||||
/**
|
||||
* 以下配置仅用于timeseries数据读取
|
||||
*/
|
||||
private boolean isTimeseriesTable = false;
|
||||
private String measurementName = null;
|
||||
/**
|
||||
* 以上配置仅用于timeseries数据读取
|
||||
*/
|
||||
private OTSMultiVersionConf multi = 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<OTSColumn> getColumns() {
|
||||
return columns;
|
||||
public OTSRange getRange() {
|
||||
return range;
|
||||
}
|
||||
public void setColumns(List<OTSColumn> 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<PrimaryKeyValue> getRangeBegin() {
|
||||
return rangeBegin;
|
||||
|
||||
public List<OTSColumn> getColumn() {
|
||||
return column;
|
||||
}
|
||||
public void setRangeBegin(List<PrimaryKeyValue> rangeBegin) {
|
||||
this.rangeBegin = rangeBegin;
|
||||
|
||||
public void setColumn(List<OTSColumn> column) {
|
||||
this.column = column;
|
||||
}
|
||||
public List<PrimaryKeyValue> getRangeEnd() {
|
||||
return rangeEnd;
|
||||
|
||||
public boolean isNewVersion() {
|
||||
return newVersion;
|
||||
}
|
||||
public void setRangeEnd(List<PrimaryKeyValue> rangeEnd) {
|
||||
this.rangeEnd = rangeEnd;
|
||||
|
||||
public void setNewVersion(boolean newVersion) {
|
||||
this.newVersion = newVersion;
|
||||
}
|
||||
public List<PrimaryKeyValue> getRangeSplit() {
|
||||
return rangeSplit;
|
||||
|
||||
@Deprecated
|
||||
public String getMetaMode() {
|
||||
return metaMode;
|
||||
}
|
||||
public void setRangeSplit(List<PrimaryKeyValue> 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;
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -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";
|
||||
|
||||
}
|
@ -0,0 +1,6 @@
|
||||
package com.alibaba.datax.plugin.reader.otsreader.model;
|
||||
|
||||
public enum OTSMode {
|
||||
NORMAL,
|
||||
MULTI_VERSION
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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<PrimaryKeyColumn> begin = null;
|
||||
private List<PrimaryKeyColumn> end = null;
|
||||
private List<PrimaryKeyColumn> 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<PrimaryKeyColumn> getBegin() {
|
||||
return begin;
|
||||
}
|
||||
public void setBegin(RowPrimaryKey begin) {
|
||||
public void setBegin(List<PrimaryKeyColumn> begin) {
|
||||
this.begin = begin;
|
||||
}
|
||||
public RowPrimaryKey getEnd() {
|
||||
public List<PrimaryKeyColumn> getEnd() {
|
||||
return end;
|
||||
}
|
||||
public void setEnd(RowPrimaryKey end) {
|
||||
public void setEnd(List<PrimaryKeyColumn> end) {
|
||||
this.end = end;
|
||||
}
|
||||
public List<PrimaryKeyColumn> getSplit() {
|
||||
return split;
|
||||
}
|
||||
public void setSplit(List<PrimaryKeyColumn> split) {
|
||||
this.split = split;
|
||||
}
|
||||
}
|
||||
|
@ -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<String> toColumnToGet(List<OTSColumn> columns, TableMeta meta) {
|
||||
Map<String, PrimaryKeyType> pk = meta.getPrimaryKeyMap();
|
||||
List<String> names = new ArrayList<String>();
|
||||
for (OTSColumn c : columns) {
|
||||
if (c.getColumnType() == OTSColumn.OTSColumnType.NORMAL && !pk.containsKey(c.getName())) {
|
||||
names.add(c.getName());
|
||||
}
|
||||
}
|
||||
return names;
|
||||
}
|
||||
|
||||
public static List<String> getPrimaryKeyNameList(TableMeta meta) {
|
||||
List<String> names = new ArrayList<String>();
|
||||
names.addAll(meta.getPrimaryKeyMap().keySet());
|
||||
return names;
|
||||
}
|
||||
|
||||
public static OTSPrimaryKeyColumn getPartitionKey(TableMeta meta) {
|
||||
List<String> keys = new ArrayList<String>();
|
||||
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<PrimaryKeyColumn> begin, List<PrimaryKeyColumn> 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<PrimaryKeyColumn> begin, List<PrimaryKeyColumn> 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<String, PrimaryKeyValue> beginMap = new HashMap<>();
|
||||
Map<String, PrimaryKeyValue> 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) {
|
||||
@ -58,104 +117,19 @@ public class Common {
|
||||
return 0;
|
||||
}
|
||||
|
||||
public static OTSPrimaryKeyColumn getPartitionKey(TableMeta meta) {
|
||||
List<String> keys = new ArrayList<String>();
|
||||
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<String> getPrimaryKeyNameList(TableMeta meta) {
|
||||
List<String> names = new ArrayList<String>();
|
||||
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<String> getNormalColumnNameList(List<OTSColumn> columns) {
|
||||
List<String> normalColumns = new ArrayList<String>();
|
||||
for (OTSColumn col : columns) {
|
||||
if (col.getColumnType() == OTSColumn.OTSColumnType.NORMAL) {
|
||||
normalColumns.add(col.getName());
|
||||
}
|
||||
}
|
||||
return normalColumns;
|
||||
}
|
||||
|
||||
public static Record parseRowToLine(Row row, List<OTSColumn> columns, Record line) {
|
||||
Map<String, ColumnValue> 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;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
return sleepTime;
|
||||
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 + ".");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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<String> getNormalColumnNameList(List<OTSColumn> columns) {
|
||||
List<String> normalColumns = new ArrayList<String>();
|
||||
for (OTSColumn col : columns) {
|
||||
if (col.getColumnType() == OTSColumn.OTSColumnType.NORMAL) {
|
||||
normalColumns.add(col.getName());
|
||||
}
|
||||
}
|
||||
return normalColumns;
|
||||
}
|
||||
|
||||
public static Record parseRowToLine(Row row, List<OTSColumn> columns, Record line) {
|
||||
Map<String, ColumnValue> 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;
|
||||
}
|
||||
}
|
@ -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<PrimaryKeyColumn> before, List<PrimaryKeyColumn> 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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
|
||||
@ -41,9 +44,24 @@ public class GsonParser {
|
||||
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) {
|
||||
@ -51,13 +69,13 @@ public class GsonParser {
|
||||
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<String, String> map) {
|
||||
Gson g = gsonBuilder();
|
||||
return g.toJson(map);
|
||||
}
|
||||
}
|
||||
|
@ -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 */
|
||||
@ -47,4 +47,12 @@ public final class Key {
|
||||
|
||||
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";
|
||||
|
||||
}
|
@ -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<String, String> extraHeaders = new HashMap<String, String>();
|
||||
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<TimeseriesScanSplitInfo> 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
|
||||
);
|
||||
}
|
||||
}
|
@ -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;
|
@ -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) {
|
||||
public static String checkStringAndGet(Configuration param, String key, boolean isTrim) throws OTSCriticalException {
|
||||
try {
|
||||
String value = param.getString(key);
|
||||
if (isTrim) {
|
||||
value = value != null ? value.trim() : null;
|
||||
}
|
||||
if (null == value) {
|
||||
throwNotExistException(key);
|
||||
throwNotExistException();
|
||||
} else if (value.length() == 0) {
|
||||
throwStringLengthZeroException(key);
|
||||
throwStringLengthZeroException();
|
||||
}
|
||||
return value;
|
||||
}
|
||||
|
||||
public static List<Object> checkListAndGet(Configuration param, String key, boolean isCheckEmpty) {
|
||||
List<Object> 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;
|
||||
}
|
||||
|
||||
public static List<Object> checkListAndGet(Map<String, Object> range, String key) {
|
||||
Object obj = range.get(key);
|
||||
if (null == obj) {
|
||||
return null;
|
||||
}
|
||||
return checkListAndGet(range, key, false);
|
||||
}
|
||||
|
||||
public static List<Object> checkListAndGet(Map<String, Object> range, String key, boolean isCheckEmpty) {
|
||||
Object obj = range.get(key);
|
||||
if (null == obj) {
|
||||
throwNotExistException(key);
|
||||
}
|
||||
if (obj instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> value = (List<Object>)obj;
|
||||
if (isCheckEmpty && value.isEmpty()) {
|
||||
throwEmptyException(key);
|
||||
}
|
||||
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<Object> checkListAndGet(Map<String, Object> range, String key, List<Object> defaultList) {
|
||||
Object obj = range.get(key);
|
||||
if (null == obj) {
|
||||
return defaultList;
|
||||
}
|
||||
if (obj instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> value = (List<Object>)obj;
|
||||
return value;
|
||||
} else {
|
||||
throw new IllegalArgumentException("Can not parse list of '" + key + "' from map.");
|
||||
}
|
||||
}
|
||||
|
||||
public static Map<String, Object> checkMapAndGet(Configuration param, String key, boolean isCheckEmpty) {
|
||||
Map<String, Object> 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<PrimaryKeyValue> 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<String, PrimaryKeyType> 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<PrimaryKeyValue> begin, List<PrimaryKeyValue> 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<PrimaryKeyColumn> begin, List<PrimaryKeyColumn> 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<PrimaryKeyColumn> checkInputPrimaryKeyAndGet(TableMeta meta, List<PrimaryKeyValue> 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()));
|
||||
}
|
||||
} else if (cmp < 0) { // 正向
|
||||
if (direction == Direction.BACKWARD) {
|
||||
throw new IllegalArgumentException("Input direction of 'range-split' is BACKWARD, but direction of 'range' is FORWARD.");
|
||||
List<PrimaryKeyColumn> pk = new ArrayList<>();
|
||||
int i = 0;
|
||||
for (Map.Entry<String, PrimaryKeyType> 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 { // 重复列
|
||||
throw new IllegalArgumentException("Multi same column in 'range-split'.");
|
||||
i++;
|
||||
}
|
||||
return pk;
|
||||
}
|
||||
|
||||
public static OTSRange checkRangeAndGet(Configuration param) throws OTSCriticalException {
|
||||
try {
|
||||
OTSRange range = new OTSRange();
|
||||
Map<String, Object> value = param.getMap(Key.RANGE);
|
||||
// 用户可以不用配置range,默认表示导出全表
|
||||
if (value == null) {
|
||||
return range;
|
||||
}
|
||||
|
||||
/**
|
||||
* 检查 points中的所有点是否是在Begin和end之间
|
||||
* @param begin
|
||||
* @param end
|
||||
* @param points
|
||||
* Range格式:{
|
||||
* "begin":[],
|
||||
* "end":[]
|
||||
* }
|
||||
*/
|
||||
private static void checkPointsRange(Direction direction, PrimaryKeyValue begin, PrimaryKeyValue end, List<PrimaryKeyValue> 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'.");
|
||||
|
||||
// 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<String, Object> 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);
|
||||
}
|
||||
}
|
||||
|
||||
private static void checkColumnByMode(List<OTSColumn> 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<OTSColumn> checkOTSColumnAndGet(Configuration param, OTSMode mode) throws OTSCriticalException {
|
||||
try {
|
||||
List<Object> value = param.getList(Key.COLUMN);
|
||||
// 用户可以不用配置Column
|
||||
if (value == null) {
|
||||
value = Collections.emptyList();
|
||||
}
|
||||
|
||||
/**
|
||||
* Column格式:[
|
||||
* {"Name":"pk1"},
|
||||
* {"type":"Binary","value" : "base64()"}
|
||||
* ]
|
||||
*/
|
||||
List<OTSColumn> columns = ParamParser.parseOTSColumnArray(value);
|
||||
checkColumnByMode(columns, mode);
|
||||
return columns;
|
||||
} catch (RuntimeException e) {
|
||||
throw new OTSCriticalException("Parse 'column' fail, " + e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
public static List<OTSColumn> checkTimeseriesColumnAndGet(Configuration param) throws OTSCriticalException {
|
||||
try {
|
||||
List<Object> value = param.getList(Key.COLUMN);
|
||||
List<OTSColumn> columns = ParamParser.parseOTSColumnArray(value);
|
||||
|
||||
List<ColumnType> columnTypes = checkColumnTypeAndGet(param);
|
||||
List<Boolean> 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<ColumnType> checkColumnTypeAndGet(Configuration param) throws OTSCriticalException {
|
||||
try {
|
||||
List<Object> value = param.getList(Key.COLUMN);
|
||||
List<ColumnType> columnTypes = ParamParser.parseColumnTypeArray(value);
|
||||
return columnTypes;
|
||||
} catch (RuntimeException e) {
|
||||
throw new OTSCriticalException("Parse 'type of column' fail, " + e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
public static List<Boolean> checkColumnIsTagAndGet(Configuration param) throws OTSCriticalException {
|
||||
try {
|
||||
List<Object> value = param.getList(Key.COLUMN);
|
||||
List<Boolean> 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<PrimaryKeyColumn> checkAndGetPrimaryKey(
|
||||
List<PrimaryKeyColumn> pk,
|
||||
List<PrimaryKeySchema> pkSchema,
|
||||
String jsonKey){
|
||||
List<PrimaryKeyColumn> result = new ArrayList<PrimaryKeyColumn>();
|
||||
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<PrimaryKeyColumn>();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 检查split的类型是否和PartitionKey一致
|
||||
* @param points
|
||||
* @param pkSchema
|
||||
*/
|
||||
private static List<PrimaryKeyColumn> checkAndGetSplit(
|
||||
List<PrimaryKeyColumn> points,
|
||||
List<PrimaryKeySchema> pkSchema){
|
||||
List<PrimaryKeyColumn> result = new ArrayList<PrimaryKeyColumn>();
|
||||
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<PrimaryKeySchema> pkSchema, List<PrimaryKeyColumn> 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<PrimaryKeyColumn> begin,
|
||||
List<PrimaryKeyColumn> end,
|
||||
List<PrimaryKeySchema> 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<PrimaryKeyColumn> begin,
|
||||
List<PrimaryKeyColumn> end,
|
||||
List<PrimaryKeyColumn> 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<PrimaryKeyValue> points) {
|
||||
if (null == points || points.isEmpty()) {
|
||||
return;
|
||||
private static void fillAndcheckBeginAndEndAndSplit(
|
||||
List<PrimaryKeyColumn> begin,
|
||||
List<PrimaryKeyColumn> end,
|
||||
List<PrimaryKeyColumn> split,
|
||||
List<PrimaryKeySchema> pkSchema
|
||||
) {
|
||||
|
||||
fillBeginAndEnd(begin, end, pkSchema);
|
||||
checkBeginAndEndAndSplit(begin, end, split);
|
||||
}
|
||||
|
||||
OTSPrimaryKeyColumn part = Common.getPartitionKey(meta);
|
||||
public static void checkAndSetOTSRange(OTSRange range, TableMeta meta) throws OTSCriticalException {
|
||||
try {
|
||||
List<PrimaryKeySchema> pkSchema = meta.getPrimaryKeyList();
|
||||
|
||||
// 处理第一个
|
||||
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());
|
||||
// 检查是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);
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = 0 ; i < points.size() - 1; i++) {
|
||||
PrimaryKeyValue before = points.get(i);
|
||||
PrimaryKeyValue after = points.get(i + 1);
|
||||
checkDirection(direction, before, after);
|
||||
public static void checkAndSetColumn(List<OTSColumn> columns, TableMeta meta, OTSMode mode) throws OTSCriticalException {
|
||||
try {
|
||||
if (mode == OTSMode.MULTI_VERSION) {
|
||||
Set<String> uniqueColumn = new HashSet<String>();
|
||||
Map<String, PrimaryKeyType> 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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
PrimaryKeyValue begin = range.getBegin().getPrimaryKey().get(part.getName());
|
||||
PrimaryKeyValue end = range.getEnd().getPrimaryKey().get(part.getName());
|
||||
|
||||
checkPointsRange(direction, begin, end, points);
|
||||
} 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());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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<Object> checkListAndGet(Configuration param, String key, boolean isCheckEmpty) {
|
||||
List<Object> 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;
|
||||
}
|
||||
|
||||
}
|
@ -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<String, Object> 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<PrimaryKeyColumn> parsePrimaryKeyColumnArray(Object arrayObj) throws OTSCriticalException {
|
||||
try {
|
||||
List<PrimaryKeyColumn> columns = new ArrayList<PrimaryKeyColumn>();
|
||||
if (arrayObj instanceof List) {
|
||||
List<Object> array = (List<Object>) arrayObj;
|
||||
for (Object o : array) {
|
||||
if (o instanceof Map) {
|
||||
Map<String, Object> column = (Map<String, Object>) 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<String, Object> 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<OTSColumn> parseOTSColumnArray(List<Object> value) throws OTSCriticalException {
|
||||
try {
|
||||
List<OTSColumn> result = new ArrayList<OTSColumn>();
|
||||
for (Object item:value) {
|
||||
if (item instanceof Map){
|
||||
Map<String, Object> column = (Map<String, Object>) 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<String, Object> 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<ColumnType> parseColumnTypeArray(List<Object> value) throws OTSCriticalException {
|
||||
try {
|
||||
List<ColumnType> result = new ArrayList<ColumnType>();
|
||||
for (Object item:value) {
|
||||
if (item instanceof Map){
|
||||
Map<String, Object> column = (Map<String, Object>) 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<String, Object> 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<Boolean> parseColumnIsTagArray(List<Object> value) throws OTSCriticalException {
|
||||
try {
|
||||
List<Boolean> result = new ArrayList<Boolean>();
|
||||
for (Object item:value) {
|
||||
if (item instanceof Map){
|
||||
Map<String, Object> column = (Map<String, Object>) 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());
|
||||
}
|
||||
}
|
||||
}
|
@ -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<String> 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<Long> splitIntegerRange(BigInteger bigBegin, BigInteger bigEnd, BigInteger bigCount) {
|
||||
@ -228,20 +224,31 @@ public class RangeSplit {
|
||||
}
|
||||
|
||||
public static List<OTSRange> rangeSplitByCount(TableMeta meta,
|
||||
RowPrimaryKey begin, RowPrimaryKey end, int count) {
|
||||
List<PrimaryKeyColumn> begin, List<PrimaryKeyColumn> end, int count) {
|
||||
List<OTSRange> results = new ArrayList<OTSRange>();
|
||||
|
||||
OTSPrimaryKeyColumn partitionKey = Common.getPartitionKey(meta);
|
||||
|
||||
PrimaryKeyValue beginPartitionKey = begin.getPrimaryKey().get(
|
||||
Map<String, PrimaryKeyValue> beginMap = new HashMap<>();
|
||||
Map<String, PrimaryKeyValue> 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<PrimaryKeyValue> 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<PrimaryKeyColumn> bPk = new ArrayList<>();
|
||||
List<PrimaryKeyColumn> 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<String> keys = new ArrayList<String>(meta.getPrimaryKey().size());
|
||||
keys.addAll(meta.getPrimaryKey().keySet());
|
||||
List<String> keys = new ArrayList<String>(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<PrimaryKeyColumn> beginPK = c.getBegin();
|
||||
List<PrimaryKeyColumn> 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<PrimaryKeyValue> getCompletePK(int num,
|
||||
PrimaryKeyValue value) {
|
||||
List<PrimaryKeyValue> values = new ArrayList<PrimaryKeyValue>();
|
||||
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<PrimaryKeyValue> getSplitPoint(PrimaryKeyValue begin, PrimaryKeyValue end, List<PrimaryKeyValue> target) {
|
||||
List<PrimaryKeyValue> result = new ArrayList<PrimaryKeyValue>();
|
||||
|
||||
int cmp = Common.primaryKeyValueCmp(begin, end);
|
||||
|
||||
if (cmp == 0) {
|
||||
return result;
|
||||
}
|
||||
|
||||
result.add(begin);
|
||||
|
||||
Comparator<PrimaryKeyValue> comparator = new Comparator<PrimaryKeyValue>(){
|
||||
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<OTSRange> rangeSplitByPoint(TableMeta meta, RowPrimaryKey beginPK, RowPrimaryKey endPK,
|
||||
List<PrimaryKeyValue> splits) {
|
||||
|
||||
List<OTSRange> results = new ArrayList<OTSRange>();
|
||||
|
||||
int pkCount = meta.getPrimaryKey().size();
|
||||
|
||||
String partName = Common.getPartitionKey(meta).getName();
|
||||
PrimaryKeyValue begin = beginPK.getPrimaryKey().get(partName);
|
||||
PrimaryKeyValue end = endPK.getPrimaryKey().get(partName);
|
||||
|
||||
List<PrimaryKeyValue> 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;
|
||||
}
|
||||
}
|
||||
|
@ -55,7 +55,7 @@ public class ReaderModelParser {
|
||||
}
|
||||
|
||||
public static OTSColumn parseOTSColumn(Map<String, Object> 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;
|
||||
|
@ -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> V executeWithRetry(Callable<V> 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;
|
||||
}
|
||||
}
|
||||
|
@ -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<String> noRetryErrorCode = prepareNoRetryErrorCode();
|
||||
|
||||
public static <V> V executeWithRetry(Callable<V> 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<String> prepareNoRetryErrorCode() {
|
||||
Set<String> pool = new HashSet<String>();
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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() + ".");
|
||||
}
|
||||
}
|
||||
}
|
@ -1,127 +1,152 @@
|
||||
## TableStore增量数据导出通道:TableStoreStreamReader
|
||||
|
||||
### 快速介绍
|
||||
|
||||
TableStoreStreamReader插件主要用于TableStore的增量数据导出,增量数据可以看作操作日志,除了数据本身外还附有操作信息。
|
||||
|
||||
与全量导出插件不同,增量导出插件只有多版本模式,同时不支持指定列。这是与增量导出的原理有关的,导出的格式下面有详细介绍。
|
||||
|
||||
使用插件前必须确保表上已经开启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);
|
||||
|
||||
### 实现原理
|
||||
|
||||
首先用户使用SDK的UpdateTable功能,指定开启Stream并设置过期时间,即开启了增量功能。
|
||||
|
||||
开启后,TableStore服务端就会将用户的操作日志额外保存起来,
|
||||
每个分区有一个有序的操作日志队列,每条操作日志会在一定时间后被垃圾回收,这个时间即用户指定的过期时间。
|
||||
|
||||
TableStore的SDK提供了几个Stream相关的API用于将这部分操作日志读取出来,增量插件也是通过TableStore SDK的接口获取到增量数据的,并将
|
||||
增量数据转化为多个6元组的形式(pk, colName, version, colValue, opType, sequenceInfo)导入到ODPS中。
|
||||
|
||||
### 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插件用于记录状态的表的名称,这些状态可用于减少对非目标范围内的数据的扫描,从而加快导出速度。<br> 1. 用户不需要创建该表,只需要给出一个表名。Reader插件会尝试在用户的instance下创建该表,若该表不存在即创建新表,若该表已存在,会判断该表的Meta是否与期望一致,若不一致会抛出异常。<br> 2. 在一次导出完成之后,用户不应删除该表,该表的状态可用于下次导出任务。<br> 3. 该表会开启TTL,数据自动过期,因此可认为其数据量很小。<br> 4. 针对同一个instance下的多个不同的dataTable的Reader配置,可以使用同一个statusTable,记录的状态信息互不影响。 <br> 综上,用户配置一个类似TableStoreStreamReaderStatusTable之类的名称即可,注意不要与业务相关的表重名。| String | 是 |
|
||||
| startTimestampMillis | 增量数据的时间范围(左闭右开)的左边界,单位毫秒。 <br> 1. Reader插件会从statusTable中找对应startTimestampMillis的位点,从该点开始读取开始导出数据。 <br> 2. 若statusTable中找不到对应的位点,则从系统保留的增量数据的第一条开始读取,并跳过写入时间小于startTimestampMillis的数据。| Long | 否 |
|
||||
| endTimestampMillis | 增量数据的时间范围(左闭右开)的右边界,单位毫秒。<br> 1. Reader插件从startTimestampMillis位置开始导出数据后,当遇到第一条时间戳大于等于endTimestampMillis的数据时,结束导出数据,导出完成。 <br> 2. 当读取完当前全部的增量数据时,结束读取,即使未达到endTimestampMillis。 | Long | 否 |
|
||||
| date | 日期格式为yyyyMMdd,如20151111,表示导出该日的数据。<br> 若没有指定date,则必须指定startTimestampMillis和endTimestampMillis,反之也成立。 | String | 否 |
|
||||
| isExportSequenceInfo | 是否导出时序信息,时序信息包含了数据的写入时间等。默认该值为false,即不导出。 | Boolean | 否 |
|
||||
| maxRetries | 从TableStore中读增量数据时,每次请求的最大重试次数,默认为30,重试之间有间隔,30次重试总时间约为5分钟,一般无需更改。| Int | 否 |
|
||||
|
||||
### 导出的数据格式
|
||||
首先,在TableStore多版本模型下,表中的数据组织为“行-列-版本”三级的模式,
|
||||
一行可以有任意列,列名也并非固定的,每一列可以含有多个版本,每个版本都有一个特定的时间戳(版本号)。
|
||||
|
||||
用户可以通过TableStore的API进行一系列读写操作,
|
||||
TableStore通过记录用户最近对表的一系列写操作(或称为数据更改操作)来实现记录增量数据的目的,
|
||||
所以也可以把增量数据看作一批操作记录。
|
||||
|
||||
TableStore有三类数据更改操作:PutRow、UpdateRow、DeleteRow。
|
||||
|
||||
+ PutRow的语义是写入一行,若该行已存在即覆盖该行。
|
||||
|
||||
+ UpdateRow的语义是更新一行,对原行其他数据不做更改,
|
||||
更新可能包括新增或覆盖(若对应列的对应版本已存在)一些列值、删除某一列的全部版本、删除某一列的某个版本。
|
||||
|
||||
+ DeleteRow的语义是删除一行。
|
||||
|
||||
TableStore会根据每种操作生成对应的增量数据记录,Reader插件会读出这些记录,并导出成Datax的数据格式。
|
||||
|
||||
同时,由于TableStore具有动态列、多版本的特性,所以Reader插件导出的一行不对应TableStore中的一行,而是对应TableStore中的一列的一个版本。
|
||||
即<B>TableStore中的一行可能会导出很多行,每行包含主键值、该列的列名、该列下该版本的时间戳(版本号)、该版本的值、操作类型</B>。若设置isExportSequenceInfo为true,还会包括时序信息。
|
||||
|
||||
转换为Datax的数据格式后,我们定义了四种操作类型,分别为:
|
||||
|
||||
+ U(UPDATE): 写入一列的一个版本
|
||||
|
||||
+ DO(DELETE_ONE_VERSION): 删除某一列的某个版本
|
||||
|
||||
+ DA(DELETE_ALL_VERSION): 删除某一列的全部版本,此时需要根据主键和列名,将对应列的全部版本删除
|
||||
|
||||
+ DR(DELETE_ROW): 删除某一行,此时需要根据主键,将该行数据全部删除
|
||||
本文为您介绍OTSStream Reader支持的数据类型、读取方式、字段映射和数据源等参数及配置示例。
|
||||
## 列模式
|
||||
|
||||
|
||||
举例如下,假设该表有两个主键列,主键列名分别为pkName1, pkName2:
|
||||
### 背景信息
|
||||
|
||||
| pkName1 | pkName2 | columnName | timestamp | columnValue | opType |
|
||||
| ------- | ------- | ---------- | --------- | ----------- | ------ |
|
||||
OTSStream Reader插件主要用于导出Table Store的增量数据。您可以将增量数据看作操作日志,除数据本身外还附有操作信息。
|
||||
|
||||
与全量导出插件不同,增量导出插件只有多版本模式,且不支持指定列。使用插件前,您必须确保表上已经开启Stream功能。您可以在建表时指定开启,也可以使用SDK的UpdateTable接口开启。
|
||||
|
||||
开启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中。
|
||||
|
||||
### 列模式
|
||||
|
||||
在Table Store多版本模型下,表中的数据组织为行>列>版本三级的模式, 一行可以有任意列,列名并不是固定的,每一列可以含有多个版本,每个版本都有一个特定的时间戳(版本号)。
|
||||
|
||||
您可以通过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_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)。
|
||||
假设导出的数据如上,共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 |
|
||||
|
||||
|
||||
|
||||
### 参数说明
|
||||
|
||||
| **参数** | **描述** | **是否必选** | **默认值** |
|
||||
| --- |---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| --- |---------|
|
||||
| **dataSource** | 数据源名称,脚本模式支持添加数据源,该配置项填写的内容必须与添加的数据源名称保持一致。 | 是 | 无 |
|
||||
| **dataTable** | 导出增量数据的表的名称。该表需要开启Stream,可以在建表时开启,或者使用UpdateTable接口开启。 | 是 | 无 |
|
||||
| **statusTable** | Reader插件用于记录状态的表的名称,这些状态可用于减少对非目标范围内的数据的扫描,从而加快导出速度。statusTable是Reader用于保存状态的表,如果该表不存在,Reader会自动创建该表。一次离线导出任务完成后,您无需删除该表,该表中记录的状态可用于下次导出任务中:<li> 您无需创建该表,只需要给出一个表名。Reader插件会尝试在您的instance下创建该表,如果该表不存在即创建新表。如果该表已存在,会判断该表的Meta是否与期望一致,如果不一致会抛出异常。<li>在一次导出完成之后,您无需删除该表,该表的状态可以用于下次的导出任务。<li> 该表会开启TTL,数据自动过期,会认为其数据量很小。<li> 针对同一个instance下的多个不同的dataTable的Reader配置,可以使用同一个statusTable,记录的状态信息互不影响。您配置一个类似**TableStoreStreamReaderStatusTable**的名称即可,请注意不要与业务相关的表重名。 | 是 | 无 |
|
||||
| **startTimestampMillis** | 增量数据的时间范围(左闭右开)的左边界,单位为毫秒: <li> Reader插件会从statusTable中找对应**startTimestampMillis**的位点,从该点开始读取开始导出数据。<li> 如果statusTable中找不到对应的位点,则从系统保留的增量数据的第一条开始读取,并跳过写入时间小于**startTimestampMillis**的数据。 | 否 | 无 |
|
||||
| **endTimestampMillis** | 增量数据的时间范围(左闭右开)的右边界,单位为毫秒:<li> Reader插件从**startTimestampMillis**位置开始导出数据后,当遇到第一条时间戳大于等于**endTimestampMillis**的数据时,结束导出数据,导出完成。<li> 当读取完当前全部的增量数据时,即使未达到**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 |
|
||||
|
||||
对于主键为(pk1_V1, pk2_V1)的一行,包含三个操作,分别是写入col_a列的两个版本和col_b列的一个版本。
|
||||
|
||||
对于主键为(pk1_V2, pk2_V2)的一行,包含两个操作,分别是删除col_a列的一个版本、删除col_b列的全部版本。
|
||||
|
||||
对于主键为(pk1_V3, pk2_V3)的一行,包含两个操作,分别是删除整行、写入col_a列的一个版本。
|
||||
|
@ -10,19 +10,20 @@
|
||||
</parent>
|
||||
<groupId>com.alibaba.datax</groupId>
|
||||
<artifactId>otsstreamreader</artifactId>
|
||||
<version>0.0.1</version>
|
||||
<version>0.0.1-SNAPSHOT</version>
|
||||
|
||||
<dependencies>
|
||||
<!--Other 依赖 -->
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-api</artifactId>
|
||||
<version>2.17.1</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<version>2.17.1</version>
|
||||
<groupId>com.aliyun.openservices</groupId>
|
||||
<artifactId>tablestore-streamclient</artifactId>
|
||||
<version>1.0.0</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.aliyun.openservices</groupId>
|
||||
<artifactId>tablestore</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.alibaba.datax</groupId>
|
||||
@ -33,22 +34,28 @@
|
||||
<artifactId>slf4j-log4j12</artifactId>
|
||||
<groupId>org.slf4j</groupId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<artifactId>logback-classic</artifactId>
|
||||
<groupId>ch.qos.logback</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ch.qos.logback</groupId>
|
||||
<artifactId>logback-classic</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.alibaba</groupId>
|
||||
<artifactId>fastjson</artifactId>
|
||||
<version>1.2.83_noneautotype</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<!--Other 依赖 -->
|
||||
<dependency>
|
||||
<groupId>com.aliyun.openservices</groupId>
|
||||
<artifactId>tablestore-streamclient</artifactId>
|
||||
<version>1.0.0</version>
|
||||
<artifactId>tablestore</artifactId>
|
||||
<version>5.13.12</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>log4j-api</artifactId>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
@ -60,12 +67,6 @@
|
||||
<artifactId>gson</artifactId>
|
||||
<version>2.2.4</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
<version>18.0</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
<build>
|
||||
@ -106,6 +107,18 @@
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<!-- test runner -->
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<version>2.5</version>
|
||||
<configuration>
|
||||
<includes>
|
||||
<include>**/unittest/*.java</include>
|
||||
<include>**/functiontest/*.java</include>
|
||||
</includes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
||||
|
@ -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<String> ownedShards = GsonParser.jsonToList(
|
||||
(String) this.getPluginJobConf().get(OTSStreamReaderConstants.OWNED_SHARDS));
|
||||
List<StreamShard> allShards = GsonParser.fromJson(
|
||||
|
||||
boolean confSimplifyEnable = this.getPluginJobConf().getBool(CONF_SIMPLIFY_ENABLE,
|
||||
DEFAULT_CONF_SIMPLIFY_ENABLE_VALUE);
|
||||
|
||||
StreamJob streamJob;
|
||||
List<StreamShard> 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<String>(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();
|
||||
}
|
||||
|
@ -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<StreamShard> 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<String> shardIds = new ArrayList<String>();
|
||||
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<String>(shardIds),
|
||||
conf.getStartTimestampMillis(), conf.getEndTimestampMillis());
|
||||
@ -97,8 +101,16 @@ public class OTSStreamReaderMasterProxy {
|
||||
|
||||
Configuration configuration = Configuration.newDefault();
|
||||
configuration.set(OTSStreamReaderConstants.CONF, GsonParser.configToJson(conf));
|
||||
|
||||
// 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);
|
||||
}
|
||||
|
@ -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<StreamShard> allShards, Set<String> 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<String, StreamShard>();
|
||||
@ -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<String, StreamShard> shard : ownedShards.entrySet()) {
|
||||
LOG.info("Shard to process, ShardInfo: [{}], StartCheckpoint: [{}].", shard.getValue(), shardToCheckpointMap.get(shard.getKey()));
|
||||
}
|
||||
|
@ -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<String> columns;
|
||||
private List<Boolean> 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<Boolean> getColumnsIsTimeseriesTags() {
|
||||
return columnsIsTimeseriesTags;
|
||||
}
|
||||
|
||||
public void setColumnsIsTimeseriesTags(List<Boolean> 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<Object> values = param.getList(KEY_COLUMN);
|
||||
if (values == null) {
|
||||
config.setColumns(new ArrayList<String>());
|
||||
config.setColumnsIsTimeseriesTags(new ArrayList<Boolean>());
|
||||
return;
|
||||
}
|
||||
|
||||
List<String> columns = new ArrayList<String>();
|
||||
List<Boolean> columnsIsTimeseriesTags = new ArrayList<Boolean>();
|
||||
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,6 +247,9 @@ 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));
|
||||
@ -205,29 +277,29 @@ public class OTSStreamReaderConfig {
|
||||
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);
|
||||
throw new OTSStreamReaderException("Can't parse endTimeString: " + endTime + ", expect format date like '201801151612'.");
|
||||
}
|
||||
|
||||
}else if (param.getString(KEY_DATE) == null) {
|
||||
} else if (param.getString(KEY_DATE) == null) {
|
||||
config.setStartTimestampMillis(param.getLong(KEY_START_TIMESTAMP_MILLIS));
|
||||
config.setEndTimestampMillis(param.getLong(KEY_END_TIMESTAMP_MILLIS));
|
||||
} else {
|
||||
@ -242,8 +314,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<Object> 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;
|
||||
}
|
||||
}
|
||||
|
@ -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");
|
||||
|
@ -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<String, StreamShard> 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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
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<StreamRecord> records, String nextShardIterator) {
|
||||
boolean process(List<StreamRecord> records, String nextShardIterator, Boolean mayMoreRecord) {
|
||||
if (records.isEmpty() && nextShardIterator != null) {
|
||||
// 没有读到更多数据
|
||||
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<StreamRecord> 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<StreamRecord> 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();
|
||||
}
|
||||
}
|
@ -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 |
|
||||
*
|
||||
* <p>
|
||||
* 注意:删除整行,删除某列(某个版本或所有),这些增量信息都会被忽略。
|
||||
*/
|
||||
public class SingleVerAndUpOnlyModeRecordSender implements IStreamRecordSender {
|
||||
|
||||
private final RecordSender dataxRecordSender;
|
||||
private String shardId;
|
||||
private final boolean isExportSequenceInfo;
|
||||
private String shardId;
|
||||
private List<String> columnNames;
|
||||
private List<Boolean> columnsIsTimeseriesTags;
|
||||
|
||||
public SingleVerAndUpOnlyModeRecordSender(RecordSender dataxRecordSender, String shardId, boolean isExportSequenceInfo, List<String> columnNames) {
|
||||
public SingleVerAndUpOnlyModeRecordSender(RecordSender dataxRecordSender, String shardId, boolean isExportSequenceInfo, List<String> columnNames, List<Boolean> columnsIsTimeseriesTags) {
|
||||
this.dataxRecordSender = dataxRecordSender;
|
||||
this.shardId = shardId;
|
||||
this.isExportSequenceInfo = isExportSequenceInfo;
|
||||
this.columnNames = columnNames;
|
||||
this.columnsIsTimeseriesTags = columnsIsTimeseriesTags;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -57,6 +60,20 @@ public class SingleVerAndUpOnlyModeRecordSender implements IStreamRecordSender {
|
||||
map.put(pkCol.getName(), pkCol.getValue());
|
||||
}
|
||||
|
||||
/**
|
||||
* 将时序数据中tags字段的字符串转化为Map
|
||||
*/
|
||||
Map<String, String> 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());
|
||||
@ -65,8 +82,17 @@ public class SingleVerAndUpOnlyModeRecordSender implements IStreamRecordSender {
|
||||
|
||||
boolean findColumn = false;
|
||||
|
||||
for (String colName : columnNames) {
|
||||
Object value = map.get(colName);
|
||||
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(new StringColumn(null));
|
||||
}
|
||||
} else {
|
||||
Object value = map.get(columnNames.get(i));
|
||||
if (value != null) {
|
||||
findColumn = true;
|
||||
if (value instanceof ColumnValue) {
|
||||
@ -78,6 +104,7 @@ public class SingleVerAndUpOnlyModeRecordSender implements IStreamRecordSender {
|
||||
line.addColumn(new StringColumn(null));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!findColumn) {
|
||||
return;
|
||||
|
@ -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<StreamShard> allShards;
|
||||
|
||||
public OTSStreamJobShard() {
|
||||
}
|
||||
|
||||
public OTSStreamJobShard(StreamJob streamJob, List<StreamShard> allShards) {
|
||||
this.streamJob = streamJob;
|
||||
this.allShards = allShards;
|
||||
}
|
||||
|
||||
public StreamJob getStreamJob() {
|
||||
return streamJob;
|
||||
}
|
||||
|
||||
public void setStreamJob(StreamJob streamJob) {
|
||||
this.streamJob = streamJob;
|
||||
}
|
||||
|
||||
public List<StreamShard> getAllShards() {
|
||||
return allShards;
|
||||
}
|
||||
|
||||
public void setAllShards(List<StreamShard> allShards) {
|
||||
this.allShards = allShards;
|
||||
}
|
||||
|
||||
}
|
@ -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<StreamShard> 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<StreamShard> getOrderedShardList(SyncClientInterface ots, String streamId, boolean isTimeseriesTable) {
|
||||
DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest(streamId);
|
||||
if (isTimeseriesTable) {
|
||||
describeStreamRequest.setSupportTimeseriesTable(true);
|
||||
}
|
||||
DescribeStreamResponse describeStreamResult = ots.describeStream(describeStreamRequest);
|
||||
List<StreamShard> shardList = new ArrayList<StreamShard>();
|
||||
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 {
|
||||
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<String,StreamShard> toShardMap(List<StreamShard> orderedShardList) {
|
||||
public static Map<String, StreamShard> toShardMap(List<StreamShard> orderedShardList) {
|
||||
Map<String, StreamShard> shardsMap = new HashMap<String, StreamShard>();
|
||||
for (StreamShard shard : orderedShardList) {
|
||||
shardsMap.put(shard.getShardId(), shard);
|
||||
}
|
||||
return shardsMap;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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<OTSStreamJobShard>() {
|
||||
@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<StreamShard> 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<String> 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<String> 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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
179
otsstreamreader/tools/tablestore_streamreader_console.py
Normal file
179
otsstreamreader/tools/tablestore_streamreader_console.py
Normal file
@ -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)
|
||||
|
1237
otsstreamreader/tools/tabulate.py
Normal file
1237
otsstreamreader/tools/tabulate.py
Normal file
File diff suppressed because it is too large
Load Diff
@ -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": {
|
||||
@ -43,42 +39,47 @@ OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务,
|
||||
"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"
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -131,6 +132,81 @@ OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务,
|
||||
|
||||
* 默认值:无 <br />
|
||||
|
||||
* **newVersion**
|
||||
|
||||
* 描述:version定义了使用的ots SDK版本。<br />
|
||||
* true,新版本插件,使用com.alicloud.openservices.tablestore的依赖(推荐)
|
||||
* false,旧版本插件,使用com.aliyun.openservices.ots的依赖,**不支持多版本数据的读取**
|
||||
|
||||
* 必选:否 <br />
|
||||
|
||||
* 默认值:false <br />
|
||||
|
||||
* **mode**
|
||||
|
||||
* 描述:是否为多版本数据,目前有两种模式。<br />
|
||||
* normal,对应普通的数据
|
||||
* multiVersion,写入数据为多版本格式的数据,多版本模式下,配置参数有所不同,详见3.4节
|
||||
|
||||
* 必选:否 <br />
|
||||
|
||||
* 默认值:normal <br />
|
||||
|
||||
|
||||
* **enableAutoIncrement**
|
||||
|
||||
* 描述:是否允许向包含主键自增列的ots表中写入数据。<br />
|
||||
* true,插件会扫描表中的自增列信息,并在写入数据时自动添加自增列
|
||||
* false,写入含主键自增列的表时会报错
|
||||
|
||||
* 必选:否 <br />
|
||||
|
||||
* 默认值:false <br />
|
||||
|
||||
|
||||
* **isTimeseriesTable**
|
||||
|
||||
* 描述:写入的对应表是否为时序表,仅在mode=normal模式下生效。<br />
|
||||
* true,写入的数据表为时序数据表
|
||||
* false,写入的数据表为普通的宽表
|
||||
|
||||
* 必选:否 <br />
|
||||
|
||||
* 默认值:false <br />
|
||||
|
||||
* 在写入时序数据表的模式下,不需要配置`primaryKey`字段,只需要配置`column`字段,配置样例:
|
||||
```json
|
||||
"column": [
|
||||
{
|
||||
"name": "_m_name", // 表示度量名称(measurement)字段
|
||||
},
|
||||
{
|
||||
"name": "_data_source", // 表示数据源(dataSource)字段
|
||||
},
|
||||
{
|
||||
"name": "_tags", // 表示标签字段,会被解析为Map<String,String>类型
|
||||
},
|
||||
{
|
||||
"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导入数据过程中,必须指定相应地字段名称。
|
||||
@ -142,10 +218,10 @@ OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务,
|
||||
配置实例:
|
||||
|
||||
```json
|
||||
"primaryKey" : [
|
||||
{"name":"pk1", "type":"string"},
|
||||
{"name":"pk2", "type":"int"}
|
||||
],
|
||||
"primaryKey":[
|
||||
"userid",
|
||||
"groupid"
|
||||
]
|
||||
```
|
||||
* 必选:是 <br />
|
||||
|
||||
@ -175,11 +251,10 @@ OTS是构建在阿里云飞天分布式系统之上的 NoSQL数据库服务,
|
||||
|
||||
* UpdateRow,对应于OTS API UpdateRow,更新指定行的数据,如果该行不存在,则新增一行;若该行存在,则根据请求的内容在这一行中新增、修改或者删除指定列的值。
|
||||
|
||||
* DeleteRow,对应于OTS API DeleteRow,删除指定行的数据。
|
||||
|
||||
* 必选:是 <br />
|
||||
* 必选:是 <br />
|
||||
|
||||
* 默认值:无 <br />
|
||||
* 默认值:无 <br />
|
||||
|
||||
|
||||
### 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。
|
||||
* 注意:这种模式的数据格式比较特殊,该writer需要reader也提供版本的输出
|
||||
* 当前只有hbase reader 与 ots reader提供这种模式,使用时切记注意
|
||||
#### 3.4.2 配置样例
|
||||
```
|
||||
{
|
||||
"job": {
|
||||
"setting": {
|
||||
},
|
||||
"content": [
|
||||
{
|
||||
"reader": {},
|
||||
"writer": {
|
||||
"name": "otswriter",
|
||||
"parameter": {
|
||||
"endpoint":"",
|
||||
"accessId":"",
|
||||
"accessKey":"",
|
||||
"instanceName":"",
|
||||
"table":"",
|
||||
|
||||
#### 4.1.2 机器参数
|
||||
// 多版本模式,插件会按照多版本模式去解析所有配置
|
||||
"mode":"multiVersion",
|
||||
|
||||
OTS端:3台前端机,5台后端机
|
||||
"newVersion":"true",
|
||||
|
||||
DataX运行端: 24核CPU, 98GB内存
|
||||
// 配置PK信息
|
||||
// 考虑到配置成本,并不需要配置PK在Record(Line)中的位置,要求
|
||||
// Record的格式固定,PK一定在行首,PK之后是columnName,格式如下:
|
||||
// 如:{pk0,pk1,pk2,pk3}, {columnName}, {timestamp}, {value}
|
||||
"primaryKey":[
|
||||
"userid",
|
||||
"groupid"
|
||||
],
|
||||
|
||||
### 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|
|
||||
// 列名前缀过滤
|
||||
// 描述: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中?
|
||||
* : 后面的覆盖前面的数据
|
||||
|
@ -10,17 +10,6 @@
|
||||
<name>otswriter</name>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-api</artifactId>
|
||||
<version>2.17.1</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<version>2.17.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.alibaba.datax</groupId>
|
||||
<artifactId>datax-common</artifactId>
|
||||
@ -44,12 +33,19 @@
|
||||
<dependency>
|
||||
<groupId>com.aliyun.openservices</groupId>
|
||||
<artifactId>ots-public</artifactId>
|
||||
<version>2.2.4</version>
|
||||
<version>2.2.6</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>log4j-api</artifactId>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.aliyun.openservices</groupId>
|
||||
<artifactId>tablestore</artifactId>
|
||||
<version>5.13.10</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
@ -63,6 +59,14 @@
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<build>
|
||||
<resources>
|
||||
<resource>
|
||||
<directory>src/main/java</directory>
|
||||
<includes>
|
||||
<include>**/*.properties</include>
|
||||
</includes>
|
||||
</resource>
|
||||
</resources>
|
||||
<plugins>
|
||||
<!-- compiler plugin -->
|
||||
<plugin>
|
||||
|
@ -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<Configuration> split(int mandatoryNumber);
|
||||
|
||||
|
||||
}
|
@ -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;
|
||||
}
|
@ -25,6 +25,10 @@ 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";
|
||||
|
||||
@ -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";
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -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";
|
||||
|
||||
}
|
@ -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 ...");
|
||||
}
|
||||
}
|
||||
|
@ -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 + "]";
|
||||
}
|
||||
}
|
||||
|
@ -1,56 +1,41 @@
|
||||
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;
|
||||
|
||||
public class OtsWriterMasterProxy {
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
private OTSConf conf = new OTSConf();
|
||||
|
||||
private OTSClient ots = null;
|
||||
|
||||
private TableMeta meta = null;
|
||||
public class OtsWriterMasterProxy implements IOtsWriterMasterProxy {
|
||||
|
||||
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));
|
||||
setStaticParams(param);
|
||||
|
||||
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);
|
||||
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));
|
||||
@ -59,37 +44,68 @@ public class OtsWriterMasterProxy {
|
||||
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 = Common.getOTSInstance(conf);
|
||||
|
||||
ots = new OTSClient(
|
||||
this.conf.getEndpoint(),
|
||||
this.conf.getAccessId(),
|
||||
this.conf.getAccessKey(),
|
||||
this.conf.getInstanceName());
|
||||
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.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());
|
||||
LOG.debug("Table Meta : {}", GsonParser.metaToJson(meta));
|
||||
conf.setPrimaryKeyColumn(WriterModelParser.parseOTSPKColumnList(meta, ParamChecker.checkListAndGet(param, Key.PRIMARY_KEY, true)));
|
||||
}
|
||||
|
||||
public List<Configuration> split(int mandatoryNumber){
|
||||
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()));
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Configuration> split(int mandatoryNumber) {
|
||||
LOG.info("Begin split and MandatoryNumber : {}", mandatoryNumber);
|
||||
List<Configuration> configurations = new ArrayList<Configuration>();
|
||||
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();
|
||||
}
|
||||
@ -100,11 +116,23 @@ public class OtsWriterMasterProxy {
|
||||
|
||||
// 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));
|
||||
|
||||
}
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user