From 83751dde97545438f2e07325c1b8d55a644c5096 Mon Sep 17 00:00:00 2001 From: Zhang Jianming Date: Sat, 22 Jul 2023 09:44:40 +0800 Subject: [PATCH 1/5] add kudu connector --- hetu-docs/en/connector/kudu.md | 350 ++++++++ hetu-docs/zh/connector/kudu.md | 326 +++++++ hetu-server/src/main/provisio/hetu.xml | 6 + pom.xml | 1 + presto-kudu/LICENSE | 201 +++++ presto-kudu/pom.xml | 343 +++++++ .../java/io/presto/kudu/KuduClientConfig.java | 279 ++++++ .../io/presto/kudu/KuduClientSession.java | 71 ++ .../java/io/presto/kudu/KuduColumnHandle.java | 114 +++ .../java/io/presto/kudu/KuduConnector.java | 126 +++ .../io/presto/kudu/KuduConnectorFactory.java | 70 ++ .../java/io/presto/kudu/KuduConnectorId.java | 58 ++ .../presto/kudu/KuduExtendedTableHandle.java | 62 ++ .../io/presto/kudu/KuduHandleResolver.java | 69 ++ .../io/presto/kudu/KuduInsertTableHandle.java | 44 + .../java/io/presto/kudu/KuduMetadata.java | 386 ++++++++ .../main/java/io/presto/kudu/KuduModule.java | 196 ++++ .../io/presto/kudu/KuduOutputTableHandle.java | 67 ++ .../java/io/presto/kudu/KuduPageSink.java | 192 ++++ .../io/presto/kudu/KuduPageSinkProvider.java | 57 ++ .../presto/kudu/KuduPageSourceProvider.java | 52 ++ .../main/java/io/presto/kudu/KuduPlugin.java | 28 + .../java/io/presto/kudu/KuduRecordCursor.java | 192 ++++ .../KuduRecordCursorWithVirtualRowId.java | 69 ++ .../java/io/presto/kudu/KuduRecordSet.java | 97 ++ .../io/presto/kudu/KuduRecordSetProvider.java | 58 ++ .../main/java/io/presto/kudu/KuduSplit.java | 78 ++ .../java/io/presto/kudu/KuduSplitManager.java | 53 ++ .../java/io/presto/kudu/KuduTableHandle.java | 96 ++ .../io/presto/kudu/KuduTableLayoutHandle.java | 98 ++ .../io/presto/kudu/KuduTransactionHandle.java | 21 + .../presto/kudu/KuduUpdatablePageSource.java | 119 +++ .../presto/kudu/NativeKuduClientSession.java | 840 ++++++++++++++++++ .../main/java/io/presto/kudu/RowHelper.java | 106 +++ .../kudu/SchemaAlreadyExistsException.java | 42 + .../main/java/io/presto/kudu/TypeHelper.java | 345 +++++++ .../src/main/java/io/presto/kudu/Types.java | 34 + .../procedures/RangePartitionProcedures.java | 80 ++ .../presto/kudu/properties/ColumnDesign.java | 71 ++ .../properties/HashPartitionDefinition.java | 43 + .../kudu/properties/KuduTableProperties.java | 499 +++++++++++ .../kudu/properties/PartitionDesign.java | 43 + .../kudu/properties/RangeBoundValue.java | 42 + .../RangeBoundValueDeserializer.java | 75 ++ .../properties/RangeBoundValueSerializer.java | 91 ++ .../kudu/properties/RangePartition.java | 41 + .../properties/RangePartitionDefinition.java | 32 + .../kudu/client/KeyEncoderAccessor.java | 47 + .../presto/kudu/KuduQueryRunnerFactory.java | 106 +++ .../src/test/java/io/presto/kudu/TestDec.java | 27 + .../presto/kudu/TestDecimalColumnsTest.java | 99 +++ .../src/test/java/io/presto/kudu/TestInt.java | 27 + .../presto/kudu/TestIntegerColumnsTest.java | 108 +++ .../kudu/TestKuduIntegrationSmokeTest.java | 88 ++ .../java/io/presto/kudu/TestKuduPlugin.java | 34 + .../kudu/TestRangePartitioningTest.java | 170 ++++ .../test/java/io/presto/kudu/TestRanges.java | 59 ++ .../io/presto/kudu/TestSchemaNoExist.java | 74 ++ .../RangePartitionSerializationTest.java | 47 + presto-main/etc/catalog/kudu.properties | 54 ++ 60 files changed, 7303 insertions(+) create mode 100644 hetu-docs/en/connector/kudu.md create mode 100644 hetu-docs/zh/connector/kudu.md create mode 100644 presto-kudu/LICENSE create mode 100644 presto-kudu/pom.xml create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduClientSession.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduColumnHandle.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduConnector.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduConnectorFactory.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduConnectorId.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduExtendedTableHandle.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduHandleResolver.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduInsertTableHandle.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduMetadata.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduModule.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduOutputTableHandle.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduPageSink.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduPageSinkProvider.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduPageSourceProvider.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduPlugin.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursorWithVirtualRowId.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduRecordSet.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduRecordSetProvider.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduSplit.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduSplitManager.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduTableHandle.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduTableLayoutHandle.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduTransactionHandle.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/KuduUpdatablePageSource.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/RowHelper.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/SchemaAlreadyExistsException.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/TypeHelper.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/Types.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/procedures/RangePartitionProcedures.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/ColumnDesign.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/HashPartitionDefinition.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/KuduTableProperties.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/PartitionDesign.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValue.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueDeserializer.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueSerializer.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/RangePartition.java create mode 100644 presto-kudu/src/main/java/io/presto/kudu/properties/RangePartitionDefinition.java create mode 100644 presto-kudu/src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/KuduQueryRunnerFactory.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestDec.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestDecimalColumnsTest.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestInt.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestIntegerColumnsTest.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestKuduIntegrationSmokeTest.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestKuduPlugin.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestRangePartitioningTest.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestRanges.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/TestSchemaNoExist.java create mode 100644 presto-kudu/src/test/java/io/presto/kudu/properties/RangePartitionSerializationTest.java create mode 100644 presto-main/etc/catalog/kudu.properties diff --git a/hetu-docs/en/connector/kudu.md b/hetu-docs/en/connector/kudu.md new file mode 100644 index 000000000..6cd66ca14 --- /dev/null +++ b/hetu-docs/en/connector/kudu.md @@ -0,0 +1,350 @@ +# Presto-Kudu +The [openLooKeng](https://openlookeng.io/en/) Kudu connector allows querying, inserting and deleting data in [Apache Kudu](https://kudu.apache.org/) + + + +## Compatibility + + +kudu 1.15 was tested and used in the production environment,if you need other version of kudu ,you can download code and modify the pom.xml of the module of presto-kudu + +### Deploying Kudu server +Follow installation guide at [Apache Kudu](https://kudu.apache.org/). + +If you want to deploy Kudu 1.8.0 on RHE 7 or CentOS 7, you may also be +interessed in my binary build project [kudu-rpm](https://github.com/MartinWeindel/kudu-rpm/releases/tag/v1.8.0-1). + +### Configuring Apache Kudu connector + +* Create a file name `kudu.properties` in `$OPENLOOKENG_HOME/etc/catalog/`: + ``` + connector.name=kudu + + ## List of Kudu master addresses, at least one is needed (comma separated) + ## Supported formats: example.com, example.com:7051, 192.0.2.1, 192.0.2.1:7051, + ## [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1 + kudu.client.master-addresses=localhost:7051 + + #if kudu with kerberos, enable this + is.kerberos.on = false + + #kerberos user for kudu with kerberos + kerberos.user= root@EXAMPLE.COM + + #keytab path + keytab.path = /root/root.keytab + + #the path of hdfs-site.xml is used for kerberos, only is.kerberos.on = true need it。 + hdfs-site.xml.path = /root/hdfs-site.xml + + #the path of core-site.xml is used for kerberos, only is.kerberos.on = true need it。 + core-site.xml.path = /root/core-site.xml + + #after the service started,the delay time of kerberos re-authentication,this is second time of re-authentication + kudu.reauth.delay.time = 60 + + #after service started,this configuration is re-authentication interval,This configuration is to prevent ticket expiration + kudu.reauth.period.time = 82800 + + ## Optional restriction of tablets for specific tenant. + ## If a tenant is set, only Kudu tablets starting with `.` will + ## be visible in openLookeng + #kudu.session.tenant=root_db + + #If configuration is enabled,you can select database for kudu metadata table + kudu.metadata.on = true + + #for example, if kudu.metadata.on = true ,metadata table in kudu is root_db._schema + kudu.metadata.database = root_db + + ####################### + ### Advanced Kudu Java client configuration + ####################### + + ## Default timeout used for administrative operations (e.g. createTable, deleteTable, etc.) + #kudu.client.defaultAdminOperationTimeout = 30s + + ## Default timeout used for user operations + #kudu.client.defaultOperationTimeout = 30s + + ## Default timeout to use when waiting on data from a socket + #kudu.client.defaultSocketReadTimeout = 10s + + ## Disable Kudu client's collection of statistics. + #kudu.client.disableStatistics = false + ``` +when show schemas,kudu connector will create metadata talbe named '_schema'。if you want to connected kudu cluster with kerberos ,you must configure the path of krb5.conf,such as -Djava.security.krb5.conf=/Users/mac/Desktop/krb5.conf +## Querying Data +A Kudu table named `mytable` is available in openLookeng as table `kudu.default.mytable`. +A Kudu table containing a dot is considered as a schema/table combination, e.g. +`dev.mytable` is mapped to the openLooKeng table `kudu.dev.mytable. +Only Kudu table names in lower case are currently supported. + + +- Now you can use any Kudu table, if it is lower case and contains no dots. +- Alternatively you can create a users table with +```sql +CREATE TABLE users ( + user_id int, + first_name varchar, + last_name varchar +) WITH ( + column_design = '{"user_id": {"key": true}}', + partition_design = '{"hash":[{"columns":["user_id"], "buckets": 2}]}', + num_replicas = 1 +); +``` +On creating a Kudu table you must/can specify addition information about +the primary key, encoding, and compression of columns and hash or range partitioning, +and the number of replicas. Details see below in section "Create Kudu Table". + +- The table can be described using +```sql +DESCRIBE kudu.default.users; +``` +You should get something like +``` + Column | Type | Extra | Comment +------------+---------+-------------------------------------------------------------------+--------- + user_id | integer | key, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | + first_name | varchar | nullable, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | + last_name | varchar | nullable, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | +(3 rows) +``` + +- Insert some data with +```sql +INSERT INTO users VALUES (1, 'Donald', 'Duck'), (2, 'Mickey', 'Mouse'); +``` + +- Select the inserted data +```sql +SELECT * FROM users; +``` + + +## Data Type Mapping +The data types of openLooKeng and Kudu are mapped as far as possible: + +| openLooKeng Data Type | Kudu Data Type | Comment | +| ---------------- | -------------- | ------- | +| `BOOLEAN` | `BOOL` | | +| `TINYINT` | `INT8` | | +| `SMALLINT` | `INT16` | | +| `INTEGER` | `INT32` | | +| `BIGINT` | `INT64` | | +| `REAL` | `FLOAT` | | +| `DOUBLE` | `DOUBLE` | | +| `VARCHAR` | `STRING` || +| `VARBINARY` | `BINARY` | | +| `TIMESTAMP` | `UNIXTIME_MICROS` | µs resolution in Kudu column is reduced to ms resolution | +| `DECIMAL` | `DECIMAL` | only supported for Kudu server >= 1.7.0 | +| `CHAR` | - | not supported, see note 2 | +| `DATE` | - | not supported, see note 2 | +| `TIME` | - | not supported | +| `JSON` | - | not supported | +| `TIME WITH TIMEZONE` | - | not supported | +| `TIMESTAMP WITH TIMEZONE` | - | not supported | +| `INTERVAL YEAR TO MONTH` | - | not supported | +| `INTERVAL DAY TO SECOND` | - | not supported | +| `ARRAY` | - | not supported | +| `MAP` | - | not supported | +| `IPADDRESS` | - | not supported | + +#### Note 1 +On performing `CREATE TABLE ... AS ...` from a openLooKeng table to Kudu, +the optional maximum length is lost + +#### Note 2 +On performing `CREATE TABLE ... AS ...` from a openLooKeng table to Kudu, +a `DATE` or `CHAR` column is converted to `STRING` + +## Supported openLooKeng SQL statements +| openLooKeng SQL statement | Supported | Comment | +| -------------------- | --------- | ------- | +| `SELECT` | [x] | | +| `INSERT INTO ... VALUES` | [x] | behaves like `upsert` | +| `INSERT INTO ... SELECT ... ` | [x] | behaves like `upsert` | +| `DELETE` | [x] | | +| `CREATE SCHEMA` | [x] | | +| `DROP SCHEMA` | [x] | | +| `CREATE TABLE` | [x] | | +| `CREATE TABLE ... AS` | [x] | | +| `DROP TABLE` | [x] | | +| `ALTER TABLE ... RENAME TO ...` | [x] | | +| `ALTER TABLE ... RENAME COLUMN ...` | [x] | if not part of primary key | +| `ALTER TABLE ... ADD COLUMN ...` | [x] | | +| `ALTER TABLE ... DROP COLUMN ...` | [x] | if not part of primary key | +| `SHOW SCHEMAS` | [x] | | +| `SHOW TABLES` | [x] | | +| `SHOW CREATE TABLE` | [x] | | +| `SHOW COLUMNS FROM` | [x] | | +| `DESCRIBE` | [x] | same as `SHOW COLUMNS FROM`| +| `CALL kudu.system.add_range_partition` | [x] | add range partition to an existing table | +| `CALL kudu.system.drop_range_partition` | [x] | drop an existing range partition from a table | + +Currently not supported are `SHOW PARTITIONS FROM ...`, `ALTER SCHEMA ... RENAME` + +## Create Kudu Table with `CREATE TABLE` +On creating a Kudu Table you need to provide following table properties: +- `column_design` +- `partition_design` (optional) +- `num_replicas` (optional, defaults to 3) + +Example: +```sql +CREATE TABLE users ( + user_id int, + first_name varchar, + last_name varchar +) WITH ( + column_design = '{"user_id": {"key": true}}', + partition_design = '{"hash":[{"columns":["user_id"], "buckets": 2}]}', + num_replicas = 1 +); +``` + +### Table property `column_design` +With the column design table property you define the columns for the primary key. +Additionally you can overwrite the encoding and compression of every single column. + +The value of this property must be a string of a valid JSON object. +The keys are the columns and the values is a JSON object with the columns properties +to set, i.e. + +``` +'{"": {"": , ...}, ...}'` +``` + +| Column property name | Value | Comment | +| -------------------- | ----- | ------- | +| `key` | `true` or `false` | if column belongs to primary key, default: `false` | +| `nullable` | `true` or `false` | if column is nullable, default: `true` for non-key columns, key columns must not be nullable | +| `encoding` | "string value" | See Apache Kudu documentation: [Column encoding](https://kudu.apache.org/docs/schema_design.html#encoding) | +| `compression` | "string value" | See Apache Kudu documentation: [Column compression](https://kudu.apache.org/docs/schema_design.html#compression) | + +Example: +``` +'{"column1": {"key": true, "encoding": "dictionary", "compression": "LZ4"}, "column2": {...}}' +``` + +### Table property `partition_design` +With the partition design table property you define the partition layout. +In Apache Kudu you can define multiple hash partitions and at most one range partition. +Details see Apache Kudu documentation: [Partitioning](https://kudu.apache.org/docs/schema_design.html#partitioning) + +The value of this property must be a string of a valid JSON object. +The keys are either `hash` or `range` or both, i.e. + +``` +'{"hash": [{...},...], "range": {...}}'` +``` +#### Hash partitioning +You can provide multiple hash partition groups in Apache Kudu. +Each group consists of a list of column names and the number of buckets. + +Example: +``` +'{"hash": [{"columns": ["region", "name"], "buckets": 5}]}' +``` +This defines a hash partition with the columns "region" and "name", +distributed over 5 buckets. All partition columns must be part of +the primary key. + +#### Range partitioning +You can provide at most one range partition in Apache Kudu. +It consists of a list of columns. The ranges themselves are given either +in the table property `range_partitions`. Alternatively, the +procedures `kudu.system.add_range_partition` and `kudu.system.drop_range_partition` +can be used to manage range partitions for existing tables. +For both ways see below for more details. + +Example: +``` +'{"range": {"columns": ["event_time"]}}' +``` + +Defines range partitioning on the column "event". + +To add concrete range partitions use either the table property `range_partitions` +or call the procedure . + +### Table property `range_partitions` +With the `range_partitions` table property you specify the concrete range partitions to be +created. The range partition definition itself must be given in the table +property `partition_design` separately. + + + +Example: +```sql +CREATE TABLE events ( + serialno varchar, + event_time timestamp, + message varchar +) WITH ( + column_design = '{"serialno": {"key": true}, "event_time": {"key": true}}', + partition_design = '{"hash":[{"columns":["serialno"], "buckets": 4}], + "range": {"columns":["event_time"]}}', + range_partitions = '[{"lower": null, "upper": "2017-01-01T00:00:00"}, + {"lower": "2017-01-01T00:00:00", "upper": "2017-07-01T00:00:00"}, + {"lower": "2017-07-01T00:00:00", "upper": "2018-01-01T00:00:00"}]', + num_replicas = 1 +); +``` +This creates a table with a hash partition on column `serialno` with 4 buckets and range partitioning +on column `event_time`. Additionally three range partitions are created: + 1. for all event_times before the year 2017 (lower bound = `null` means it is unbound) + 2. for the first half of the year 2017 + 3. for the second half the year 2017 +This means any try to add rows with `event_time` of year 2018 or greater will fail, +as no partition is defined. + +#### Managing range partitions +For existing tables, there are procedures to add and drop a range partition. + +- adding a range partition +```sql +CALL kudu.system.add_range_partition(, , ), +``` + +- dropping a range partition +```sql +CALL kudu.system.drop_range_partition(,
, ) +``` + +- ``: schema of the table +- `
`: table names +- ``: lower and upper bound of the range partition + as json string in the form `'{"lower": , "upper": }'`, or if the range + partition has multiple columns: `'{"lower": [,...], "upper": [,...]}'`. + The concrete literal for lower and upper bound values are depending on the + column types. + + Examples: + +| openLooKeng Data Type | JSON string example | +| ---------------- | ------------------- | +| BIGINT | '{"lower": 0, "upper": 1000000}' | +| SMALLINT | '{"lower": 10, "upper": null}' | +| VARCHAR | '{"lower": "A", "upper": "M"}' | +| TIMESTAMP | '{"lower": "2018-02-01T00:00:00.000", "upper": "2018-02-01T12:00:00.000"}' | +| BOOLEAN | '{"lower": false, "upper": true}' | +| VARBINARY | values encoded as base64 strings | + + To specified an unbounded bound, use the value `null`. + + +Example: +```sql +CALL kudu.system.add_range_partition('myschema', 'events', '{"lower": "2018-01-01", "upper": "2018-06-01"}') +``` +This would add a range partition for a table `events` in the schema `myschema` with +the lower bound `2018-01-01` (more exactly `2018-01-01T00:00:00.000`) and the upper bound `2018-07-01`. + +Use the sql statement `SHOW CREATE TABLE` to request the existing range partitions (they are shown +in the table property `range_partitions`). + +## Known limitations +- Only lower case table and column names in Kudu are supported +- As schemas are not directly supported by Kudu, a special table named `_schemas` \ No newline at end of file diff --git a/hetu-docs/zh/connector/kudu.md b/hetu-docs/zh/connector/kudu.md new file mode 100644 index 000000000..d6536eeec --- /dev/null +++ b/hetu-docs/zh/connector/kudu.md @@ -0,0 +1,326 @@ +# Presto-Kudu +[openLooKeng](https://openlookeng.io/en/) Kudu connector 允许用户向[Apache Kudu](https://kudu.apache.org/)查询、删除和插入数据。 + + + +## 兼容性 + + +kudu 1.15版本已经已经测试并在生产环境使用,如果您需要使用其他版本的kudu,可以下载源码,并修改presto-kudu模块中的pom.xml,修改kudu依赖版本,并重新编译。 + +### 部署 Kudu server +通过如下向导安装kudu [Apache Kudu](https://kudu.apache.org/)。 + +### Kudu connector的配置 + +* 在如下目录 `$OPENLOOKENG_HOME/etc/catalog/` 创建`kudu.properties`: + ``` + #连接器名称 + connector.name=kudu + + ## kudu master地址,至少一个,多个可用逗号分开 + ## 支持的格式: example.com, example.com:7051, 192.0.2.1, 192.0.2.1:7051, + ## [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1 + kudu.client.master-addresses=localhost:7051 + + #如果kudu集成了kerberos认证,则开启如下配置 + is.kerberos.on = false + + #kerberos认证主体 + kerberos.user= root@EXAMPLE.COM + + #keytab的路径 + keytab.path = /root/root.keytab + + #使用kerberos认证时,需要配置hdfs-site.xml的路径 + hdfs-site.xml.path = /root/hdfs-site.xml + + #使用kerberos认证时,需要配置core-site.xml的路径 + core-site.xml.path = /root/core-site.xml + + #服务启动后会自动认证,该配置为服务启动后从启动开始到第二次认证的时间间隔 + kudu.reauth.delay.time = 60 + + #第二次认证后的时间间隔,保证票据刷新 + kudu.reauth.period.time = 82800 + + ## 可选项,对每个租户的表的限制,如果被设置后,以其开头的kudu表才可见,若配置为“root_db”,则表名称以“root_db.”开头的表才可见。 + #kudu.session.tenant=root_db + + #如果配置开启,则可以为元数据表选择数据库去存放 + kudu.metadata.on = true + + #例如, 如果开启kudu.metadata.on,在kudu端看到的元数据表为 root_db._schema,这两种配置目的是为了将“_schemas"表指定到特定数据库,以防止租户保存元数据时无权限 + kudu.metadata.database = root_db + + ####################### + ### Advanced Kudu Java client configuration + ####################### + + ## kudu管理功能操作的超时时间 + #kudu.client.defaultAdminOperationTimeout = 30s + + ## kudu用户操作的超时时间 + #kudu.client.defaultOperationTimeout = 30s + + ## 等待socket的超时时间 + #kudu.client.defaultSocketReadTimeout = 10s + + ## 是否开启kudu客户端的statistics. + #kudu.client.disableStatistics = false + ``` +当执行“show schemas”命令时,kudu连接器将会创建名称为 '_schema'的元数据表。此外,使用kerberos认证的kudu集群,请在jvm.config里配置krb5.conf的路径,例如:-Djava.security.krb5.conf=/Users/mac/Desktop/krb5.conf +## Querying Data +创建表的基本语如下所示,一个名称为"mytable"的表,实际在Openlookeng中为kudu.default.mytable,一个带有"."的前缀,被认为是schema名称,例如dev.mytable表的实际映射为kudu.dev.mytable,仅小写的kudu表名称被支持。 + + +- 现在可以创建kudu表,表名称为小写且不含“.”。 +- 例如你可以创建一个用户表。 +```sql +CREATE TABLE users ( + user_id int, + first_name varchar, + last_name varchar +) WITH ( + column_design = '{"user_id": {"key": true}}', + partition_design = '{"hash":[{"columns":["user_id"], "buckets": 2}]}', + num_replicas = 1 +); +``` +创建kudu表时需要指定一些额外的信息,包括 +primary key, encoding, 和compression of columns以及hash or range partitioning, +此外还包括副本数量. 详细的参数可以到kudu建表部分去查看。 + +- 表的describe命令如下: +```sql +DESCRIBE kudu.default.users; +``` +得到的结果如下: +``` + Column | Type | Extra | Comment +------------+---------+-------------------------------------------------------------------+--------- + user_id | integer | key, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | + first_name | varchar | nullable, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | + last_name | varchar | nullable, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | +(3 rows) +``` + +- 插入数据如下: +```sql +INSERT INTO users VALUES (1, 'Donald', 'Duck'), (2, 'Mickey', 'Mouse'); +``` + +- Select数据如下: +```sql +SELECT * FROM users; +``` + + +## 数据类型映射 +数据类型映射如下: + +| openLooKeng Data Type | Kudu Data Type | Comment | +| ---------------- | -------------- | ------- | +| `BOOLEAN` | `BOOL` | | +| `TINYINT` | `INT8` | | +| `SMALLINT` | `INT16` | | +| `INTEGER` | `INT32` | | +| `BIGINT` | `INT64` | | +| `REAL` | `FLOAT` | | +| `DOUBLE` | `DOUBLE` | | +| `VARCHAR` | `STRING` || +| `VARBINARY` | `BINARY` | | +| `TIMESTAMP` | `UNIXTIME_MICROS` | µs resolution in Kudu column is reduced to ms resolution | +| `DECIMAL` | `DECIMAL` | kudu连接器 >= 1.7.0 | +| `CHAR` | - | not supported| +| `DATE` | - | not supported | +| `TIME` | - | not supported | +| `JSON` | - | not supported | +| `TIME WITH TIMEZONE` | - | not supported | +| `TIMESTAMP WITH TIMEZONE` | - | not supported | +| `INTERVAL YEAR TO MONTH` | - | not supported | +| `INTERVAL DAY TO SECOND` | - | not supported | +| `ARRAY` | - | not supported | +| `MAP` | - | not supported | +| `IPADDRESS` | - | not supported | + +#### Note 1 +执行 `CREATE TABLE ... AS ...` 命令时 +可选的最大长度会丢失 + +#### Note 2 +执行 `CREATE TABLE ... AS ...` 命令时 +`DATE` 或 `CHAR` 转换为 `STRING` + +## Supported openLooKeng SQL statements +| openLooKeng SQL statement | Supported | Comment | +| -------------------- | --------- | ------- | +| `SELECT` | [x] | | +| `INSERT INTO ... VALUES` | [x] | behaves like `upsert` | +| `INSERT INTO ... SELECT ... ` | [x] | behaves like `upsert` | +| `DELETE` | [x] | | +| `CREATE SCHEMA` | [x] | | +| `DROP SCHEMA` | [x] | | +| `CREATE TABLE` | [x] | | +| `CREATE TABLE ... AS` | [x] | | +| `DROP TABLE` | [x] | | +| `ALTER TABLE ... RENAME TO ...` | [x] | | +| `ALTER TABLE ... RENAME COLUMN ...` | [x] | if not part of primary key | +| `ALTER TABLE ... ADD COLUMN ...` | [x] | | +| `ALTER TABLE ... DROP COLUMN ...` | [x] | if not part of primary key | +| `SHOW SCHEMAS` | [x] | | +| `SHOW TABLES` | [x] | | +| `SHOW CREATE TABLE` | [x] | | +| `SHOW COLUMNS FROM` | [x] | | +| `DESCRIBE` | [x] | same as `SHOW COLUMNS FROM`| +| `CALL kudu.system.add_range_partition` | [x] | add range partition to an existing table | +| `CALL kudu.system.drop_range_partition` | [x] | drop an existing range partition from a table | + +目前不支持 `SHOW PARTITIONS FROM ...`, `ALTER SCHEMA ... RENAME` + +## 通过 `CREATE TABLE` 创建表 +创建kudu表需要提供以下属性: +- `column_design` +- `partition_design` (optional) +- `num_replicas` (optional, defaults to 3) + +示例: +```sql +CREATE TABLE users ( + user_id int, + first_name varchar, + last_name varchar +) WITH ( + column_design = '{"user_id": {"key": true}}', + partition_design = '{"hash":[{"columns":["user_id"], "buckets": 2}]}', + num_replicas = 1 +); +``` + +### 表属性 `column_design` +可以为主键定义column design。 +此外,可以为每一列设置encoding和compression, +值必须为String类型的json对象。 +key为列名称,value为属性,例如: + +``` +'{"": {"": , ...}, ...}'` +``` + +| Column property name | Value | Comment | +| -------------------- | ----- | ------- | +| `key` | `true` or `false` | 主键配置, default: `false` | +| `nullable` | `true` or `false` | 是否为 nullable, 对于非主键列:default: `true`, 主键列必须非null| +| `encoding` | "string value" | Kudu文档: [Column encoding](https://kudu.apache.org/docs/schema_design.html#encoding) | +| `compression` | "string value" | Kudu文档: [Column compression](https://kudu.apache.org/docs/schema_design.html#compression) | + +Example: +``` +'{"column1": {"key": true, "encoding": "dictionary", "compression": "LZ4"}, "column2": {...}}' +``` + +### 表属性 `partition_design` +通过partition design能够定义partition layout。 +在kudu中可以定义多个hash partitions和最多一个range partition. +详细请看kudu官网: [Partitioning](https://kudu.apache.org/docs/schema_design.html#partitioning) + +值必须为String类型的json对象, +keys必须为 `hash`或`range`全部, 例如: + +``` +'{"hash": [{...},...], "range": {...}}'` +``` +#### Hash partitioning +你可以在kudu中提供多个hash partition组 +每一个组包括一组列和他们的bucket数量 + +例如: +``` +'{"hash": [{"columns": ["region", "name"], "buckets": 5}]}' +``` + +上述例子定义了由 "region"和"name"组成的hash partition,并且包含5个buckets.所有的分区列必须是主键的一部分 + +#### Range partitioning +你可以在kudu中定义最多一个range partition,它包括一组列。ranges本身由属性`range_partitions`定义,同样的,通过程序调用,由`kudu.system.add_range_partition` 和 `kudu.system.drop_range_partition`同样可以对已存在的表进行range partitions的管理。 +两种方式更详细的用法如下所示: + +Example: +``` +'{"range": {"columns": ["event_time"]}}' +``` + +对"event_time"列定义range partitions。 + +要添加具体的range partitions,可以使用表属性 `range_partitions` 或者调用程序。 + +### 表属性 `range_partitions` +通过表属性 `range_partitions` table property,你可以创建具体的range partition。 range partition 定义必须与 `partition_design` 分开定义。 + + + +例如: +```sql +CREATE TABLE events ( + serialno varchar, + event_time timestamp, + message varchar +) WITH ( + column_design = '{"serialno": {"key": true}, "event_time": {"key": true}}', + partition_design = '{"hash":[{"columns":["serialno"], "buckets": 4}], + "range": {"columns":["event_time"]}}', + range_partitions = '[{"lower": null, "upper": "2017-01-01T00:00:00"}, + {"lower": "2017-01-01T00:00:00", "upper": "2017-07-01T00:00:00"}, + {"lower": "2017-07-01T00:00:00", "upper": "2018-01-01T00:00:00"}]', + num_replicas = 1 +); +``` +上述示例创建了一个表,包括由`serialno` 列和4个buckets组成的hash partition,和由`event_time`列组成的range partition,此外创建了三个range partition: + 1. 所有的event_times在2017年之前 (lower bound = `null` means 表示上界无边界)。 + 2. 2017年上半年。 + 3. 2017年下半年。 +这意味着任何试图添加`event_time`大于等于2018年的数据将会失败。因为没有此分区被定义。 + +#### 管理 range partitions +对于已经存在的表, 可以通过程序去添加和删除range partition。 + +- 添加一个range partition +```sql +CALL kudu.system.add_range_partition(,
, ), +``` + +- 删除一个range partition +```sql +CALL kudu.system.drop_range_partition(,
, ) +``` + +- ``: 表的schema +- `
`: 表名称 +- ``: range partition的上界和下界,以json的格式如下所示: `'{"lower": , "upper": }'`,或者range partition 包含多列可如下定义: `'{"lower": [,...], "upper": [,...]}'`。 + 下界值和上界值的具体写法和规范取决于列类型。 + + 例如: + +| openLooKeng Data Type | JSON string example | +| ---------------- | ------------------- | +| BIGINT | '{"lower": 0, "upper": 1000000}' | +| SMALLINT | '{"lower": 10, "upper": null}' | +| VARCHAR | '{"lower": "A", "upper": "M"}' | +| TIMESTAMP | '{"lower": "2018-02-01T00:00:00.000", "upper": "2018-02-01T12:00:00.000"}' | +| BOOLEAN | '{"lower": false, "upper": true}' | +| VARBINARY | values encoded as base64 strings | + + 对于一个具体的无边界类型, 使用 `null`。 + + +例如: +```sql +CALL kudu.system.add_range_partition('myschema', 'events', '{"lower": "2018-01-01", "upper": "2018-06-01"}') +``` +上述操作将会为schema为`myschema`和名称为`events`的表创建range partition,其上界为`2018-07-01`,下界为`2018-01-01`(精确值为`2018-01-01T00:00:00.000`)。 + +用语句 `SHOW CREATE TABLE` 可以获取已经存在的range partitions (他们展示在表属性 `range_partitions`中)。 + +## 已知的局限性 +- kudu连接器仅支持小写的表和列名称 +- kudu并不能直接支持schema, 通过一个特殊的表 `_schemas`来存储元数据 \ No newline at end of file diff --git a/hetu-server/src/main/provisio/hetu.xml b/hetu-server/src/main/provisio/hetu.xml index 41899c4f6..46e849eb7 100644 --- a/hetu-server/src/main/provisio/hetu.xml +++ b/hetu-server/src/main/provisio/hetu.xml @@ -74,6 +74,12 @@ + + + + + + diff --git a/pom.xml b/pom.xml index 84168c5d5..eed69f0b6 100644 --- a/pom.xml +++ b/pom.xml @@ -162,6 +162,7 @@ hetu-singledata hetu-iceberg hetu-dm + presto-kudu diff --git a/presto-kudu/LICENSE b/presto-kudu/LICENSE new file mode 100644 index 000000000..261eeb9e9 --- /dev/null +++ b/presto-kudu/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/presto-kudu/pom.xml b/presto-kudu/pom.xml new file mode 100644 index 000000000..e6a4231d3 --- /dev/null +++ b/presto-kudu/pom.xml @@ -0,0 +1,343 @@ + + + + + presto-root + io.hetu.core + 1.11.0-SNAPSHOT + + 4.0.0 + + io.hetu.core + presto-kudu + presto-kudu + 1.11.0-SNAPSHOT + hetu-plugin + + + 0.208 + 1.7.0 + 6.10 + 0.36 + 4.2.0 + 24.1-jre + ${project.parent.basedir} + + + + + org.apache.kudu + kudu-client + 1.15.0 + + + + io.airlift + bootstrap + + + org.slf4j + log4j-over-slf4j + + + + + + io.airlift + json + + + + io.airlift + log + + + io.airlift + configuration + 0.193 + compile + + + javax.inject + javax.inject + 1 + compile + + + io.hetu.core + presto-tpch + 1.11.0-SNAPSHOT + test + + + com.fasterxml.jackson.core + jackson-core + 2.11.4 + compile + + + io.airlift.tpch + tpch + 0.10 + test + + + joda-time + joda-time + 2.10.5 + + + javax.validation + validation-api + 2.0.1.Final + compile + + + com.fasterxml.jackson.core + jackson-databind + 2.11.4 + compile + + + com.google.guava + guava + + + + com.google.inject + guice + 4.2.2 + + + + + io.hetu.core + presto-spi + provided + + + + io.airlift + slice + 0.39 + + + + io.airlift + units + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + com.fasterxml.jackson.core + jackson-core + 2.14.0 + + + + + org.testng + testng + test + + + + io.airlift + testing + test + + + + io.hetu.core + presto-main + test + + + io.prestosql.hadoop + hadoop-apache + + + + + io.prestosql.hadoop + hadoop-apache + 3.2.0-8 + + + io.hetu.core + presto-tests + test + + + + + + + + maven-compiler-plugin + + 1.8 + 1.8 + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.20.1 + + + + **/Test*.java + **/*Test.java + **/*TestCase.java + **/*Suite.java + + ${project.build.directory}/surefire-reports + -Xmx3g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + + true + + false + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + false + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 3.0.0-M1 + + true + + + + + pl.project13.maven + git-commit-id-plugin + + true + + + + io.hetu + presto-maven-plugin + 9 + true + + true + io.prestosql.spi.Plugin + + + + com.ning.maven.plugins + maven-duplicate-finder-plugin + 1.0.8 + + false + + + + + org.apache.maven.plugins + maven-dependency-plugin + + false + + + + com.ning.maven.plugins + maven-dependency-versions-check-plugin + 2.0.2 + + true + false + + + + + + \ No newline at end of file diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java b/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java new file mode 100644 index 000000000..3613a9987 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java @@ -0,0 +1,279 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; +import com.google.common.base.Splitter; +import io.airlift.configuration.Config; +import io.airlift.units.Duration; +import io.airlift.units.MaxDuration; +import io.airlift.units.MinDuration; +import io.prestosql.spi.function.Mandatory; + +import javax.validation.constraints.NotNull; +import javax.validation.constraints.Size; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * Configuration read from etc/catalog/kudu.properties + */ +public class KuduClientConfig +{ + private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); + private List masterAddresses; + private Duration defaultAdminOperationTimeout = new Duration(30, TimeUnit.SECONDS); + private Duration defaultOperationTimeout = new Duration(30, TimeUnit.SECONDS); + private Duration defaultSocketReadTimeout = new Duration(10, TimeUnit.SECONDS); + private boolean disableStatistics; + private String tenant; + private String hdfsXmlPath; + private String coreXmlPath; + private String isKerberosOn; + private String keytabPath; + private String kerberosUser; + private String kuduMetadataOn; + private String kuduMetadataDatabase; + private String reauthDelayTime; + private String reauthPeriodTime; + + public String getReauthDelayTime() + { + return reauthDelayTime; + } + + @Mandatory(name = "kudu.reauth.delay.time", + description = "kudu.reauth.delay.time", + defaultValue = "60", + required = false) + @Config("kudu.reauth.delay.time") + public KuduClientConfig setReauthDelayTime(String reauthDelayTime) + { + this.reauthDelayTime = reauthDelayTime; + return this; + } + + public String getReauthPeriodTime() + { + return reauthPeriodTime; + } + + @Mandatory(name = "kudu.reauth.period.time", + description = "kudu.reauth.period.time", + defaultValue = "82800", + required = false) + @Config("kudu.reauth.period.time") + public KuduClientConfig setReauthPeriodTime(String reauthPeriodTime) + { + this.reauthPeriodTime = reauthPeriodTime; + return this; + } + + public String getKuduMetadataOn() + { + return kuduMetadataOn; + } + + @Mandatory(name = "kudu.metadata.on", + description = "kudu.metadata.on", + defaultValue = "false", + required = false) + @Config("kudu.metadata.on") + public KuduClientConfig setKuduMetadataOn(String kuduMetadataOn) + { + this.kuduMetadataOn = kuduMetadataOn; + return this; + } + + public String getKuduMetadataDatabase() + { + return kuduMetadataDatabase; + } + + @Mandatory(name = "kudu.metadata.database", + description = "kudu.metadata.database", + defaultValue = "", + required = false) + @Config("kudu.metadata.database") + public KuduClientConfig setKuduMetadataDatabase(String kuduMetadataDatabase) + { + this.kuduMetadataDatabase = kuduMetadataDatabase; + return this; + } + + @NotNull + @Size(min = 1) + public List getMasterAddresses() + { + return masterAddresses; + } + + @Config("kudu.client.master-addresses") + public KuduClientConfig setMasterAddresses(String commaSeparatedList) + { + this.masterAddresses = SPLITTER.splitToList(commaSeparatedList); + return this; + } + + public KuduClientConfig setMasterAddresses(String... contactPoints) + { + this.masterAddresses = Arrays.asList(contactPoints); + return this; + } + + @Config("kudu.client.defaultAdminOperationTimeout") + public KuduClientConfig setDefaultAdminOperationTimeout(Duration timeout) + { + this.defaultAdminOperationTimeout = timeout; + return this; + } + + @MinDuration("1s") + @MaxDuration("1h") + public Duration getDefaultAdminOperationTimeout() + { + return defaultAdminOperationTimeout; + } + + @Config("kudu.client.defaultOperationTimeout") + public KuduClientConfig setDefaultOperationTimeout(Duration timeout) + { + this.defaultOperationTimeout = timeout; + return this; + } + + @MinDuration("1s") + @MaxDuration("1h") + public Duration getDefaultOperationTimeout() + { + return defaultOperationTimeout; + } + + @Config("kudu.client.defaultSocketReadTimeout") + public KuduClientConfig setDefaultSocketReadTimeout(Duration timeout) + { + this.defaultSocketReadTimeout = timeout; + return this; + } + + @MinDuration("1s") + @MaxDuration("1h") + public Duration getDefaultSocketReadTimeout() + { + return defaultSocketReadTimeout; + } + + public boolean isDisableStatistics() + { + return this.disableStatistics; + } + + @Config("kudu.client.disableStatistics") + public KuduClientConfig setDisableStatistics(boolean disableStatistics) + { + this.disableStatistics = disableStatistics; + return this; + } + + public String getTenant() + { + return tenant; + } + + @Config("kudu.session.tenant") + public KuduClientConfig setTenant(String tenant) + { + this.tenant = tenant; + return this; + } + + public String getHdfsXmlPath() + { + return hdfsXmlPath; + } + + @Mandatory(name = "hdfs-site.xml.path", + description = "hdfs-site.xml.path", + defaultValue = "", + required = false) + @Config("hdfs-site.xml.path") + public KuduClientConfig setHdfsXmlPath(String hdfsXmlPath) + { + this.hdfsXmlPath = hdfsXmlPath; + return this; + } + + public String getCoreXmlPath() + { + return coreXmlPath; + } + + @Mandatory(name = "core-site.xml.path", + description = "core-site.xml.path", + defaultValue = "", + required = false) + @Config("core-site.xml.path") + public KuduClientConfig setCoreXmlPath(String coreXmlPath) + { + this.coreXmlPath = coreXmlPath; + return this; + } + + public String getIsKerberosOn() + { + return isKerberosOn; + } + + @Mandatory(name = "is.kerberos.on", + description = "is.kerberos.on", + defaultValue = "false", + required = false) + @Config("is.kerberos.on") + public KuduClientConfig setIsKerberosOn(String isKerberosOn) + { + this.isKerberosOn = isKerberosOn; + return this; + } + + public String getKeytabPath() + { + return keytabPath; + } + + @Mandatory(name = "keytab.path", + description = "keytab.path", + defaultValue = "", + required = false) + @Config("keytab.path") + public void setKeytabPath(String keytabPath) + { + this.keytabPath = keytabPath; + } + + public String getKerberosUser() + { + return kerberosUser; + } + + @Mandatory(name = "kerberos.user", + description = "kerberos.user", + defaultValue = "", + required = false) + @Config("kerberos.user") + public void setKerberosUser(String kerberosUser) + { + this.kerberosUser = kerberosUser; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduClientSession.java b/presto-kudu/src/main/java/io/presto/kudu/KuduClientSession.java new file mode 100644 index 000000000..f451b21c3 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduClientSession.java @@ -0,0 +1,71 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.presto.kudu.properties.RangePartition; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorTableMetadata; +import io.prestosql.spi.connector.SchemaTableName; +import org.apache.kudu.Schema; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.KuduSession; +import org.apache.kudu.client.KuduTable; + +import java.util.List; +import java.util.Map; + +public interface KuduClientSession +{ + List listSchemaNames(ConnectorSession session); + + List listTables(String schemaNameOrNull); + + Schema getTableSchema(KuduTableHandle tableName, ConnectorSession session); + + boolean tableExists(SchemaTableName schemaTableName); + + Map getTableProperties(KuduTableHandle tableName, ConnectorSession session); + + List buildKuduSplits(KuduTableLayoutHandle layoutHandle, ConnectorSession session); + + KuduScanner createScanner(KuduSplit kuduSplit); + + KuduTable openTable(SchemaTableName schemaTableName, ConnectorSession session); + + KuduSession newSession(); + + void createSchema(String schemaName, ConnectorSession session); + + boolean schemaExists(String schemaName, ConnectorSession session); + + void dropSchema(String schemaName, ConnectorSession session); + + void dropTable(SchemaTableName schemaTableName); + + void renameTable(SchemaTableName schemaTableName, SchemaTableName newSchemaTableName); + + KuduTable createTable(ConnectorTableMetadata tableMetadata, boolean ignoreExisting, ConnectorSession session); + + void addColumn(SchemaTableName schemaTableName, ColumnMetadata column); + + void dropColumn(SchemaTableName schemaTableName, String name); + + void renameColumn(SchemaTableName schemaTableName, String oldName, String newName); + + void addRangePartition(SchemaTableName schemaTableName, RangePartition rangePartition); + + void dropRangePartition(SchemaTableName schemaTableName, RangePartition rangePartition); +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduColumnHandle.java b/presto-kudu/src/main/java/io/presto/kudu/KuduColumnHandle.java new file mode 100644 index 000000000..1cb97bbef --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduColumnHandle.java @@ -0,0 +1,114 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class KuduColumnHandle + implements ColumnHandle +{ + public static final String ROW_ID = "row_uuid"; + public static final KuduColumnHandle ROW_ID_HANDLE = new KuduColumnHandle(ROW_ID, -1, VarbinaryType.VARBINARY); + private final String name; + private final int ordinalPosition; + private final Type type; + + @JsonCreator + public KuduColumnHandle( + @JsonProperty("name") String name, + @JsonProperty("ordinalPosition") int ordinalPosition, + @JsonProperty("type") Type type) + { + this.name = requireNonNull(name, "name is null"); + this.ordinalPosition = ordinalPosition; + this.type = requireNonNull(type, "type is null"); + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public int getOrdinalPosition() + { + return ordinalPosition; + } + + @Override + public String getColumnName() + { + return name; + } + + @JsonProperty + public Type getType() + { + return type; + } + + public ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(name, type); + } + + public boolean isVirtualRowId() + { + return name.equals(ROW_ID); + } + + @Override + public int hashCode() + { + return Objects.hash( + name, + ordinalPosition, + type); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + KuduColumnHandle other = (KuduColumnHandle) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.ordinalPosition, other.ordinalPosition) && + Objects.equals(this.type, other.type); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("ordinalPosition", ordinalPosition) + .add("type", type).toString(); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduConnector.java b/presto-kudu/src/main/java/io/presto/kudu/KuduConnector.java new file mode 100644 index 000000000..dbe4353e6 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduConnector.java @@ -0,0 +1,126 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.presto.kudu; + +import com.google.common.collect.ImmutableSet; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.log.Logger; +import io.presto.kudu.properties.KuduTableProperties; +import io.prestosql.spi.connector.Connector; +import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorPageSinkProvider; +import io.prestosql.spi.connector.ConnectorPageSourceProvider; +import io.prestosql.spi.connector.ConnectorRecordSetProvider; +import io.prestosql.spi.connector.ConnectorSplitManager; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.procedure.Procedure; +import io.prestosql.spi.session.PropertyMetadata; +import io.prestosql.spi.transaction.IsolationLevel; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Set; + +import static io.prestosql.spi.transaction.IsolationLevel.READ_COMMITTED; +import static io.prestosql.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +public class KuduConnector + implements Connector +{ + private static final Logger log = Logger.get(KuduConnector.class); + private final LifeCycleManager lifeCycleManager; + private final KuduMetadata metadata; + private final ConnectorSplitManager splitManager; + private final ConnectorRecordSetProvider recordSetProvider; + private final ConnectorPageSourceProvider pageSourceProvider; + private final KuduTableProperties tableProperties; + private final ConnectorPageSinkProvider pageSinkProvider; + private final Set procedures; + + @Inject + public KuduConnector(LifeCycleManager lifeCycleManager, KuduMetadata metadata, + ConnectorSplitManager splitManager, ConnectorRecordSetProvider recordSetProvider, + KuduTableProperties tableProperties, + ConnectorPageSourceProvider pageSourceProvider, + ConnectorPageSinkProvider pageSinkProvider, + Set procedures) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); + this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); + this.tableProperties = requireNonNull(tableProperties, "tableProperties is null"); + this.pageSinkProvider = requireNonNull(pageSinkProvider, "pageSinkProvider is null"); + this.procedures = ImmutableSet.copyOf(requireNonNull(procedures, "procedures is null")); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, + boolean readOnly) + { + checkConnectorSupports(READ_COMMITTED, isolationLevel); + return KuduTransactionHandle.INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) + { + return metadata; + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return pageSourceProvider; + } + + @Override + public ConnectorPageSinkProvider getPageSinkProvider() + { + return pageSinkProvider; + } + + @Override + public List> getTableProperties() + { + return tableProperties.getTableProperties(); + } + + @Override + public Set getProcedures() + { + return procedures; + } + + @Override + public final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduConnectorFactory.java b/presto-kudu/src/main/java/io/presto/kudu/KuduConnectorFactory.java new file mode 100644 index 000000000..5b4fb0528 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduConnectorFactory.java @@ -0,0 +1,70 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonModule; +import io.prestosql.spi.connector.Connector; +import io.prestosql.spi.connector.ConnectorContext; +import io.prestosql.spi.connector.ConnectorFactory; +import io.prestosql.spi.connector.ConnectorHandleResolver; + +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public class KuduConnectorFactory + implements ConnectorFactory +{ + private final String name; + + public KuduConnectorFactory(String connectorName) + { + this.name = connectorName; + } + + @Override + public String getName() + { + return name; + } + + @Override + public ConnectorHandleResolver getHandleResolver() + { + return new KuduHandleResolver(); + } + + @Override + public Connector create(String connectorId, Map config, + ConnectorContext context) + { + requireNonNull(config, "config is null"); + + try { + Bootstrap app = new Bootstrap(new JsonModule(), + new KuduModule(connectorId, context.getTypeManager())); + + Injector injector = + app.strictConfig().doNotInitializeLogging().setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(KuduConnector.class); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduConnectorId.java b/presto-kudu/src/main/java/io/presto/kudu/KuduConnectorId.java new file mode 100644 index 000000000..5d063c67a --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduConnectorId.java @@ -0,0 +1,58 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.presto.kudu; + +import com.google.inject.Inject; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class KuduConnectorId +{ + private final String id; + + @Inject + public KuduConnectorId(String id) + { + this.id = requireNonNull(id, "id is null"); + } + + @Override + public String toString() + { + return id; + } + + @Override + public int hashCode() + { + return Objects.hash(id); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + return Objects.equals(this.id, ((KuduConnectorId) obj).id); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduExtendedTableHandle.java b/presto-kudu/src/main/java/io/presto/kudu/KuduExtendedTableHandle.java new file mode 100644 index 000000000..5689f13c1 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduExtendedTableHandle.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.type.Type; +import org.apache.kudu.client.KuduTable; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public abstract class KuduExtendedTableHandle + extends KuduTableHandle +{ + private final List columnNames; + private final List columnTypes; + + public KuduExtendedTableHandle(String connectorId, SchemaTableName schemaTableName, + List columnNames, List columnTypes, + KuduTable table) + { + super(connectorId, schemaTableName, table); + + requireNonNull(columnNames, "columnNames is null"); + requireNonNull(columnTypes, "columnTypes is null"); + checkArgument(columnNames.size() == columnTypes.size(), "columnNames and columnTypes sizes don't match"); + this.columnNames = ImmutableList.copyOf(columnNames); + this.columnTypes = ImmutableList.copyOf(columnTypes); + } + + @JsonProperty + public List getColumnNames() + { + return columnNames; + } + + @JsonProperty + public List getColumnTypes() + { + return columnTypes; + } + + public List getOriginalColumnTypes() + { + return columnTypes; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduHandleResolver.java b/presto-kudu/src/main/java/io/presto/kudu/KuduHandleResolver.java new file mode 100644 index 000000000..8b1a508a0 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduHandleResolver.java @@ -0,0 +1,69 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorHandleResolver; +import io.prestosql.spi.connector.ConnectorInsertTableHandle; +import io.prestosql.spi.connector.ConnectorOutputTableHandle; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTableLayoutHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; + +public class KuduHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableHandleClass() + { + return KuduTableHandle.class; + } + + @Override + public Class getTableLayoutHandleClass() + { + return KuduTableLayoutHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return KuduColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return KuduSplit.class; + } + + @Override + public Class getTransactionHandleClass() + { + return KuduTransactionHandle.class; + } + + @Override + public Class getInsertTableHandleClass() + { + return KuduInsertTableHandle.class; + } + + @Override + public Class getOutputTableHandleClass() + { + return KuduOutputTableHandle.class; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduInsertTableHandle.java b/presto-kudu/src/main/java/io/presto/kudu/KuduInsertTableHandle.java new file mode 100644 index 000000000..602550ed5 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduInsertTableHandle.java @@ -0,0 +1,44 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.connector.ConnectorInsertTableHandle; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.type.Type; +import org.apache.kudu.client.KuduTable; + +import java.util.List; + +public class KuduInsertTableHandle + extends KuduExtendedTableHandle + implements ConnectorInsertTableHandle +{ + @JsonCreator + public KuduInsertTableHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaTableName") SchemaTableName schemaTableName, + @JsonProperty("columnNames") List columnNames, + @JsonProperty("columnTypes") List columnTypes) + { + super(connectorId, schemaTableName, columnNames, columnTypes, null); + } + + public KuduInsertTableHandle(String connectorId, SchemaTableName schemaTableName, + List columnNames, List columnTypes, KuduTable table) + { + super(connectorId, schemaTableName, columnNames, columnTypes, null); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduMetadata.java b/presto-kudu/src/main/java/io/presto/kudu/KuduMetadata.java new file mode 100644 index 000000000..888cb8620 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduMetadata.java @@ -0,0 +1,386 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.presto.kudu; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.slice.Slice; +import io.presto.kudu.properties.KuduTableProperties; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorInsertTableHandle; +import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorNewTableLayout; +import io.prestosql.spi.connector.ConnectorOutputMetadata; +import io.prestosql.spi.connector.ConnectorOutputTableHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTableLayout; +import io.prestosql.spi.connector.ConnectorTableLayoutHandle; +import io.prestosql.spi.connector.ConnectorTableLayoutResult; +import io.prestosql.spi.connector.ConnectorTableMetadata; +import io.prestosql.spi.connector.Constraint; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.connector.SchemaTablePrefix; +import io.prestosql.spi.statistics.ComputedStatistics; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Schema; +import org.apache.kudu.client.KuduTable; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Objects.requireNonNull; + +public class KuduMetadata + implements ConnectorMetadata +{ + private final String connectorId; + private KuduClientSession clientSession; + + @Inject + public KuduMetadata(KuduConnectorId connectorId, KuduClientSession clientSession) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.clientSession = requireNonNull(clientSession, "clientSession is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return clientSession.listSchemaNames(session); + } + + @Override + public List listTables(ConnectorSession session, Optional schemaName) + { + return clientSession.listTables(schemaName.get()); + } + + @Override + public Map> listTableColumns(ConnectorSession session, + SchemaTablePrefix prefix) + { + requireNonNull(prefix, "SchemaTablePrefix is null"); + + List tables; + if (!prefix.getSchema().isPresent()) { + tables = listTables(session, Optional.empty()); + } + else if (!prefix.getTable().isPresent()) { + tables = listTables(session, prefix.getSchema()); + } + else { + tables = ImmutableList.of(new SchemaTableName(prefix.getSchema().get(), prefix.getTable().get())); + } + + ImmutableMap.Builder> columns = ImmutableMap.builder(); + for (SchemaTableName tableName : tables) { + KuduTableHandle tableHandle = getTableHandle(session, tableName); + ConnectorTableMetadata tableMetadata = getTableMetadata(tableHandle, session); + columns.put(tableName, tableMetadata.getColumns()); + } + return columns.build(); + } + + public ConnectorTableMetadata getTableMetadata(KuduTableHandle tableHandle, ConnectorSession session) + { + KuduTable table = tableHandle.getTable(clientSession, session); + Schema schema = table.getSchema(); + + List columnsMetaList = schema.getColumns().stream() + .filter(col -> !col.isKey() || !col.getName().equals(KuduColumnHandle.ROW_ID)) + .map(col -> { + StringBuilder extra = new StringBuilder(); + if (col.isKey()) { + extra.append("key, "); + } + else if (col.isNullable()) { + extra.append("nullable, "); + } + if (col.getEncoding() != null) { + extra.append("encoding=").append(col.getEncoding().name()).append(", "); + } + if (col.getCompressionAlgorithm() != null) { + extra.append("compression=").append(col.getCompressionAlgorithm().name()).append(", "); + } + if (extra.length() > 2) { + extra.setLength(extra.length() - 2); + } + Type prestoType = TypeHelper.fromKuduColumn(col); + return new ColumnMetadata(col.getName(), prestoType, null, extra.toString(), false); + }).collect(toImmutableList()); + + Map properties = clientSession.getTableProperties(tableHandle, session); + return new ConnectorTableMetadata(tableHandle.getSchemaTableName(), columnsMetaList, properties); + } + + private KuduTableHandle fromConnectorTableHandle(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return Types.checkType(tableHandle, KuduTableHandle.class, "tableHandle"); + } + + @Override + public Map getColumnHandles(ConnectorSession session, + ConnectorTableHandle connectorTableHandle) + { + KuduTableHandle tableHandle = fromConnectorTableHandle(session, connectorTableHandle); + Schema schema = clientSession.getTableSchema(tableHandle, session); + + ImmutableMap.Builder columnHandles = ImmutableMap.builder(); + for (int i = 0; i < schema.getColumnCount(); i++) { + ColumnSchema col = schema.getColumnByIndex(i); + String name = col.getName(); + Type type = TypeHelper.fromKuduColumn(col); + KuduColumnHandle columnHandle = new KuduColumnHandle(name, i, type); + columnHandles.put(name, columnHandle); + } + + return columnHandles.build(); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, + ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + fromConnectorTableHandle(session, tableHandle); + KuduColumnHandle kuduColumnHandle = Types.checkType(columnHandle, KuduColumnHandle.class, "columnHandle"); + if (kuduColumnHandle.isVirtualRowId()) { + return new ColumnMetadata(KuduColumnHandle.ROW_ID, VarbinaryType.VARBINARY, null, true); + } + else { + return kuduColumnHandle.getColumnMetadata(); + } + } + + @Override + public KuduTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) + { + try { + KuduTable table = clientSession.openTable(schemaTableName, session); + return new KuduTableHandle(connectorId, schemaTableName, table); + } + catch (Exception e) { + return null; + } + } + + @Override + public List getTableLayouts(ConnectorSession session, + ConnectorTableHandle tableHandle, + Constraint constraint, + Optional> desiredColumns) + { + KuduTableHandle handle = fromConnectorTableHandle(session, tableHandle); + ConnectorTableLayout layout = new ConnectorTableLayout( + new KuduTableLayoutHandle(handle, constraint.getSummary(), desiredColumns)); + return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); + } + + @Override + public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) + { + return new ConnectorTableLayout(handle); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return getTableMetadataInternal(session, tableHandle); + } + + private ConnectorTableMetadata getTableMetadataInternal(ConnectorSession session, + ConnectorTableHandle tableHandle) + { + KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); + return getTableMetadata(kuduTableHandle, session); + } + + @Override + public void createSchema(ConnectorSession session, String schemaName, Map properties) + { + clientSession.createSchema(schemaName, session); + } + + @Override + public void dropSchema(ConnectorSession session, String schemaName) + { + clientSession.dropSchema(schemaName, session); + } + + @Override + public void createTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, boolean ignoreExisting) + { + clientSession.createTable(tableMetadata, ignoreExisting, session); + } + + @Override + public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle) + { + KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); + clientSession.dropTable(kuduTableHandle.getSchemaTableName()); + } + + @Override + public void renameTable(ConnectorSession session, ConnectorTableHandle tableHandle, SchemaTableName newTableName) + { + KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); + clientSession.renameTable(kuduTableHandle.getSchemaTableName(), newTableName); + } + + @Override + public void addColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnMetadata column) + { + KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); + clientSession.addColumn(kuduTableHandle.getSchemaTableName(), column); + } + + @Override + public void dropColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle column) + { + KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); + KuduColumnHandle kuduColumnHandle = (KuduColumnHandle) column; + clientSession.dropColumn(kuduTableHandle.getSchemaTableName(), kuduColumnHandle.getName()); + } + + @Override + public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle source, + String target) + { + KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); + KuduColumnHandle kuduColumnHandle = (KuduColumnHandle) source; + clientSession.renameColumn(kuduTableHandle.getSchemaTableName(), kuduColumnHandle.getName(), target); + } + + @Override + public ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle connectorTableHandle) + { + KuduTableHandle tableHandle = fromConnectorTableHandle(session, connectorTableHandle); + + KuduTable table = tableHandle.getTable(clientSession, session); + Schema schema = table.getSchema(); + + List columns = schema.getColumns(); + List columnNames = columns.stream().map(ColumnSchema::getName).collect(toImmutableList()); + List columnTypes = columns.stream() + .map(TypeHelper::fromKuduColumn).collect(toImmutableList()); + + return new KuduInsertTableHandle( + connectorId, + tableHandle.getSchemaTableName(), + columnNames, + columnTypes, + table); + } + + @Override + public ColumnHandle getDeleteRowIdColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return KuduColumnHandle.ROW_ID_HANDLE; + } + + @Override + public Optional finishInsert(ConnectorSession session, + ConnectorInsertTableHandle insertHandle, + Collection fragments, + Collection computedStatistics) + { + return Optional.empty(); + } + + @Override + public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, + Optional layout) + { + boolean generateUUID = !tableMetadata.getProperties().containsKey(KuduTableProperties.PARTITION_DESIGN); + ConnectorTableMetadata finalTableMetadata = tableMetadata; + if (generateUUID) { + String rowId = KuduColumnHandle.ROW_ID; + List copy = new ArrayList<>(tableMetadata.getColumns()); + copy.add(0, new ColumnMetadata(rowId, VarcharType.VARCHAR, "key=true", null, true)); + List finalColumns = ImmutableList.copyOf(copy); + Map propsCopy = new HashMap<>(tableMetadata.getProperties()); + propsCopy.put(KuduTableProperties.COLUMN_DESIGN, "{\"" + rowId + "\": {\"key\": true}}"); + propsCopy.put(KuduTableProperties.PARTITION_DESIGN, "{\"hash\": [{\"columns\": [\"" + rowId + "\"], " + + "\"buckets\": 2}]}"); + propsCopy.put(KuduTableProperties.NUM_REPLICAS, 1); + Map finalProperties = ImmutableMap.copyOf(propsCopy); + finalTableMetadata = new ConnectorTableMetadata(tableMetadata.getTable(), + finalColumns, finalProperties, tableMetadata.getComment()); + } + KuduTable table = clientSession.createTable(finalTableMetadata, false, session); + + Schema schema = table.getSchema(); + + List columns = schema.getColumns(); + List columnNames = columns.stream().map(ColumnSchema::getName).collect(toImmutableList()); + List columnTypes = columns.stream() + .map(TypeHelper::fromKuduColumn).collect(toImmutableList()); + List columnOriginalTypes = finalTableMetadata.getColumns().stream() + .map(ColumnMetadata::getType).collect(toImmutableList()); + + return new KuduOutputTableHandle( + connectorId, + finalTableMetadata.getTable(), + columnOriginalTypes, + columnNames, + columnTypes, + generateUUID, + table); + } + + @Override + public Optional finishCreateTable(ConnectorSession session, + ConnectorOutputTableHandle tableHandle, + Collection fragments, + Collection computedStatistics) + { + return Optional.empty(); + } + + public ColumnHandle getUpdateRowIdColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return KuduColumnHandle.ROW_ID_HANDLE; + } + + @Override + public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return fromConnectorTableHandle(session, tableHandle); + } + + @Override + public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHandle, Collection fragments) + { + } + + @Override + public boolean supportsMetadataDelete(ConnectorSession session, ConnectorTableHandle tableHandle, + ConnectorTableLayoutHandle tableLayoutHandle) + { + return false; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java b/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java new file mode 100644 index 000000000..39cd4c709 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java @@ -0,0 +1,196 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.google.inject.AbstractModule; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.multibindings.Multibinder; +import com.google.inject.multibindings.ProvidesIntoSet; +import io.airlift.log.Logger; +import io.presto.kudu.procedures.RangePartitionProcedures; +import io.presto.kudu.properties.KuduTableProperties; +import io.prestosql.spi.connector.ConnectorPageSinkProvider; +import io.prestosql.spi.connector.ConnectorPageSourceProvider; +import io.prestosql.spi.connector.ConnectorRecordSetProvider; +import io.prestosql.spi.connector.ConnectorSplitManager; +import io.prestosql.spi.procedure.Procedure; +import io.prestosql.spi.type.TypeManager; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.kudu.client.KuduClient; + +import javax.inject.Singleton; + +import java.io.FileInputStream; +import java.io.InputStream; +import java.security.PrivilegedExceptionAction; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static io.airlift.configuration.ConfigBinder.configBinder; +import static java.util.Objects.requireNonNull; + +public class KuduModule + extends AbstractModule +{ + private final String connectorId; + private final TypeManager typeManager; + private static final Logger log = Logger.get(KuduModule.class); + private NativeKuduClientSession nativeKuduClientSession; + private KuduClient kuduClient; + + public KuduModule(String connectorId, TypeManager typeManager) + { + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + protected void configure() + { + bind(TypeManager.class).toInstance(typeManager); + bind(KuduConnector.class).in(Scopes.SINGLETON); + bind(KuduConnectorId.class).toInstance(new KuduConnectorId(connectorId)); + bind(KuduMetadata.class).in(Scopes.SINGLETON); + bind(KuduTableProperties.class).in(Scopes.SINGLETON); + bind(ConnectorSplitManager.class).to(KuduSplitManager.class).in(Scopes.SINGLETON); + bind(ConnectorRecordSetProvider.class).to(KuduRecordSetProvider.class) + .in(Scopes.SINGLETON); + bind(ConnectorPageSourceProvider.class).to(KuduPageSourceProvider.class) + .in(Scopes.SINGLETON); + bind(ConnectorPageSinkProvider.class).to(KuduPageSinkProvider.class).in(Scopes.SINGLETON); + bind(KuduHandleResolver.class).in(Scopes.SINGLETON); + bind(KuduRecordSetProvider.class).in(Scopes.SINGLETON); + configBinder(binder()).bindConfig(KuduClientConfig.class); + + bind(RangePartitionProcedures.class).in(Scopes.SINGLETON); + Multibinder.newSetBinder(binder(), Procedure.class); + } + + @ProvidesIntoSet + Procedure getAddRangePartitionProcedure(RangePartitionProcedures procedures) + { + return procedures.getAddPartitionProcedure(); + } + + @ProvidesIntoSet + Procedure getDropRangePartitionProcedure(RangePartitionProcedures procedures) + { + return procedures.getDropPartitionProcedure(); + } + + @Singleton + @Provides + KuduClientSession createKuduClientSession( + KuduConnectorId connectorId, + KuduClientConfig config) + { + requireNonNull(config, "config is null"); + if ("true".equalsIgnoreCase(config.getIsKerberosOn())) { + kbAuth(config); + reauth(config, connectorId); + KuduClient client; + try { + //String name = UserGroupInformation.getLoginUser().getUserName(); + client = UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction() + { + @Override + public KuduClient run() throws Exception + { + KuduClient.KuduClientBuilder builder = new KuduClient.KuduClientBuilder(config.getMasterAddresses()); + builder.defaultAdminOperationTimeoutMs(config.getDefaultAdminOperationTimeout().toMillis()); + builder.defaultOperationTimeoutMs(config.getDefaultOperationTimeout().toMillis()); + builder.defaultSocketReadTimeoutMs(config.getDefaultSocketReadTimeout().toMillis()); + if (config.isDisableStatistics()) { + builder.disableStatistics(); + } + return builder.build(); + } + }); + this.kuduClient = client; + } + catch (Exception e) { + log.error(e, "failed to create kudu client"); + } + String tenant = config.getTenant(); + nativeKuduClientSession = new NativeKuduClientSession(connectorId, kuduClient, tenant, config); + return nativeKuduClientSession; + } + else { + KuduClient.KuduClientBuilder builder = new KuduClient.KuduClientBuilder(config.getMasterAddresses()); + builder.defaultAdminOperationTimeoutMs(config.getDefaultAdminOperationTimeout().toMillis()); + builder.defaultOperationTimeoutMs(config.getDefaultOperationTimeout().toMillis()); + builder.defaultSocketReadTimeoutMs(config.getDefaultSocketReadTimeout().toMillis()); + if (config.isDisableStatistics()) { + builder.disableStatistics(); + } + KuduClient client = builder.build(); + String tenant = config.getTenant(); + return new NativeKuduClientSession(connectorId, client, tenant, config); + } + } + + public void kbAuth(KuduClientConfig config) + { + try { + System.setProperty("javax.security.auth.useSubjectCredsOnly", "false"); + Configuration configuration = new Configuration(); + InputStream hdfs = new FileInputStream(config.getHdfsXmlPath()); + InputStream core = new FileInputStream(config.getCoreXmlPath()); + configuration.addResource(hdfs); + configuration.addResource(core); + UserGroupInformation.setConfiguration(configuration); + UserGroupInformation.loginUserFromKeytab(config.getKerberosUser(), config.getKeytabPath()); + } + catch (Exception e) { + e.printStackTrace(); + } + } + + public void reauth(KuduClientConfig config, KuduConnectorId connectorId) + { + ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = new ScheduledThreadPoolExecutor(1); + scheduledThreadPoolExecutor.scheduleAtFixedRate(new Runnable() { + @Override + public void run() + { + kbAuth(config); + try { + KuduClient client = UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction() { + @Override + public KuduClient run() throws Exception + { + KuduClient.KuduClientBuilder builder = new KuduClient.KuduClientBuilder(config.getMasterAddresses()); + builder.defaultAdminOperationTimeoutMs(config.getDefaultAdminOperationTimeout().toMillis()); + builder.defaultOperationTimeoutMs(config.getDefaultOperationTimeout().toMillis()); + builder.defaultSocketReadTimeoutMs(config.getDefaultSocketReadTimeout().toMillis()); + if (config.isDisableStatistics()) { + builder.disableStatistics(); + } + return builder.build(); + } + }); + //log.info("client==+++=======" + client.toString()); + KuduClient oldClient = nativeKuduClientSession.getClient(); + nativeKuduClientSession.setClient(client); + //oldClient.close(); + } + catch (Exception e) { + log.error(e, "failed to create kudu client"); + } + } + }, Integer.parseInt(config.getReauthDelayTime()), Integer.parseInt(config.getReauthPeriodTime()), TimeUnit.SECONDS); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduOutputTableHandle.java b/presto-kudu/src/main/java/io/presto/kudu/KuduOutputTableHandle.java new file mode 100644 index 000000000..9f9a1a102 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduOutputTableHandle.java @@ -0,0 +1,67 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.connector.ConnectorOutputTableHandle; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.type.Type; +import org.apache.kudu.client.KuduTable; + +import java.util.List; + +public class KuduOutputTableHandle + extends KuduExtendedTableHandle + implements ConnectorOutputTableHandle +{ + private final boolean generateUUID; + private final List originalColumnTypes; + + @JsonCreator + public KuduOutputTableHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaTableName") SchemaTableName schemaTableName, + @JsonProperty("originalColumnTypes") List originalColumnTypes, + @JsonProperty("columnNames") List columnNames, + @JsonProperty("columnTypes") List columnTypes, + @JsonProperty("generateUUID") boolean generateUUID) + { + this(connectorId, schemaTableName, originalColumnTypes, columnNames, columnTypes, generateUUID, null); + } + + public KuduOutputTableHandle(String connectorId, SchemaTableName schemaTableName, + List originalColumnTypes, + List columnNames, List columnTypes, + boolean generateUUID, KuduTable table) + { + super(connectorId, schemaTableName, columnNames, columnTypes, table); + this.originalColumnTypes = ImmutableList.copyOf(originalColumnTypes); + this.generateUUID = generateUUID; + } + + @JsonProperty + public boolean isGenerateUUID() + { + return generateUUID; + } + + @JsonProperty + @Override + public List getOriginalColumnTypes() + { + return originalColumnTypes; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduPageSink.java b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSink.java new file mode 100644 index 000000000..620121eda --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSink.java @@ -0,0 +1,192 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import io.prestosql.spi.Page; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.connector.ConnectorPageSink; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.type.DecimalType; +import io.prestosql.spi.type.SqlDate; +import io.prestosql.spi.type.SqlDecimal; +import io.prestosql.spi.type.Type; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduSession; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.PartialRow; +import org.apache.kudu.client.SessionConfiguration; +import org.apache.kudu.client.Upsert; + +import java.nio.charset.StandardCharsets; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static io.prestosql.spi.sql.expression.Time.Function.TIMESTAMP; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.StandardTypes.BIGINT; +import static io.prestosql.spi.type.StandardTypes.BOOLEAN; +import static io.prestosql.spi.type.StandardTypes.DATE; +import static io.prestosql.spi.type.StandardTypes.DOUBLE; +import static io.prestosql.spi.type.StandardTypes.INTEGER; +import static io.prestosql.spi.type.StandardTypes.VARBINARY; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.Varchars.isVarcharType; +import static java.lang.Float.intBitsToFloat; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; + +public class KuduPageSink + implements ConnectorPageSink +{ + private final ConnectorSession connectorSession; + private final KuduSession session; + private final KuduTable table; + private final List columnTypes; + private final List originalColumnTypes; + private final boolean generateUUID; + + private final String uuid; + private int nextSubId; + + public KuduPageSink(ConnectorSession connectorSession, KuduClientSession clientSession, + KuduExtendedTableHandle extendedTableHandle, + boolean generateUUID) + { + requireNonNull(clientSession, "clientSession is null"); + this.connectorSession = connectorSession; + this.columnTypes = extendedTableHandle.getColumnTypes(); + this.originalColumnTypes = extendedTableHandle.getOriginalColumnTypes(); + this.generateUUID = generateUUID; + + this.table = extendedTableHandle.getTable(clientSession, connectorSession); + this.session = clientSession.newSession(); + this.session.setFlushMode(SessionConfiguration.FlushMode.AUTO_FLUSH_BACKGROUND); + uuid = java.util.UUID.randomUUID().toString(); + } + + @Override + public CompletableFuture appendPage(Page page) + { + for (int position = 0; position < page.getPositionCount(); position++) { + Upsert upsert = table.newUpsert(); + PartialRow row = upsert.getRow(); + int start = 0; + if (generateUUID) { + String id = String.format("%s-%08x", uuid, nextSubId++); + row.addString(0, id); + start = 1; + } + + for (int channel = 0; channel < page.getChannelCount(); channel++) { + appendColumn(row, page, position, channel, channel + start); + } + + try { + session.apply(upsert); + } + catch (KuduException e) { + throw new RuntimeException(e); + } + } + return NOT_BLOCKED; + } + + private void appendColumn(PartialRow row, Page page, int position, int channel, int destChannel) + { + Block block = page.getBlock(channel); + Type type = columnTypes.get(destChannel); + if (block.isNull(position)) { + row.setNull(destChannel); + } + else if (TIMESTAMP.equals(type.toString())) { + row.addLong(destChannel, type.getLong(block, position) * 1000); + } + else if (REAL.equals(type.toString())) { + row.addFloat(destChannel, intBitsToFloat((int) type.getLong(block, position))); + } + else if (BIGINT.equals(type.toString())) { + row.addLong(destChannel, type.getLong(block, position)); + } + else if (INTEGER.equals(type.toString())) { + row.addInt(destChannel, (int) type.getLong(block, position)); + } + else if (SMALLINT.equals(type.toString())) { + row.addShort(destChannel, (short) type.getLong(block, position)); + } + else if (TINYINT.equals(type.toString())) { + row.addByte(destChannel, (byte) type.getLong(block, position)); + } + else if (BOOLEAN.equals(type.toString())) { + row.addBoolean(destChannel, type.getBoolean(block, position)); + } + else if (DOUBLE.equals(type.toString())) { + row.addDouble(destChannel, type.getDouble(block, position)); + } + else if (isVarcharType(type)) { + Type originalType = originalColumnTypes.get(destChannel); + if (DATE.equals(originalType.toString())) { + SqlDate date = (SqlDate) originalType.getObjectValue(connectorSession, block, position); + LocalDateTime ldt = LocalDateTime.ofEpochSecond(TimeUnit.DAYS.toSeconds(date.getDays()), 0, ZoneOffset.UTC); + byte[] bytes = ldt.format(DateTimeFormatter.ISO_LOCAL_DATE).getBytes(StandardCharsets.UTF_8); + row.addStringUtf8(destChannel, bytes); + } + else { + row.addString(destChannel, type.getSlice(block, position).toStringUtf8()); + } + } + else if (VARBINARY.equals(type.toString())) { + row.addBinary(destChannel, type.getSlice(block, position).toByteBuffer()); + } + else if (type instanceof DecimalType) { + SqlDecimal sqlDecimal = (SqlDecimal) type.getObjectValue(connectorSession, block, position); + row.addDecimal(destChannel, sqlDecimal.toBigDecimal()); + } + else { + throw new UnsupportedOperationException("Type is not supported: " + type); + } + } + + @Override + public CompletableFuture> finish() + { + closeSession(); + // the committer does not need any additional info + return completedFuture(ImmutableList.of()); + } + + @Override + public void abort() + { + closeSession(); + } + + private void closeSession() + { + try { + session.close(); + } + catch (KuduException e) { + throw new RuntimeException(e); + } + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduPageSinkProvider.java b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSinkProvider.java new file mode 100644 index 000000000..b37b5cc3d --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSinkProvider.java @@ -0,0 +1,57 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; +import io.prestosql.spi.connector.ConnectorInsertTableHandle; +import io.prestosql.spi.connector.ConnectorOutputTableHandle; +import io.prestosql.spi.connector.ConnectorPageSink; +import io.prestosql.spi.connector.ConnectorPageSinkProvider; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorTransactionHandle; + +import javax.inject.Inject; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class KuduPageSinkProvider + implements ConnectorPageSinkProvider +{ + private final KuduClientSession clientSession; + + @Inject + public KuduPageSinkProvider(KuduClientSession clientSession) + { + this.clientSession = requireNonNull(clientSession, "clientSession is null"); + } + + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorOutputTableHandle outputTableHandle) + { + requireNonNull(outputTableHandle, "outputTableHandle is null"); + checkArgument(outputTableHandle instanceof KuduOutputTableHandle, "outputTableHandle is not an instance of KuduOutputTableHandle"); + KuduOutputTableHandle handle = (KuduOutputTableHandle) outputTableHandle; + + return new KuduPageSink(session, clientSession, handle, handle.isGenerateUUID()); + } + + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorInsertTableHandle insertTableHandle) + { + requireNonNull(insertTableHandle, "insertTableHandle is null"); + checkArgument(insertTableHandle instanceof KuduInsertTableHandle, "insertTableHandle is not an instance of KuduInsertTableHandle"); + KuduInsertTableHandle handle = (KuduInsertTableHandle) insertTableHandle; + return new KuduPageSink(session, clientSession, handle, false); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduPageSourceProvider.java b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSourceProvider.java new file mode 100644 index 000000000..dac6a7963 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSourceProvider.java @@ -0,0 +1,52 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.google.inject.Inject; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorPageSource; +import io.prestosql.spi.connector.ConnectorPageSourceProvider; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.connector.RecordPageSource; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class KuduPageSourceProvider + implements ConnectorPageSourceProvider +{ + private KuduRecordSetProvider recordSetProvider; + + @Inject + public KuduPageSourceProvider(KuduRecordSetProvider recordSetProvider) + { + this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); + } + + @Override + public ConnectorPageSource createPageSource(ConnectorTransactionHandle transactionHandle, + ConnectorSession session, ConnectorSplit split, List columns) + { + KuduRecordSet recordSet = (KuduRecordSet) recordSetProvider.getRecordSet(transactionHandle, session, split, columns); + if (columns.contains(KuduColumnHandle.ROW_ID_HANDLE)) { + return new KuduUpdatablePageSource(recordSet); + } + else { + return new RecordPageSource(recordSet); + } + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduPlugin.java b/presto-kudu/src/main/java/io/presto/kudu/KuduPlugin.java new file mode 100644 index 000000000..f67de04fb --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduPlugin.java @@ -0,0 +1,28 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.Plugin; +import io.prestosql.spi.connector.ConnectorFactory; + +public class KuduPlugin + implements Plugin +{ + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new KuduConnectorFactory("kudu")); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java new file mode 100644 index 000000000..0bfefb258 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java @@ -0,0 +1,192 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.prestosql.spi.connector.RecordCursor; +import io.prestosql.spi.type.Type; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.client.RowResultIterator; + +import java.lang.reflect.Field; +import java.util.List; + +public class KuduRecordCursor + implements RecordCursor +{ + private static final Logger log = Logger.get(KuduRecordCursor.class); + + private final KuduScanner scanner; + private final List columnTypes; + private final Field rowDataField; + private RowResultIterator nextRows; + protected RowResult currentRow; + + private long totalBytes; + private long nanoStart; + private long nanoEnd; + private boolean started; + + public KuduRecordCursor(KuduScanner scanner, List columnTypes) + { + this.scanner = scanner; + this.columnTypes = columnTypes; + Field field = null; + try { + field = RowResult.class.getDeclaredField("rawData"); + field.setAccessible(true); + } + catch (NoSuchFieldException e) { + // ignore + } + this.rowDataField = field; + } + + @Override + public long getCompletedBytes() + { + return totalBytes; + } + + @Override + public long getReadTimeNanos() + { + return nanoStart > 0L ? (nanoEnd == 0 ? System.nanoTime() : nanoEnd) - nanoStart : 0L; + } + + @Override + public Type getType(int field) + { + return columnTypes.get(field); + } + + protected int mapping(int field) + { + return field; + } + + /** + * get next Row/Page + */ + @Override + public boolean advanceNextPosition() + { + boolean needNextRows = !started || !nextRows.hasNext(); + + if (!started) { + started = true; + nanoStart = System.nanoTime(); + } + + if (needNextRows) { + currentRow = null; + try { + do { + if (!scanner.hasMoreRows()) { + return false; + } + nextRows = scanner.nextRows(); + } while (!nextRows.hasNext()); + log.debug("Fetched " + nextRows.getNumRows() + " rows"); + } + catch (KuduException e) { + currentRow = null; + throw new RuntimeException(e); + } + } + + currentRow = nextRows.next(); + totalBytes += getRowLength(); + return true; + } + + private org.apache.kudu.util.Slice getCurrentRowRawData() + { + if (rowDataField != null && currentRow != null) { + try { + return ((org.apache.kudu.util.Slice) rowDataField.get(currentRow)); + } + catch (IllegalAccessException e) { + return null; + } + } + else { + return null; + } + } + + private int getRowLength() + { + org.apache.kudu.util.Slice rawData = getCurrentRowRawData(); + if (rawData != null) { + return rawData.length(); + } + else { + return columnTypes.size(); + } + } + + @Override + public boolean getBoolean(int field) + { + int index = mapping(field); + return TypeHelper.getBoolean(columnTypes.get(field), currentRow, index); + } + + @Override + public long getLong(int field) + { + int index = mapping(field); + return TypeHelper.getLong(columnTypes.get(field), currentRow, index); + } + + @Override + public double getDouble(int field) + { + int index = mapping(field); + return TypeHelper.getDouble(columnTypes.get(field), currentRow, index); + } + + @Override + public Slice getSlice(int field) + { + int index = mapping(field); + return TypeHelper.getSlice(columnTypes.get(field), currentRow, index); + } + + @Override + public Object getObject(int field) + { + int index = mapping(field); + return TypeHelper.getObject(columnTypes.get(field), currentRow, index); + } + + @Override + public boolean isNull(int field) + { + int mappedField = mapping(field); + return mappedField >= 0 && currentRow.isNull(mappedField); + } + + @Override + public void close() + { + nanoEnd = System.nanoTime(); + currentRow = null; + nextRows = null; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursorWithVirtualRowId.java b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursorWithVirtualRowId.java new file mode 100644 index 000000000..0ce9947ea --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursorWithVirtualRowId.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.prestosql.spi.type.Type; +import org.apache.kudu.Schema; +import org.apache.kudu.client.KeyEncoderAccessor; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.PartialRow; + +import java.util.List; +import java.util.Map; + +public class KuduRecordCursorWithVirtualRowId + extends KuduRecordCursor +{ + private final KuduTable table; + private final Map fieldMapping; + + public KuduRecordCursorWithVirtualRowId(KuduScanner scanner, KuduTable table, + List columnTypes, + Map fieldMapping) + { + super(scanner, columnTypes); + this.table = table; + this.fieldMapping = fieldMapping; + } + + @Override + protected int mapping(int field) + { + return fieldMapping.get(field); + } + + @Override + public Slice getSlice(int field) + { + if (fieldMapping.get(field) == -1) { + PartialRow partialRow = buildPrimaryKey(); + return Slices.wrappedBuffer(KeyEncoderAccessor.encodePrimaryKey(partialRow)); + } + else { + return super.getSlice(field); + } + } + + private PartialRow buildPrimaryKey() + { + Schema schema = table.getSchema(); + PartialRow row = new PartialRow(schema); + RowHelper.copyPrimaryKey(schema, currentRow, row); + return row; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordSet.java b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordSet.java new file mode 100644 index 000000000..cde57e60e --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordSet.java @@ -0,0 +1,97 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.RecordCursor; +import io.prestosql.spi.connector.RecordSet; +import io.prestosql.spi.type.Type; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.KuduTable; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static com.google.common.collect.ImmutableList.toImmutableList; + +public class KuduRecordSet + implements RecordSet +{ + private final KuduClientSession clientSession; + private final KuduSplit kuduSplit; + private final List columns; + private final boolean containsVirtualRowId; + private final ConnectorSession session; + + public KuduRecordSet(KuduClientSession clientSession, KuduSplit kuduSplit, List columns, ConnectorSession session) + { + this.clientSession = clientSession; + this.kuduSplit = kuduSplit; + this.columns = columns; + this.containsVirtualRowId = columns.contains(KuduColumnHandle.ROW_ID_HANDLE); + this.session = session; + } + + @Override + public List getColumnTypes() + { + return columns.stream() + .map(column -> ((KuduColumnHandle) column).getType()) + .collect(toImmutableList()); + } + + @Override + public RecordCursor cursor() + { + KuduScanner scanner = clientSession.createScanner(kuduSplit); + if (!containsVirtualRowId) { + return new KuduRecordCursor(scanner, getColumnTypes()); + } + else { + final int primaryKeyColumnCount = kuduSplit.getPrimaryKeyColumnCount(); + + Map fieldMapping = new HashMap<>(); + int index = primaryKeyColumnCount; + for (int i = 0; i < columns.size(); i++) { + KuduColumnHandle handle = (KuduColumnHandle) columns.get(i); + if (!handle.isVirtualRowId()) { + if (handle.getOrdinalPosition() < primaryKeyColumnCount) { + fieldMapping.put(i, handle.getOrdinalPosition()); + } + else { + fieldMapping.put(i, index); + index++; + } + } + else { + fieldMapping.put(i, -1); + } + } + KuduTable table = getTable(); + return new KuduRecordCursorWithVirtualRowId(scanner, table, getColumnTypes(), fieldMapping); + } + } + + KuduTable getTable() + { + return kuduSplit.getTableHandle().getTable(clientSession, session); + } + + KuduClientSession getClientSession() + { + return clientSession; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordSetProvider.java b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordSetProvider.java new file mode 100644 index 000000000..11e1a538d --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordSetProvider.java @@ -0,0 +1,58 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorRecordSetProvider; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.connector.RecordSet; + +import javax.inject.Inject; + +import java.util.List; + +import static io.presto.kudu.Types.checkType; +import static java.util.Objects.requireNonNull; + +public class KuduRecordSetProvider + implements ConnectorRecordSetProvider +{ + private final String connectorId; + private final KuduClientSession clientSession; + + @Inject + public KuduRecordSetProvider(KuduConnectorId connectorId, KuduClientSession clientSession) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.clientSession = clientSession; + } + + @Override + public RecordSet getRecordSet(ConnectorTransactionHandle transactionHandle, + ConnectorSession session, ConnectorSplit split, List columns) + { + requireNonNull(split, "split is null"); + requireNonNull(columns, "columns is null"); + + KuduSplit kuduSplit = checkType(split, KuduSplit.class, "split is not class KuduSplit"); + return new KuduRecordSet(clientSession, kuduSplit, columns, session); + } + + public KuduClientSession getClientSession() + { + return clientSession; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduSplit.java b/presto-kudu/src/main/java/io/presto/kudu/KuduSplit.java new file mode 100644 index 000000000..7ca0aebfb --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduSplit.java @@ -0,0 +1,78 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.HostAddress; +import io.prestosql.spi.connector.ConnectorSplit; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class KuduSplit + implements ConnectorSplit +{ + private final KuduTableHandle tableHandle; + private final int primaryKeyColumnCount; + private final byte[] pb; + + @JsonCreator + public KuduSplit(@JsonProperty("tableHandle") KuduTableHandle tableHandle, + @JsonProperty("primaryKeyColumnCount") int primaryKeyColumnCount, + @JsonProperty("pb") byte[] pb) + { + this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); + this.primaryKeyColumnCount = primaryKeyColumnCount; + this.pb = requireNonNull(pb, "pb is null"); + } + + @JsonProperty + public KuduTableHandle getTableHandle() + { + return tableHandle; + } + + @JsonProperty + public byte[] getPb() + { + return pb; + } + + @JsonProperty + public int getPrimaryKeyColumnCount() + { + return primaryKeyColumnCount; + } + + @Override + public boolean isRemotelyAccessible() + { + return true; + } + + @Override + public List getAddresses() + { + return ImmutableList.of(); + } + + @Override + public Object getInfo() + { + return this; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduSplitManager.java b/presto-kudu/src/main/java/io/presto/kudu/KuduSplitManager.java new file mode 100644 index 000000000..da983e80b --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduSplitManager.java @@ -0,0 +1,53 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.presto.kudu; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplitManager; +import io.prestosql.spi.connector.ConnectorSplitSource; +import io.prestosql.spi.connector.ConnectorTableLayoutHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.connector.FixedSplitSource; + +import javax.inject.Inject; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class KuduSplitManager + implements ConnectorSplitManager +{ + private final String connectorId; + private final KuduClientSession clientSession; + + @Inject + public KuduSplitManager(KuduConnectorId connectorId, KuduClientSession clientSession) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.clientSession = requireNonNull(clientSession, "clientSession is null"); + } + + @Override + public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHandle, + ConnectorSession session, ConnectorTableLayoutHandle layout, + SplitSchedulingStrategy splitSchedulingStrategy) + { + KuduTableLayoutHandle layoutHandle = (KuduTableLayoutHandle) layout; + + List splits = clientSession.buildKuduSplits(layoutHandle, session); + + return new FixedSplitSource(splits); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduTableHandle.java b/presto-kudu/src/main/java/io/presto/kudu/KuduTableHandle.java new file mode 100644 index 000000000..07f578a82 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduTableHandle.java @@ -0,0 +1,96 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.SchemaTableName; +import org.apache.kudu.client.KuduTable; + +import java.util.Objects; + +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class KuduTableHandle + implements ConnectorTableHandle +{ + private final String connectorId; + private final SchemaTableName schemaTableName; + private transient KuduTable table; + + @JsonCreator + public KuduTableHandle(@JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaTableName") SchemaTableName schemaTableName) + { + this(connectorId, schemaTableName, null); + } + + public KuduTableHandle(String connectorId, SchemaTableName schemaTableName, + KuduTable table) + { + this.connectorId = requireNonNull(connectorId.toLowerCase(ENGLISH), "connectorId is null"); + this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); + this.table = table; + } + + public KuduTable getTable(KuduClientSession session, ConnectorSession connectorSession) + { + if (table == null) { + table = session.openTable(schemaTableName, connectorSession); + } + return table; + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public SchemaTableName getSchemaTableName() + { + return schemaTableName; + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, schemaTableName); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + KuduTableHandle other = (KuduTableHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) && this.schemaTableName + .equals(other.getSchemaTableName()); + } + + @Override + public String toString() + { + return connectorId + ":" + schemaTableName; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduTableLayoutHandle.java b/presto-kudu/src/main/java/io/presto/kudu/KuduTableLayoutHandle.java new file mode 100644 index 000000000..c751359ee --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduTableLayoutHandle.java @@ -0,0 +1,98 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorTableLayoutHandle; +import io.prestosql.spi.predicate.TupleDomain; + +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class KuduTableLayoutHandle + implements ConnectorTableLayoutHandle +{ + private final KuduTableHandle tableHandle; + private final TupleDomain constraintSummary; + private final Optional> desiredColumns; + + @JsonCreator + public KuduTableLayoutHandle(@JsonProperty("tableHandle") KuduTableHandle tableHandle, + @JsonProperty("constraintSummary") TupleDomain constraintSummary, + @JsonProperty("desiredColumns") Optional> desiredColumns) + { + this.tableHandle = requireNonNull(tableHandle, "table is null"); + this.constraintSummary = constraintSummary; + this.desiredColumns = desiredColumns; + } + + @JsonProperty + public KuduTableHandle getTableHandle() + { + return tableHandle; + } + + @JsonProperty + public TupleDomain getConstraintSummary() + { + return constraintSummary; + } + + @JsonProperty + public Optional> getDesiredColumns() + { + return desiredColumns; + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KuduTableLayoutHandle other = (KuduTableLayoutHandle) obj; + return Objects.equals(tableHandle, other.tableHandle) + && Objects.equals(constraintSummary, other.constraintSummary) + && Objects.equals(desiredColumns, other.desiredColumns); + } + + @Override + public int hashCode() + { + return Objects.hash(tableHandle, + constraintSummary, + desiredColumns); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("tableHandle", tableHandle) + .add("constraintSummary", constraintSummary) + .add("desiredColumns", desiredColumns) + .toString(); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduTransactionHandle.java b/presto-kudu/src/main/java/io/presto/kudu/KuduTransactionHandle.java new file mode 100644 index 000000000..9d46880e6 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduTransactionHandle.java @@ -0,0 +1,21 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.spi.connector.ConnectorTransactionHandle; + +public enum KuduTransactionHandle implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduUpdatablePageSource.java b/presto-kudu/src/main/java/io/presto/kudu/KuduUpdatablePageSource.java new file mode 100644 index 000000000..ed16bce41 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduUpdatablePageSource.java @@ -0,0 +1,119 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.airlift.slice.Slice; +import io.prestosql.spi.Page; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.connector.RecordPageSource; +import io.prestosql.spi.connector.UpdatablePageSource; +import org.apache.kudu.Schema; +import org.apache.kudu.client.Delete; +import org.apache.kudu.client.KeyEncoderAccessor; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduSession; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.PartialRow; +import org.apache.kudu.client.SessionConfiguration.FlushMode; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; + +public class KuduUpdatablePageSource + implements UpdatablePageSource +{ + private final KuduClientSession clientSession; + private final KuduTable table; + private final RecordPageSource inner; + + public KuduUpdatablePageSource(KuduRecordSet recordSet) + { + this.clientSession = recordSet.getClientSession(); + this.table = recordSet.getTable(); + this.inner = new RecordPageSource(recordSet); + } + + @Override + public void deleteRows(Block rowIds) + { + Schema schema = table.getSchema(); + KuduSession session = clientSession.newSession(); + session.setFlushMode(FlushMode.AUTO_FLUSH_BACKGROUND); + try { + try { + for (int i = 0; i < rowIds.getPositionCount(); i++) { + int len = rowIds.getSliceLength(i); + Slice slice = rowIds.getSlice(i, 0, len); + PartialRow row = KeyEncoderAccessor.decodePrimaryKey(schema, slice.getBytes()); + Delete delete = table.newDelete(); + RowHelper.copyPrimaryKey(schema, row, delete.getRow()); + session.apply(delete); + } + } + finally { + session.close(); + } + } + catch (KuduException e) { + throw new RuntimeException(e); + } + } + + @Override + public CompletableFuture> finish() + { + CompletableFuture> cf = new CompletableFuture<>(); + cf.complete(Collections.emptyList()); + return cf; + } + + @Override + public long getCompletedBytes() + { + return inner.getCompletedBytes(); + } + + @Override + public long getReadTimeNanos() + { + return inner.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return inner.isFinished(); + } + + @Override + public Page getNextPage() + { + return inner.getNextPage(); + } + + @Override + public long getSystemMemoryUsage() + { + return inner.getSystemMemoryUsage(); + } + + @Override + public void close() throws IOException + { + inner.close(); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java b/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java new file mode 100644 index 000000000..7d1eb8721 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java @@ -0,0 +1,840 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.presto.kudu.properties.ColumnDesign; +import io.presto.kudu.properties.HashPartitionDefinition; +import io.presto.kudu.properties.KuduTableProperties; +import io.presto.kudu.properties.PartitionDesign; +import io.presto.kudu.properties.RangePartition; +import io.presto.kudu.properties.RangePartitionDefinition; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorTableMetadata; +import io.prestosql.spi.connector.SchemaNotFoundException; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.connector.TableNotFoundException; +import io.prestosql.spi.predicate.DiscreteValues; +import io.prestosql.spi.predicate.Domain; +import io.prestosql.spi.predicate.EquatableValueSet; +import io.prestosql.spi.predicate.Marker; +import io.prestosql.spi.predicate.Range; +import io.prestosql.spi.predicate.Ranges; +import io.prestosql.spi.predicate.SortedRangeSet; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.predicate.ValueSet; +import io.prestosql.spi.type.DecimalType; +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.ColumnTypeAttributes; +import org.apache.kudu.Schema; +import org.apache.kudu.Type; +import org.apache.kudu.client.AlterTableOptions; +import org.apache.kudu.client.CreateTableOptions; +import org.apache.kudu.client.Delete; +import org.apache.kudu.client.Insert; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduPredicate; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.KuduSession; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.PartialRow; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.client.RowResultIterator; +import org.apache.kudu.client.SessionConfiguration; +import org.apache.kudu.client.Upsert; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.prestosql.spi.StandardErrorCode.GENERIC_USER_ERROR; +import static io.prestosql.spi.StandardErrorCode.QUERY_REJECTED; + +public class NativeKuduClientSession + implements KuduClientSession +{ + public static final String DEFAULT_SCHEMA = "default"; + private final Logger log = Logger.get(getClass()); + private final KuduConnectorId connectorId; + private final String tenantPrefix; + private String rawSchemasTableName; + private KuduClient client; + private KuduTable rawSchemasTable; + private KuduClientConfig config; + + public NativeKuduClientSession(KuduConnectorId connectorId, KuduClient client, String tenant, KuduClientConfig config) + { + this.connectorId = connectorId; + this.client = client; + this.tenantPrefix = tenant == null ? "" : tenant + "."; + this.rawSchemasTableName = "_schemas"; + this.config = config; + String kuduMetadataOn = config.getKuduMetadataOn(); + if ("true".equalsIgnoreCase(kuduMetadataOn)) { + this.rawSchemasTableName = config.getKuduMetadataDatabase() + "." + this.rawSchemasTableName; + } + } + + public void setClient(KuduClient client) + { + this.client = client; + } + + public KuduClient getClient() + { + return client; + } + + public String getSchemaName(ConnectorSession session) + { + String user = session.getUser(); + return Optional.ofNullable(user).orElse("") + "_db."; + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + synchronized (DEFAULT_SCHEMA) { + try { + //log.info("client=========" + client.toString()); + if (rawSchemasTable == null) { + if (client.tableExists(rawSchemasTableName)) { + deleteSchemaTables(session); + } + createAndFillSchemasTable(session); + rawSchemasTable = getSchemasTable(session); + } + else { + if (!client.tableExists(rawSchemasTableName)) { + createAndFillSchemasTable(session); + rawSchemasTable = client.openTable(rawSchemasTableName); + } + else { + deleteSchemaTables(session); + createAndFillSchemasTable(session); + rawSchemasTable = getSchemasTable(session); + } + } + //log.info("client=1========" + client.toString()); + ColumnSchema tenantColumn = rawSchemasTable.getSchema().getColumnByIndex(0); + KuduScanner scanner = client.newScannerBuilder(rawSchemasTable) + .addPredicate(KuduPredicate.newComparisonPredicate(tenantColumn, KuduPredicate.ComparisonOp.EQUAL, tenantPrefix)) + .setProjectedColumnIndexes(ImmutableList.of(1)) + .build(); + //log.info("client=2========" + client.toString()); + RowResultIterator iterator = scanner.nextRows(); + ArrayList result = new ArrayList<>(); + while (iterator != null) { + for (RowResult row : iterator) { + result.add(row.getString(0)); + } + iterator = scanner.nextRows(); + } + return result; + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + } + + @Override + public boolean schemaExists(String schemaName, ConnectorSession session) + { + List list = listSchemaNames(session); + return list.contains(schemaName); + } + + private KuduTable getSchemasTable(ConnectorSession session) throws KuduException + { + //if (rawSchemasTable == null) { + rawSchemasTable = client.openTable(rawSchemasTableName); + //} + return rawSchemasTable; + } + + private void deleteSchemaTables(ConnectorSession connectorSession) throws KuduException + { + client.deleteTable(rawSchemasTableName); + } + + private void createAndFillSchemasTable(ConnectorSession connectorSession) throws KuduException + { + List existingSchemaNames = listSchemaNamesFromTablets(); + ColumnSchema tenantColumnSchema = new ColumnSchema.ColumnSchemaBuilder("tenant", Type.STRING) + .key(true).build(); + ColumnSchema schemaColumnSchema = new ColumnSchema.ColumnSchemaBuilder("schema", Type.STRING) + .key(true).build(); + Schema schema = new Schema(ImmutableList.of(tenantColumnSchema, schemaColumnSchema)); + CreateTableOptions options = new CreateTableOptions(); + options.setNumReplicas(1); // TODO config + options.addHashPartitions(ImmutableList.of(tenantColumnSchema.getName()), 2); + KuduTable schemasTable = client.createTable(rawSchemasTableName, schema, options); + KuduSession session = client.newSession(); + session.setFlushMode(SessionConfiguration.FlushMode.AUTO_FLUSH_BACKGROUND); + try { + if (!existingSchemaNames.contains(rawSchemasTableName)) { + int index = rawSchemasTableName.indexOf('.', tenantPrefix.length()); + if (index > tenantPrefix.length()) { + String schemaResult = rawSchemasTableName.substring(tenantPrefix.length(), index); + Insert insert = schemasTable.newInsert(); + fillSchemaRow(insert.getRow(), schemaResult); + session.apply(insert); + } + } + for (String schemaName : existingSchemaNames) { + Insert insert = schemasTable.newInsert(); + fillSchemaRow(insert.getRow(), schemaName); + session.apply(insert); + } + } + finally { + session.close(); + } + } + + private List listSchemaNamesFromTablets() + { + final String prefix = tenantPrefix; + List tables = internalListTables(prefix); + LinkedHashSet schemas = new LinkedHashSet<>(); + schemas.add(DEFAULT_SCHEMA); + for (String table : tables) { + int index = table.indexOf('.', prefix.length()); + if (index > prefix.length()) { + String schema = table.substring(prefix.length(), index); + schemas.add(schema); + } + } + return ImmutableList.copyOf(schemas); + } + + private List internalListTables(String prefix) + { + try { + List tables; + if (prefix.isEmpty()) { + tables = client.getTablesList().getTablesList(); + } + else { + tables = client.getTablesList(prefix).getTablesList().stream().filter(name -> name.startsWith(prefix)).collect(toImmutableList()); + } + return tables; + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + @Override + public List listTables(String schemaNameOrNull) + { + final int offset = tenantPrefix.length(); + final String prefix; + if (schemaNameOrNull == null || schemaNameOrNull.equals(DEFAULT_SCHEMA)) { + prefix = tenantPrefix; + } + else { + prefix = tenantPrefix + schemaNameOrNull + "."; + } + List tables = internalListTables(prefix); + return tables.stream().map(name -> { + int index = name.indexOf('.', offset); + if (index > offset) { + String schema = name.substring(offset, index); + String table = name.substring(index + 1); + return new SchemaTableName(schema, table); + } + else { + String schema = DEFAULT_SCHEMA; + String table = name.substring(offset); + return new SchemaTableName(schema, table); + } + }).collect(toImmutableList()); + } + + @Override + public boolean tableExists(SchemaTableName schemaTableName) + { + String rawName = toRawName(schemaTableName); + try { + return client.tableExists(rawName); + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + @Override + public Schema getTableSchema(KuduTableHandle tableHandle, ConnectorSession session) + { + KuduTable table = tableHandle.getTable(this, session); + return table.getSchema(); + } + + @Override + public Map getTableProperties(KuduTableHandle tableHandle, ConnectorSession session) + { + KuduTable table = tableHandle.getTable(this, session); + return KuduTableProperties.toMap(table); + } + + @Override + public List buildKuduSplits(KuduTableLayoutHandle layoutHandle, ConnectorSession session) + { + KuduTableHandle tableHandle = layoutHandle.getTableHandle(); + KuduTable table = tableHandle.getTable(this, session); + final int primaryKeyColumnCount = table.getSchema().getPrimaryKeyColumnCount(); + KuduScanToken.KuduScanTokenBuilder builder = client.newScanTokenBuilder(table); + + TupleDomain constraintSummary = layoutHandle.getConstraintSummary(); + if (!addConstraintPredicates(table, builder, constraintSummary)) { + return ImmutableList.of(); + } + + Optional> desiredColumns = layoutHandle.getDesiredColumns(); + if (desiredColumns.isPresent()) { + if (desiredColumns.get().contains(KuduColumnHandle.ROW_ID_HANDLE)) { + List columnIndexes = IntStream + .range(0, primaryKeyColumnCount) + .boxed().collect(Collectors.toList()); + for (ColumnHandle columnHandle : desiredColumns.get()) { + if (columnHandle instanceof KuduColumnHandle) { + KuduColumnHandle k = (KuduColumnHandle) columnHandle; + int index = k.getOrdinalPosition(); + if (index >= primaryKeyColumnCount) { + columnIndexes.add(index); + } + } + } + builder.setProjectedColumnIndexes(columnIndexes); + } + else { + List columnIndexes = desiredColumns.get().stream() + .map(handle -> ((KuduColumnHandle) handle).getOrdinalPosition()) + .collect(toImmutableList()); + builder.setProjectedColumnIndexes(columnIndexes); + } + } + + List tokens = builder.build(); + return tokens.stream() + .map(token -> toKuduSplit(tableHandle, token, primaryKeyColumnCount)) + .collect(toImmutableList()); + } + + @Override + public KuduScanner createScanner(KuduSplit kuduSplit) + { + try { + KuduScanner scanner = KuduScanToken.deserializeIntoScanner(kuduSplit.getPb(), client); + return scanner; + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public KuduTable openTable(SchemaTableName schemaTableName, ConnectorSession session) + { + String rawName = toRawName(schemaTableName); + try { + KuduTable table = client.openTable(rawName); + return table; + } + catch (Exception e) { + log.debug("Error on doOpenTable: " + e, e); + if (!listSchemaNames(session).contains(schemaTableName.getSchemaName())) { + throw new SchemaNotFoundException(schemaTableName.getSchemaName()); + } + else { + throw new TableNotFoundException(schemaTableName); + } + } + } + + @Override + public KuduSession newSession() + { + return client.newSession(); + } + + @Override + public void createSchema(String schemaName, ConnectorSession connectorSession) + { + if (DEFAULT_SCHEMA.equals(schemaName)) { + throw new SchemaAlreadyExistsException(schemaName); + } + else { + try { + KuduTable schemasTable = getSchemasTable(connectorSession); + KuduSession session = client.newSession(); + try { + Upsert upsert = schemasTable.newUpsert(); + fillSchemaRow(upsert.getRow(), schemaName); + session.apply(upsert); + } + finally { + session.close(); + } + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + } + + private void fillSchemaRow(PartialRow row, String schemaName) + { + row.addString(0, tenantPrefix); + row.addString(1, schemaName); + } + + @Override + public void dropSchema(String schemaName, ConnectorSession connectorSession) + { + if (DEFAULT_SCHEMA.equals(schemaName)) { + throw new PrestoException(GENERIC_USER_ERROR, "Deleting default schema not allowed."); + } + else { + try { + for (SchemaTableName table : listTables(schemaName)) { + dropTable(table); + } + KuduTable schemasTable = getSchemasTable(connectorSession); + KuduSession session = client.newSession(); + try { + Delete delete = schemasTable.newDelete(); + fillSchemaRow(delete.getRow(), schemaName); + session.apply(delete); + } + finally { + session.close(); + } + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + } + + @Override + public void dropTable(SchemaTableName schemaTableName) + { + try { + String rawName = toRawName(schemaTableName); + client.deleteTable(rawName); + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + @Override + public void renameTable(SchemaTableName schemaTableName, SchemaTableName newSchemaTableName) + { + try { + String rawName = toRawName(schemaTableName); + String newRawName = toRawName(newSchemaTableName); + AlterTableOptions alterOptions = new AlterTableOptions(); + alterOptions.renameTable(newRawName); + client.alterTable(rawName, alterOptions); + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + @Override + public KuduTable createTable(ConnectorTableMetadata tableMetadata, boolean ignoreExisting, ConnectorSession session) + { + try { + SchemaTableName schemeTableName = tableMetadata.getTable(); + String rawName = toRawName(schemeTableName); + if (ignoreExisting) { + if (client.tableExists(rawName)) { + return null; + } + } + if (!schemaExists(schemeTableName.getSchemaName(), session)) { + throw new SchemaNotFoundException(schemeTableName.getSchemaName()); + } + List columns = tableMetadata.getColumns(); + Map properties = tableMetadata.getProperties(); + + Schema schema = buildSchema(columns, properties); + CreateTableOptions options = buildCreateTableOptions(schema, properties); + return client.createTable(rawName, schema, options); + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + @Override + public void addColumn(SchemaTableName schemaTableName, ColumnMetadata column) + { + try { + String rawName = toRawName(schemaTableName); + AlterTableOptions alterOptions = new AlterTableOptions(); + Type type = TypeHelper.toKuduClientType(column.getType()); + alterOptions.addNullableColumn(column.getName(), type); + client.alterTable(rawName, alterOptions); + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + @Override + public void dropColumn(SchemaTableName schemaTableName, String name) + { + try { + String rawName = toRawName(schemaTableName); + AlterTableOptions alterOptions = new AlterTableOptions(); + alterOptions.dropColumn(name); + client.alterTable(rawName, alterOptions); + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + @Override + public void renameColumn(SchemaTableName schemaTableName, String oldName, String newName) + { + try { + String rawName = toRawName(schemaTableName); + AlterTableOptions alterOptions = new AlterTableOptions(); + alterOptions.renameColumn(oldName, newName); + client.alterTable(rawName, alterOptions); + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + private enum RangePartitionChange + { + ADD, DROP + } + + @Override + public void addRangePartition(SchemaTableName schemaTableName, RangePartition rangePartition) + { + changeRangePartition(schemaTableName, rangePartition, RangePartitionChange.ADD); + } + + @Override + public void dropRangePartition(SchemaTableName schemaTableName, RangePartition rangePartition) + { + changeRangePartition(schemaTableName, rangePartition, RangePartitionChange.DROP); + } + + private void changeRangePartition(SchemaTableName schemaTableName, RangePartition rangePartition, + RangePartitionChange change) + { + try { + String rawName = toRawName(schemaTableName); + KuduTable table = client.openTable(rawName); + Schema schema = table.getSchema(); + PartitionDesign design = KuduTableProperties.getPartitionDesign(table); + RangePartitionDefinition definition = design.getRange(); + if (definition == null) { + throw new PrestoException(QUERY_REJECTED, "Table " + schemaTableName + " has no range partition"); + } + PartialRow lowerBound = KuduTableProperties.toRangeBoundToPartialRow(schema, definition, rangePartition.getLower()); + PartialRow upperBound = KuduTableProperties.toRangeBoundToPartialRow(schema, definition, rangePartition.getUpper()); + AlterTableOptions alterOptions = new AlterTableOptions(); + switch (change) { + case ADD: + alterOptions.addRangePartition(lowerBound, upperBound); + break; + case DROP: + alterOptions.dropRangePartition(lowerBound, upperBound); + break; + } + client.alterTable(rawName, alterOptions); + } + catch (PrestoException e) { + throw e; + } + catch (KuduException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, e); + } + } + + private Schema buildSchema(List columns, Map properties) + { + Optional> optColumnDesign = KuduTableProperties.getColumnDesign(properties); + + Map columnDesignMap = optColumnDesign.orElse(ImmutableMap.of()); + + List kuduColumns = columns.stream() + .map(columnMetadata -> toColumnSchema(columnMetadata, columnDesignMap)) + .collect(ImmutableList.toImmutableList()); + return new Schema(kuduColumns); + } + + private ColumnSchema toColumnSchema(ColumnMetadata columnMetadata, Map columnDesignMap) + { + String name = columnMetadata.getName(); + ColumnDesign design = columnDesignMap.getOrDefault(name, ColumnDesign.DEFAULT); + Type ktype = TypeHelper.toKuduClientType(columnMetadata.getType()); + ColumnSchema.ColumnSchemaBuilder builder = new ColumnSchema.ColumnSchemaBuilder(name, ktype); + builder.key(design.isKey()).nullable(design.isNullable()); + setEncoding(name, builder, design); + setCompression(name, builder, design); + setTypeAttributes(columnMetadata, builder); + return builder.build(); + } + + private void setTypeAttributes(ColumnMetadata columnMetadata, ColumnSchema.ColumnSchemaBuilder builder) + { + if (columnMetadata.getType() instanceof DecimalType) { + DecimalType type = (DecimalType) columnMetadata.getType(); + ColumnTypeAttributes attributes = new ColumnTypeAttributes.ColumnTypeAttributesBuilder() + .precision(type.getPrecision()) + .scale(type.getScale()).build(); + builder.typeAttributes(attributes); + } + } + + private void setCompression(String name, ColumnSchema.ColumnSchemaBuilder builder, ColumnDesign design) + { + if (design.getCompression() != null) { + try { + ColumnSchema.CompressionAlgorithm algorithm = + ColumnSchema.CompressionAlgorithm.valueOf(design.getCompression().toUpperCase()); + builder.compressionAlgorithm(algorithm); + } + catch (IllegalArgumentException e) { + throw new RuntimeException("Unknown compression algorithm " + design.getCompression() + " for column " + name); + } + } + } + + private void setEncoding(String name, ColumnSchema.ColumnSchemaBuilder builder, ColumnDesign design) + { + if (design.getEncoding() != null) { + try { + ColumnSchema.Encoding encoding = + ColumnSchema.Encoding.valueOf(design.getEncoding().toUpperCase()); + builder.encoding(encoding); + } + catch (IllegalArgumentException e) { + throw new RuntimeException("Unknown encoding " + design.getEncoding() + " for column " + name); + } + } + } + + private CreateTableOptions buildCreateTableOptions(Schema schema, Map properties) + { + CreateTableOptions options = new CreateTableOptions(); + + RangePartitionDefinition rangePartitionDefinition = null; + Optional optPartitionDesign = KuduTableProperties.getPartitionDesign(properties); + if (optPartitionDesign.isPresent()) { + PartitionDesign partitionDesign = optPartitionDesign.get(); + if (partitionDesign.getHash() != null) { + for (HashPartitionDefinition partition : partitionDesign.getHash()) { + options.addHashPartitions(partition.getColumns(), partition.getBuckets()); + } + } + if (partitionDesign.getRange() != null) { + rangePartitionDefinition = partitionDesign.getRange(); + options.setRangePartitionColumns(rangePartitionDefinition.getColumns()); + } + } + else { + String firstColumn = schema.getColumnByIndex(0).getName(); + options.setRangePartitionColumns(Collections.singletonList(firstColumn)); + } + + List rangePartitions = KuduTableProperties.getRangePartitions(properties); + if (rangePartitionDefinition != null && !rangePartitions.isEmpty()) { + for (RangePartition rangePartition : rangePartitions) { + PartialRow lower = KuduTableProperties.toRangeBoundToPartialRow(schema, rangePartitionDefinition, rangePartition.getLower()); + PartialRow upper = KuduTableProperties.toRangeBoundToPartialRow(schema, rangePartitionDefinition, rangePartition.getUpper()); + options.addRangePartition(lower, upper); + } + } + Optional numReplicas = KuduTableProperties.getNumReplicas(properties); + numReplicas.ifPresent(options::setNumReplicas); + return options; + } + + /** + * translates TupleDomain to KuduPredicates. + * + * @return false if TupleDomain or one of its domains is none + */ + private boolean addConstraintPredicates(KuduTable table, KuduScanToken.KuduScanTokenBuilder builder, + TupleDomain constraintSummary) + { + if (constraintSummary.isNone()) { + return false; + } + else if (!constraintSummary.isAll()) { + Schema schema = table.getSchema(); + for (TupleDomain.ColumnDomain columnDomain : constraintSummary.getColumnDomains().get()) { + int position = ((KuduColumnHandle) columnDomain.getColumn()).getOrdinalPosition(); + ColumnSchema columnSchema = schema.getColumnByIndex(position); + Domain domain = columnDomain.getDomain(); + if (domain.isNone()) { + return false; + } + else if (domain.isAll()) { + // no restriction + } + else if (domain.isOnlyNull()) { + builder.addPredicate(KuduPredicate.newIsNullPredicate(columnSchema)); + } + else if (domain.getValues().isAll() && domain.isNullAllowed()) { + builder.addPredicate(KuduPredicate.newIsNotNullPredicate(columnSchema)); + } + else if (domain.isSingleValue()) { + KuduPredicate predicate = createEqualsPredicate(columnSchema, domain.getSingleValue()); + builder.addPredicate(predicate); + } + else { + ValueSet valueSet = domain.getValues(); + if (valueSet instanceof EquatableValueSet) { + DiscreteValues discreteValues = valueSet.getDiscreteValues(); + KuduPredicate predicate = createInListPredicate(columnSchema, discreteValues); + builder.addPredicate(predicate); + } + else if (valueSet instanceof SortedRangeSet) { + Ranges ranges = ((SortedRangeSet) valueSet).getRanges(); + Range span = ranges.getSpan(); + Marker low = span.getLow(); + if (!low.isLowerUnbounded()) { + KuduPredicate.ComparisonOp op = (low.getBound() == Marker.Bound.ABOVE) + ? KuduPredicate.ComparisonOp.GREATER : KuduPredicate.ComparisonOp.GREATER_EQUAL; + KuduPredicate predicate = createComparisonPredicate(columnSchema, op, low.getValue()); + builder.addPredicate(predicate); + } + Marker high = span.getHigh(); + if (!high.isUpperUnbounded()) { + KuduPredicate.ComparisonOp op = (low.getBound() == Marker.Bound.BELOW) + ? KuduPredicate.ComparisonOp.LESS : KuduPredicate.ComparisonOp.LESS_EQUAL; + KuduPredicate predicate = createComparisonPredicate(columnSchema, op, high.getValue()); + builder.addPredicate(predicate); + } + } + else { + throw new IllegalStateException("Unexpected domain: " + domain); + } + } + } + } + return true; + } + + private KuduPredicate createInListPredicate(ColumnSchema columnSchema, DiscreteValues discreteValues) + { + io.prestosql.spi.type.Type type = TypeHelper.fromKuduColumn(columnSchema); + List javaValues = discreteValues.getValues().stream().map(value -> TypeHelper.getJavaValue(type, value)).collect(toImmutableList()); + return KuduPredicate.newInListPredicate(columnSchema, javaValues); + } + + private KuduPredicate createEqualsPredicate(ColumnSchema columnSchema, Object value) + { + return createComparisonPredicate(columnSchema, KuduPredicate.ComparisonOp.EQUAL, value); + } + + private KuduPredicate createComparisonPredicate(ColumnSchema columnSchema, + KuduPredicate.ComparisonOp op, + Object value) + { + io.prestosql.spi.type.Type type = TypeHelper.fromKuduColumn(columnSchema); + Object javaValue = TypeHelper.getJavaValue(type, value); + if (javaValue instanceof Long) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (Long) javaValue); + } + else if (javaValue instanceof Integer) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (Integer) javaValue); + } + else if (javaValue instanceof Short) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (Short) javaValue); + } + else if (javaValue instanceof Byte) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (Byte) javaValue); + } + else if (javaValue instanceof String) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (String) javaValue); + } + else if (javaValue instanceof Double) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (Double) javaValue); + } + else if (javaValue instanceof Float) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (Float) javaValue); + } + else if (javaValue instanceof Boolean) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (Boolean) javaValue); + } + else if (javaValue instanceof byte[]) { + return KuduPredicate.newComparisonPredicate(columnSchema, op, (byte[]) javaValue); + } + else if (javaValue == null) { + throw new IllegalStateException("Unexpected null java value for column " + columnSchema.getName()); + } + else { + throw new IllegalStateException("Unexpected java value for column " + + columnSchema.getName() + ": " + javaValue + "(" + javaValue.getClass() + ")"); + } + } + + private KuduSplit toKuduSplit(KuduTableHandle tableHandle, KuduScanToken token, + int primaryKeyColumnCount) + { + try { + byte[] pb = token.serialize(); + return new KuduSplit(tableHandle, primaryKeyColumnCount, pb); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private String toRawName(SchemaTableName schemaTableName) + { + String rawName; + if (schemaTableName.getSchemaName().equals(DEFAULT_SCHEMA)) { + rawName = tenantPrefix + schemaTableName.getTableName(); + } + else { + rawName = tenantPrefix + schemaTableName.getSchemaName() + "." + schemaTableName.getTableName(); + } + return rawName; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/RowHelper.java b/presto-kudu/src/main/java/io/presto/kudu/RowHelper.java new file mode 100644 index 000000000..687087821 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/RowHelper.java @@ -0,0 +1,106 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import org.apache.kudu.Schema; +import org.apache.kudu.client.PartialRow; +import org.apache.kudu.client.RowResult; + +import java.nio.charset.StandardCharsets; + +public class RowHelper +{ + private RowHelper() + { + } + + public static void copyPrimaryKey(Schema schema, RowResult from, PartialRow to) + { + for (int i = 0; i < schema.getPrimaryKeyColumnCount(); i++) { + switch (schema.getColumnByIndex(i).getType()) { + case STRING: + to.addStringUtf8(i, from.getString(i).getBytes(StandardCharsets.UTF_8)); + break; + case INT64: + case UNIXTIME_MICROS: + to.addLong(i, from.getLong(i)); + break; + case INT32: + to.addInt(i, from.getInt(i)); + break; + case INT16: + to.addShort(i, from.getShort(i)); + break; + case INT8: + to.addByte(i, from.getByte(i)); + break; + case DOUBLE: + to.addDouble(i, from.getDouble(i)); + break; + case FLOAT: + to.addFloat(i, from.getFloat(i)); + break; + case BOOL: + to.addBoolean(i, from.getBoolean(i)); + break; + case BINARY: + to.addBinary(i, from.getBinary(i)); + break; + default: + throw new IllegalStateException("Unknown type " + schema.getColumnByIndex(i).getType() + + " for column " + schema.getColumnByIndex(i).getName()); + } + } + } + + public static void copyPrimaryKey(Schema schema, PartialRow from, PartialRow to) + { + for (int i = 0; i < schema.getPrimaryKeyColumnCount(); i++) { + switch (schema.getColumnByIndex(i).getType()) { + case STRING: + to.addStringUtf8(i, from.getString(i).getBytes(StandardCharsets.UTF_8)); + break; + case INT64: + case UNIXTIME_MICROS: + to.addLong(i, from.getLong(i)); + break; + case INT32: + to.addInt(i, from.getInt(i)); + break; + case INT16: + to.addShort(i, from.getShort(i)); + break; + case INT8: + to.addByte(i, from.getByte(i)); + break; + case DOUBLE: + to.addDouble(i, from.getDouble(i)); + break; + case FLOAT: + to.addFloat(i, from.getFloat(i)); + break; + case BOOL: + to.addBoolean(i, from.getBoolean(i)); + break; + case BINARY: + to.addBinary(i, from.getBinary(i)); + break; + default: + throw new IllegalStateException("Unknown type " + schema.getColumnByIndex(i).getType() + + " for column " + schema.getColumnByIndex(i).getName()); + } + } + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/SchemaAlreadyExistsException.java b/presto-kudu/src/main/java/io/presto/kudu/SchemaAlreadyExistsException.java new file mode 100644 index 000000000..e24564572 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/SchemaAlreadyExistsException.java @@ -0,0 +1,42 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.presto.kudu; + +import io.prestosql.spi.PrestoException; + +import static io.prestosql.spi.StandardErrorCode.ALREADY_EXISTS; +import static java.lang.String.format; + +public class SchemaAlreadyExistsException + extends PrestoException +{ + private final String schemaName; + + public SchemaAlreadyExistsException(String schemaName) + { + this(schemaName, format("Schema already exists: '%s'", schemaName)); + } + + public SchemaAlreadyExistsException(String schemaName, String message) + { + super(ALREADY_EXISTS, message); + this.schemaName = schemaName; + } + + public String getSchemaName() + { + return schemaName; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/TypeHelper.java b/presto-kudu/src/main/java/io/presto/kudu/TypeHelper.java new file mode 100644 index 000000000..f20e50d1d --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/TypeHelper.java @@ -0,0 +1,345 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.presto.kudu; + +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.prestosql.spi.predicate.NullableValue; +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.CharType; +import io.prestosql.spi.type.DateType; +import io.prestosql.spi.type.DecimalType; +import io.prestosql.spi.type.Decimals; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.RealType; +import io.prestosql.spi.type.SmallintType; +import io.prestosql.spi.type.TimestampType; +import io.prestosql.spi.type.TinyintType; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.ColumnTypeAttributes; +import org.apache.kudu.client.PartialRow; +import org.apache.kudu.client.RowResult; + +import java.math.BigDecimal; + +import static io.airlift.slice.Slices.utf8Slice; +import static io.airlift.slice.Slices.wrappedBuffer; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.Float.intBitsToFloat; + +public class TypeHelper +{ + private TypeHelper() + { + } + + static org.apache.kudu.Type toKuduClientType(Type type) + { + if (type instanceof VarcharType) { + return org.apache.kudu.Type.STRING; + } + else if (type == TimestampType.TIMESTAMP) { + return org.apache.kudu.Type.UNIXTIME_MICROS; + } + else if (type == BigintType.BIGINT) { + return org.apache.kudu.Type.INT64; + } + else if (type == IntegerType.INTEGER) { + return org.apache.kudu.Type.INT32; + } + else if (type == SmallintType.SMALLINT) { + return org.apache.kudu.Type.INT16; + } + else if (type == TinyintType.TINYINT) { + return org.apache.kudu.Type.INT8; + } + else if (type == RealType.REAL) { + return org.apache.kudu.Type.FLOAT; + } + else if (type == DoubleType.DOUBLE) { + return org.apache.kudu.Type.DOUBLE; + } + else if (type == BooleanType.BOOLEAN) { + return org.apache.kudu.Type.BOOL; + } + else if (type instanceof VarbinaryType) { + return org.apache.kudu.Type.BINARY; + } + else if (type instanceof DecimalType) { + return org.apache.kudu.Type.DECIMAL; + } + else if (type == DateType.DATE) { + return org.apache.kudu.Type.STRING; + } + else if (type instanceof CharType) { + return org.apache.kudu.Type.STRING; + } + else { + throw new IllegalStateException("Type mapping implemented for Presto type: " + type); + } + } + + public static Type fromKuduColumn(ColumnSchema column) + { + return fromKuduClientType(column.getType(), column.getTypeAttributes()); + } + + private static Type fromKuduClientType(org.apache.kudu.Type ktype, ColumnTypeAttributes attributes) + { + switch (ktype) { + case STRING: + return VarcharType.VARCHAR; + case UNIXTIME_MICROS: + return TimestampType.TIMESTAMP; + case INT64: + return BigintType.BIGINT; + case INT32: + return IntegerType.INTEGER; + case INT16: + return SmallintType.SMALLINT; + case INT8: + return TinyintType.TINYINT; + case FLOAT: + return RealType.REAL; + case DOUBLE: + return DoubleType.DOUBLE; + case BOOL: + return BooleanType.BOOLEAN; + case BINARY: + return VarbinaryType.VARBINARY; + case DECIMAL: + return DecimalType.createDecimalType(attributes.getPrecision(), attributes.getScale()); + default: + throw new IllegalStateException("Kudu type not implemented for " + ktype); + } + } + + public static Type mappedType(Type sourceType) + { + if (sourceType == DateType.DATE) { + return VarcharType.VARCHAR; + } + else { + return sourceType; + } + } + + public static NullableValue getColumnValue(Type type, PartialRow row, int i) + { + if (row.isNull(i)) { + return NullableValue.asNull(type); + } + else { + if (type instanceof VarcharType) { + return NullableValue.of(type, utf8Slice(row.getString(i))); + } + else if (type == TimestampType.TIMESTAMP) { + return NullableValue.of(type, row.getLong(i) / 1000); + } + else if (type == BigintType.BIGINT) { + return NullableValue.of(type, row.getLong(i)); + } + else if (type == IntegerType.INTEGER) { + return NullableValue.of(type, row.getInt(i)); + } + else if (type == SmallintType.SMALLINT) { + return NullableValue.of(type, row.getShort(i)); + } + else if (type == TinyintType.TINYINT) { + return NullableValue.of(type, row.getByte(i)); + } + else if (type == DoubleType.DOUBLE) { + return NullableValue.of(type, row.getDouble(i)); + } + else if (type == RealType.REAL) { + return NullableValue.of(type, (long) floatToRawIntBits(row.getFloat(i))); + } + else if (type == BooleanType.BOOLEAN) { + return NullableValue.of(type, row.getBoolean(i)); + } + else if (type instanceof VarbinaryType) { + return NullableValue.of(type, wrappedBuffer(row.getBinary(i))); + } + else if (type instanceof DecimalType) { + return NullableValue.of(type, row.getDecimal(i)); + } + else { + throw new IllegalStateException("Handling of type " + type + " is not implemented"); + } + } + } + + public static Object getJavaValue(Type type, Object nativeValue) + { + if (type instanceof VarcharType) { + return ((Slice) nativeValue).toStringUtf8(); + } + else if (type == TimestampType.TIMESTAMP) { + return ((Long) nativeValue) * 1000; + } + else if (type == BigintType.BIGINT) { + return nativeValue; + } + else if (type == IntegerType.INTEGER) { + return ((Long) nativeValue).intValue(); + } + else if (type == SmallintType.SMALLINT) { + return ((Long) nativeValue).shortValue(); + } + else if (type == TinyintType.TINYINT) { + return ((Long) nativeValue).byteValue(); + } + else if (type == DoubleType.DOUBLE) { + return nativeValue; + } + else if (type == RealType.REAL) { + // conversion can result in precision lost + return intBitsToFloat(((Long) nativeValue).intValue()); + } + else if (type == BooleanType.BOOLEAN) { + return nativeValue; + } + else if (type instanceof VarbinaryType) { + return ((Slice) nativeValue).toByteBuffer(); + } + else if (type instanceof DecimalType) { + return nativeValue; + } + else { + throw new IllegalStateException("Back conversion not implemented for " + type); + } + } + + public static Object getObject(Type type, RowResult row, int field) + { + if (row.isNull(field)) { + return null; + } + else { + if (type instanceof VarcharType) { + return row.getString(field); + } + else if (type == TimestampType.TIMESTAMP) { + return row.getLong(field) / 1000; + } + else if (type == BigintType.BIGINT) { + return row.getLong(field); + } + else if (type == IntegerType.INTEGER) { + return row.getInt(field); + } + else if (type == SmallintType.SMALLINT) { + return row.getShort(field); + } + else if (type == TinyintType.TINYINT) { + return row.getByte(field); + } + else if (type == DoubleType.DOUBLE) { + return row.getDouble(field); + } + else if (type == RealType.REAL) { + return row.getFloat(field); + } + else if (type == BooleanType.BOOLEAN) { + return row.getBoolean(field); + } + else if (type instanceof VarbinaryType) { + return Slices.wrappedBuffer(row.getBinary(field)); + } + else if (type instanceof DecimalType) { + return row.getDecimal(field); + } + else { + throw new IllegalStateException("getObject not implemented for " + type); + } + } + } + + public static long getLong(Type type, RowResult row, int field) + { + if (type == TimestampType.TIMESTAMP) { + return row.getLong(field) / 1000; + } + else if (type == BigintType.BIGINT) { + return row.getLong(field); + } + else if (type == IntegerType.INTEGER) { + return row.getInt(field); + } + else if (type == SmallintType.SMALLINT) { + return row.getShort(field); + } + else if (type == TinyintType.TINYINT) { + return row.getByte(field); + } + else if (type == RealType.REAL) { + return floatToRawIntBits(row.getFloat(field)); + } + else if (type instanceof DecimalType) { + DecimalType dtype = (DecimalType) type; + if (dtype.isShort()) { + return row.getDecimal(field).unscaledValue().longValue(); + } + else { + throw new IllegalStateException("getLong not supported for long decimal: " + type); + } + } + else { + throw new IllegalStateException("getLong not implemented for " + type); + } + } + + public static boolean getBoolean(Type type, RowResult row, int field) + { + if (type == BooleanType.BOOLEAN) { + return row.getBoolean(field); + } + else { + throw new IllegalStateException("getBoolean not implemented for " + type); + } + } + + public static double getDouble(Type type, RowResult row, int field) + { + if (type == DoubleType.DOUBLE) { + return row.getDouble(field); + } + else { + throw new IllegalStateException("getDouble not implemented for " + type); + } + } + + public static Slice getSlice(Type type, RowResult row, int field) + { + if (type instanceof VarcharType) { + return Slices.utf8Slice(row.getString(field)); + } + else if (type instanceof VarbinaryType) { + return Slices.wrappedBuffer(row.getBinary(field)); + } + else if (type instanceof DecimalType) { + BigDecimal dec = row.getDecimal(field); + return Decimals.encodeScaledValue(dec); + } + else { + throw new IllegalStateException("getSlice not implemented for " + type); + } + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/Types.java b/presto-kudu/src/main/java/io/presto/kudu/Types.java new file mode 100644 index 000000000..426cdbdfd --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/Types.java @@ -0,0 +1,34 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import java.util.Locale; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class Types +{ + private Types() + { + } + + public static B checkType(A value, Class target, String name) + { + requireNonNull(value, String.format(Locale.ENGLISH, "%s is null", name)); + checkArgument(target.isInstance(value), "%s must be of type %s, not %s", name, target.getName(), + value.getClass().getName()); + return target.cast(value); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/procedures/RangePartitionProcedures.java b/presto-kudu/src/main/java/io/presto/kudu/procedures/RangePartitionProcedures.java new file mode 100644 index 000000000..590f71076 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/procedures/RangePartitionProcedures.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.procedures; + +import com.google.common.collect.ImmutableList; +import io.presto.kudu.KuduClientSession; +import io.presto.kudu.properties.KuduTableProperties; +import io.presto.kudu.properties.RangePartition; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.procedure.Procedure; + +import javax.inject.Inject; + +import java.lang.invoke.MethodHandle; + +import static io.prestosql.spi.block.MethodHandleUtil.methodHandle; +import static io.prestosql.spi.type.StandardTypes.VARCHAR; +import static java.util.Objects.requireNonNull; + +public class RangePartitionProcedures +{ + private static final MethodHandle ADD = methodHandle(RangePartitionProcedures.class, "addRangePartition", + String.class, String.class, String.class); + private static final MethodHandle DROP = methodHandle(RangePartitionProcedures.class, "dropRangePartition", + String.class, String.class, String.class); + + private final KuduClientSession clientSession; + + @Inject + public RangePartitionProcedures(KuduClientSession clientSession) + { + this.clientSession = requireNonNull(clientSession); + } + + public Procedure getAddPartitionProcedure() + { + return new Procedure( + "system", + "add_range_partition", + ImmutableList.of(new Procedure.Argument("schema", VARCHAR), new Procedure.Argument("table", VARCHAR), + new Procedure.Argument("range_bounds", VARCHAR)), + ADD.bindTo(this)); + } + + public Procedure getDropPartitionProcedure() + { + return new Procedure( + "system", + "drop_range_partition", + ImmutableList.of(new Procedure.Argument("schema", VARCHAR), new Procedure.Argument("table", VARCHAR), + new Procedure.Argument("range_bounds", VARCHAR)), + DROP.bindTo(this)); + } + + public void addRangePartition(String schema, String table, String rangeBounds) + { + SchemaTableName schemaTableName = new SchemaTableName(schema, table); + RangePartition rangePartition = KuduTableProperties.parseRangePartition(rangeBounds); + clientSession.addRangePartition(schemaTableName, rangePartition); + } + + public void dropRangePartition(String schema, String table, String rangeBounds) + { + SchemaTableName schemaTableName = new SchemaTableName(schema, table); + RangePartition rangePartition = KuduTableProperties.parseRangePartition(rangeBounds); + clientSession.dropRangePartition(schemaTableName, rangePartition); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/ColumnDesign.java b/presto-kudu/src/main/java/io/presto/kudu/properties/ColumnDesign.java new file mode 100644 index 000000000..f392c0d7b --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/ColumnDesign.java @@ -0,0 +1,71 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +public class ColumnDesign +{ + public static final ColumnDesign DEFAULT; + + static { + ColumnDesign design = new ColumnDesign(); + design.setNullable(true); + DEFAULT = design; + } + + private boolean key; + private boolean nullable; + private String encoding; + private String compression; + + public boolean isKey() + { + return key; + } + + public void setKey(boolean key) + { + this.key = key; + } + + public String getEncoding() + { + return encoding; + } + + public void setEncoding(String encoding) + { + this.encoding = encoding; + } + + public String getCompression() + { + return compression; + } + + public void setCompression(String compression) + { + this.compression = compression; + } + + public boolean isNullable() + { + return nullable; + } + + public void setNullable(boolean nullable) + { + this.nullable = nullable; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/HashPartitionDefinition.java b/presto-kudu/src/main/java/io/presto/kudu/properties/HashPartitionDefinition.java new file mode 100644 index 000000000..73903dd44 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/HashPartitionDefinition.java @@ -0,0 +1,43 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +import java.util.List; + +public class HashPartitionDefinition +{ + private List columns; + private int buckets; + + public List getColumns() + { + return columns; + } + + public void setColumns(List columns) + { + this.columns = columns; + } + + public int getBuckets() + { + return buckets; + } + + public void setBuckets(int buckets) + { + this.buckets = buckets; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/KuduTableProperties.java b/presto-kudu/src/main/java/io/presto/kudu/properties/KuduTableProperties.java new file mode 100644 index 000000000..6c0b22cdd --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/KuduTableProperties.java @@ -0,0 +1,499 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.session.PropertyMetadata; +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Schema; +import org.apache.kudu.Type; +import org.apache.kudu.client.KeyEncoderAccessor; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.LocatedTablet; +import org.apache.kudu.client.PartialRow; +import org.apache.kudu.client.Partition; +import org.apache.kudu.client.PartitionSchema; +import org.apache.kudu.shaded.com.google.common.base.Predicates; +import org.apache.kudu.shaded.com.google.common.collect.Iterators; +import org.joda.time.DateTimeZone; +import org.joda.time.format.ISODateTimeFormat; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Base64; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.prestosql.spi.session.PropertyMetadata.integerProperty; +import static io.prestosql.spi.session.PropertyMetadata.stringProperty; +import static java.util.Objects.requireNonNull; + +public final class KuduTableProperties +{ + public static final String COLUMN_DESIGN = "column_design"; + public static final String PARTITION_DESIGN = "partition_design"; + public static final String RANGE_PARTITIONS = "range_partitions"; + public static final String NUM_REPLICAS = "num_replicas"; + + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final long DEFAULT_DEADLINE = 20000; // deadline for retrieving range partitions in milliseconds + + private final List> tableProperties; + + public KuduTableProperties() + { + PropertyMetadata s1 = stringProperty( + COLUMN_DESIGN, + "Kudu-specific column design (key, encoding, and compression) as JSON, like {\"column1\": {\"key\": true, \"encoding\": \"dictionary\", \"compression\": \"LZ4\"}, \"column2\": {...}}", + null, + false); + + PropertyMetadata s2 = stringProperty( + PARTITION_DESIGN, + "Partition design (hash partition(s) and/or range partition) as JSON.", + null, + false); + + PropertyMetadata s3 = integerProperty( + NUM_REPLICAS, + "Number of tablet replicas. Default 3.", + 3, + false); + + PropertyMetadata s4 = stringProperty( + RANGE_PARTITIONS, + "Initial range partitions as JSON", + null, + false); + + tableProperties = ImmutableList.of(s1, s2, s3, s4); + } + + public List> getTableProperties() + { + return tableProperties; + } + + /** + * Gets the value of the column_design property, or Optional.empty() if not set. + * + * @param tableProperties The map of table properties + * @return The column design settings + */ + public static Optional> getColumnDesign( + Map tableProperties) + { + requireNonNull(tableProperties); + + @SuppressWarnings("unchecked") + String json = (String) tableProperties.get(COLUMN_DESIGN); + if (json == null) { + return Optional.empty(); + } + + try { + Map map = mapper.readValue(json, new TypeReference>() { + }); + return Optional.of(map); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static Optional getPartitionDesign(Map tableProperties) + { + requireNonNull(tableProperties); + + String json = (String) tableProperties.get(PARTITION_DESIGN); + if (json != null) { + try { + PartitionDesign design = mapper.readValue(json, PartitionDesign.class); + return Optional.of(design); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + else { + return Optional.empty(); + } + } + + public static List getRangePartitions(Map tableProperties) + { + requireNonNull(tableProperties); + + @SuppressWarnings("unchecked") + String json = (String) tableProperties.get(RANGE_PARTITIONS); + if (json != null) { + try { + RangePartition[] partitions = mapper.readValue(json, RangePartition[].class); + if (partitions == null) { + return ImmutableList.of(); + } + else { + return ImmutableList.copyOf(partitions); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + else { + return ImmutableList.of(); + } + } + + public static RangePartition parseRangePartition(String json) + { + if (json == null) { + return null; + } + else { + try { + return mapper.readValue(json, RangePartition.class); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + public static Optional getNumReplicas(Map tableProperties) + { + requireNonNull(tableProperties); + + @SuppressWarnings("unchecked") + Integer numReplicas = (Integer) tableProperties.get(NUM_REPLICAS); + return Optional.ofNullable(numReplicas); + } + + public static Map toMap(KuduTable table) + { + Map properties = new HashMap<>(); + + LinkedHashMap columns = getColumns(table); + + PartitionDesign partitionDesign = getPartitionDesign(table); + + List rangePartitionList = getRangePartitionList(table, DEFAULT_DEADLINE); + + try { + String columnDesignValue = mapper.writeValueAsString(columns); + properties.put(COLUMN_DESIGN, columnDesignValue); + + String partitionDesignValue = mapper.writeValueAsString(partitionDesign); + properties.put(PARTITION_DESIGN, partitionDesignValue); + + String partitionRangesValue = mapper.writeValueAsString(rangePartitionList); + properties.put(RANGE_PARTITIONS, partitionRangesValue); + + properties.put(NUM_REPLICAS, table.getNumReplicas()); + + return properties; + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static List getRangePartitionList(KuduTable table, long deadline) + { + List rangePartitions = new ArrayList(); + if (!table.getPartitionSchema().getRangeSchema().getColumns().isEmpty()) { + try { + Iterator var4 = table.getTabletsLocations(deadline).iterator(); + + while (var4.hasNext()) { + LocatedTablet tablet = (LocatedTablet) var4.next(); + Partition partition = tablet.getPartition(); + if (Iterators.all(partition.getHashBuckets().iterator(), Predicates.equalTo(0))) { + RangePartition rangePartition = buildRangePartition(table, partition); + rangePartitions.add(rangePartition); + } + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + return rangePartitions; + } + + private static RangePartition buildRangePartition(KuduTable table, Partition partition) throws Exception + { + RangePartition rangePartition = new RangePartition(); + + RangeBoundValue lower = buildRangePartitionBound(table, partition.getRangeKeyStart()); + RangeBoundValue upper = buildRangePartitionBound(table, partition.getRangeKeyEnd()); + rangePartition.setLower(lower); + rangePartition.setUpper(upper); + + return rangePartition; + } + + private static RangeBoundValue buildRangePartitionBound(KuduTable table, byte[] rangeKey) throws Exception + { + if (rangeKey.length == 0) { + return null; + } + else { + Schema schema = table.getSchema(); + PartitionSchema partitionSchema = table.getPartitionSchema(); + PartitionSchema.RangeSchema rangeSchema = partitionSchema.getRangeSchema(); + List rangeColumns = rangeSchema.getColumns(); + + final int numColumns = rangeColumns.size(); + + PartialRow bound = KeyEncoderAccessor.decodeRangePartitionKey(schema, partitionSchema, rangeKey); + + RangeBoundValue value = new RangeBoundValue(); + ArrayList list = new ArrayList<>(); + for (int i = 0; i < numColumns; i++) { + Object obj = toValue(schema, bound, rangeColumns.get(i)); + list.add(obj); + } + value.setValues(list); + return value; + } + } + + private static Object toValue(Schema schema, PartialRow bound, Integer idx) + { + Type type = schema.getColumnByIndex(idx).getType(); + switch (type) { + case UNIXTIME_MICROS: + long millis = bound.getLong(idx) / 1000; + return ISODateTimeFormat.dateTime().withZone(DateTimeZone.UTC).print(millis); + case STRING: + return bound.getString(idx); + case INT64: + return bound.getLong(idx); + case INT32: + return bound.getInt(idx); + case INT16: + return bound.getShort(idx); + case INT8: + short s = bound.getByte(idx); + return s; + case BOOL: + return bound.getBoolean(idx); + case BINARY: + return bound.getBinaryCopy(idx); + default: + throw new IllegalStateException("Unhandled type " + type + " for range partition"); + } + } + + private static LinkedHashMap getColumns(KuduTable table) + { + Schema schema = table.getSchema(); + LinkedHashMap columns = new LinkedHashMap<>(); + for (ColumnSchema columnSchema : schema.getColumns()) { + ColumnDesign design = new ColumnDesign(); + design.setNullable(columnSchema.isNullable()); + design.setKey(columnSchema.isKey()); + if (columnSchema.getCompressionAlgorithm() != null) { + design.setCompression(columnSchema.getCompressionAlgorithm().name()); + } + if (columnSchema.getEncoding() != null) { + design.setEncoding(columnSchema.getEncoding().name()); + } + columns.put(columnSchema.getName(), design); + } + return columns; + } + + public static PartitionDesign getPartitionDesign(KuduTable table) + { + Schema schema = table.getSchema(); + PartitionDesign partitionDesign = new PartitionDesign(); + PartitionSchema partitionSchema = table.getPartitionSchema(); + + List hashPartitions = partitionSchema.getHashBucketSchemas().stream() + .map(hashBucketSchema -> { + HashPartitionDefinition hash = new HashPartitionDefinition(); + List cols = hashBucketSchema.getColumnIds().stream() + .map(idx -> schema.getColumnByIndex(idx).getName()).collect(toImmutableList()); + hash.setColumns(cols); + hash.setBuckets(hashBucketSchema.getNumBuckets()); + return hash; + }).collect(toImmutableList()); + partitionDesign.setHash(hashPartitions); + + List rangeColumns = partitionSchema.getRangeSchema().getColumns(); + if (!rangeColumns.isEmpty()) { + RangePartitionDefinition definition = new RangePartitionDefinition(); + definition.setColumns(rangeColumns.stream() + .map(i -> schema.getColumns().get(i).getName()) + .collect(ImmutableList.toImmutableList())); + partitionDesign.setRange(definition); + } + + return partitionDesign; + } + + public static PartialRow toRangeBoundToPartialRow(Schema schema, RangePartitionDefinition definition, + RangeBoundValue boundValue) + { + PartialRow partialRow = new PartialRow(schema); + if (boundValue != null) { + List rangeColumns = definition.getColumns().stream() + .map(name -> schema.getColumnIndex(name)).collect(toImmutableList()); + + if (rangeColumns.size() != boundValue.getValues().size()) { + throw new IllegalStateException("Expected " + rangeColumns.size() + + " range columns, but got " + boundValue.getValues().size()); + } + for (int i = 0; i < rangeColumns.size(); i++) { + Object obj = boundValue.getValues().get(i); + int idx = rangeColumns.get(i); + ColumnSchema columnSchema = schema.getColumnByIndex(idx); + setColumnValue(partialRow, idx, obj, columnSchema.getType(), columnSchema.getName()); + } + } + return partialRow; + } + + private static void setColumnValue(PartialRow partialRow, int idx, Object obj, Type type, String name) + { + Number n; + switch (type) { + case STRING: + if (obj instanceof String) { + partialRow.addString(idx, (String) obj); + } + else { + handleInvalidValue(name, type, obj); + } + break; + case INT64: + n = toNumber(obj, type, name); + partialRow.addLong(idx, n.longValue()); + break; + case INT32: + n = toNumber(obj, type, name); + partialRow.addInt(idx, n.intValue()); + break; + case INT16: + n = toNumber(obj, type, name); + partialRow.addShort(idx, n.shortValue()); + break; + case INT8: + n = toNumber(obj, type, name); + partialRow.addByte(idx, n.byteValue()); + break; + case DOUBLE: + n = toNumber(obj, type, name); + partialRow.addDouble(idx, n.doubleValue()); + break; + case FLOAT: + n = toNumber(obj, type, name); + partialRow.addFloat(idx, n.floatValue()); + break; + case UNIXTIME_MICROS: + long l = toUnixTimeMicros(obj, type, name); + partialRow.addLong(idx, l); + break; + case BOOL: + boolean b = toBoolean(obj, type, name); + partialRow.addBoolean(idx, b); + break; + case BINARY: + byte[] bytes = toByteArray(obj, type, name); + partialRow.addBinary(idx, bytes); + break; + default: + handleInvalidValue(name, type, obj); + break; + } + } + + private static byte[] toByteArray(Object obj, Type type, String name) + { + if (obj instanceof byte[]) { + return (byte[]) obj; + } + else if (obj instanceof String) { + return Base64.getDecoder().decode((String) obj); + } + else { + handleInvalidValue(name, type, obj); + return null; + } + } + + private static boolean toBoolean(Object obj, Type type, String name) + { + if (obj instanceof Boolean) { + return (Boolean) obj; + } + else if (obj instanceof String) { + return Boolean.valueOf((String) obj); + } + else { + handleInvalidValue(name, type, obj); + return false; + } + } + + private static long toUnixTimeMicros(Object obj, Type type, String name) + { + if (Number.class.isAssignableFrom(obj.getClass())) { + return ((Number) obj).longValue(); + } + else if (obj instanceof String) { + String s = (String) obj; + s = s.trim().replace(' ', 'T'); + long millis = ISODateTimeFormat.dateOptionalTimeParser().withZone(DateTimeZone.UTC).parseMillis(s); + return millis * 1000; + } + else { + handleInvalidValue(name, type, obj); + return 0; + } + } + + private static Number toNumber(Object obj, Type type, String name) + { + if (Number.class.isAssignableFrom(obj.getClass())) { + return (Number) obj; + } + else if (obj instanceof String) { + String s = (String) obj; + BigDecimal d = new BigDecimal((String) obj); + return d; + } + else { + handleInvalidValue(name, type, obj); + return 0; + } + } + + private static void handleInvalidValue(String name, Type type, Object obj) + { + throw new IllegalStateException("Invalid value " + obj + " for column " + name + " of type " + type); + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/PartitionDesign.java b/presto-kudu/src/main/java/io/presto/kudu/properties/PartitionDesign.java new file mode 100644 index 000000000..419d2330a --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/PartitionDesign.java @@ -0,0 +1,43 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +import java.util.List; + +public class PartitionDesign +{ + private List hash; + private RangePartitionDefinition range; + + public List getHash() + { + return hash; + } + + public void setHash(List hash) + { + this.hash = hash; + } + + public RangePartitionDefinition getRange() + { + return range; + } + + public void setRange(RangePartitionDefinition range) + { + this.range = range; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValue.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValue.java new file mode 100644 index 000000000..2428f1651 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValue.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; + +import java.util.ArrayList; +import java.util.List; + +@JsonDeserialize(using = RangeBoundValueDeserializer.class) +@JsonSerialize(using = RangeBoundValueSerializer.class) +public class RangeBoundValue +{ + private List values; + + public List getValues() + { + return values; + } + + public void setValues(List values) + { + List list = new ArrayList<>(); + for (int i = 0; i < values.size(); i++) { + list.add(values.get(i)); + } + this.values = list; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueDeserializer.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueDeserializer.java new file mode 100644 index 000000000..48dba0fb2 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueDeserializer.java @@ -0,0 +1,75 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; + +public class RangeBoundValueDeserializer + extends JsonDeserializer +{ + @Override + public RangeBoundValue deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException + { + JsonNode node = jp.getCodec().readTree(jp); + if (node.isNull()) { + return null; + } + else { + RangeBoundValue value = new RangeBoundValue(); + if (node.isArray()) { + ArrayList list = new ArrayList<>(); + Iterator iter = node.elements(); + while (iter.hasNext()) { + Object v = toValue(iter.next()); + list.add(v); + } + value.setValues(ImmutableList.copyOf(list)); + } + else { + Object v = toValue(node); + value.setValues(ImmutableList.of(v)); + } + return value; + } + } + + private Object toValue(JsonNode node) throws IOException + { + if (node.isTextual()) { + return node.asText(); + } + else if (node.isNumber()) { + return node.numberValue(); + } + else if (node.isBoolean()) { + return node.asBoolean(); + } + else if (node.isBinary()) { + return node.binaryValue(); + } + else { + throw new IllegalStateException("Unexpected range bound value: " + node); + } + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueSerializer.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueSerializer.java new file mode 100644 index 000000000..47bff28fb --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueSerializer.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; + +public class RangeBoundValueSerializer + extends JsonSerializer +{ + @Override + public void serialize(Object value, JsonGenerator gen, SerializerProvider serializers) + throws IOException + { + if (value == null) { + gen.writeNull(); + } + else { + RangeBoundValue rbv = (RangeBoundValue) value; + if (rbv.getValues().size() == 1) { + writeValue(rbv.getValues().get(0), gen); + } + else { + gen.writeStartArray(rbv.getValues().size()); + for (Object obj : rbv.getValues()) { + writeValue(obj, gen); + } + gen.writeEndArray(); + } + } + } + + private void writeValue(Object obj, JsonGenerator gen) throws IOException + { + if (obj == null) { + throw new IllegalStateException("Unexpected null value"); + } + else if (obj instanceof String) { + gen.writeString((String) obj); + } + else if (Number.class.isAssignableFrom(obj.getClass())) { + if (obj instanceof Long) { + gen.writeNumber((Long) obj); + } + else if (obj instanceof Integer) { + gen.writeNumber((Integer) obj); + } + else if (obj instanceof Short) { + gen.writeNumber((Short) obj); + } + else if (obj instanceof Double) { + gen.writeNumber((Double) obj); + } + else if (obj instanceof Float) { + gen.writeNumber((Float) obj); + } + else if (obj instanceof BigInteger) { + gen.writeNumber((BigInteger) obj); + } + else if (obj instanceof BigDecimal) { + gen.writeNumber((BigDecimal) obj); + } + else { + throw new IllegalStateException("Unknown number value: " + obj); + } + } + else if (obj instanceof Boolean) { + gen.writeBoolean((Boolean) obj); + } + else if (obj instanceof byte[]) { + gen.writeBinary((byte[]) obj); + } + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartition.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartition.java new file mode 100644 index 000000000..d41b36039 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartition.java @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +public class RangePartition +{ + private RangeBoundValue lower; + private RangeBoundValue upper; + + public RangeBoundValue getLower() + { + return lower; + } + + public void setLower(RangeBoundValue lower) + { + this.lower = lower; + } + + public RangeBoundValue getUpper() + { + return upper; + } + + public void setUpper(RangeBoundValue upper) + { + this.upper = upper; + } +} diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartitionDefinition.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartitionDefinition.java new file mode 100644 index 000000000..b2c454608 --- /dev/null +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartitionDefinition.java @@ -0,0 +1,32 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +import java.util.List; + +public class RangePartitionDefinition +{ + private List columns; + + public List getColumns() + { + return columns; + } + + public void setColumns(List columns) + { + this.columns = columns; + } +} diff --git a/presto-kudu/src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java b/presto-kudu/src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java new file mode 100644 index 000000000..e6dbdb1df --- /dev/null +++ b/presto-kudu/src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kudu.client; + +import org.apache.kudu.Schema; + +/** + * Little wrapper to access KeyEncoder in Kudu Java client. + */ +public class KeyEncoderAccessor +{ + private KeyEncoderAccessor() + { + } + + public static byte[] encodePrimaryKey(PartialRow row) + { + return KeyEncoder.encodePrimaryKey(row); + } + + public static PartialRow decodePrimaryKey(Schema schema, byte[] key) + { + return KeyEncoder.decodePrimaryKey(schema, key); + } + + public static byte[] encodeRangePartitionKey(PartialRow row, PartitionSchema.RangeSchema rangeSchema) + { + return KeyEncoder.encodeRangePartitionKey(row, rangeSchema); + } + + public static PartialRow decodeRangePartitionKey(Schema schema, PartitionSchema partitionSchema, byte[] key) + { + return KeyEncoder.decodeRangePartitionKey(schema, partitionSchema, key); + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/KuduQueryRunnerFactory.java b/presto-kudu/src/test/java/io/presto/kudu/KuduQueryRunnerFactory.java new file mode 100644 index 000000000..58ef0a229 --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/KuduQueryRunnerFactory.java @@ -0,0 +1,106 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.tpch.TpchTable; +import io.prestosql.Session; +import io.prestosql.plugin.tpch.TpchPlugin; +import io.prestosql.testing.QueryRunner; +import io.prestosql.tests.DistributedQueryRunner; + +import java.util.Map; + +import static io.airlift.testing.Closeables.closeAllSuppress; +import static io.prestosql.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.prestosql.spi.type.TimeZoneKey.UTC_KEY; +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static io.prestosql.tests.QueryAssertions.copyTpchTables; +import static java.util.Locale.ENGLISH; + +public class KuduQueryRunnerFactory +{ + private KuduQueryRunnerFactory() {} + + public static QueryRunner createKuduQueryRunner(String schema) + throws Exception + { + QueryRunner runner = null; + String kuduSchema = schema; + try { + runner = DistributedQueryRunner.builder(createSession(kuduSchema)).setNodeCount(3).build(); + + installKuduConnector(runner, kuduSchema); + + return runner; + } + catch (Throwable e) { + closeAllSuppress(e, runner); + throw e; + } + } + + public static QueryRunner createKuduQueryRunnerTpch(TpchTable... tables) + throws Exception + { + return createKuduQueryRunnerTpch(ImmutableList.copyOf(tables)); + } + + public static QueryRunner createKuduQueryRunnerTpch(Iterable> tables) + throws Exception + { + DistributedQueryRunner runner = null; + String kuduSchema = "tpch"; + try { + runner = DistributedQueryRunner.builder(createSession(kuduSchema)).setNodeCount(3).build(); + + runner.installPlugin(new TpchPlugin()); + runner.createCatalog("tpch", "tpch"); + + installKuduConnector(runner, kuduSchema); + + copyTpchTables(runner, "tpch", TINY_SCHEMA_NAME, createSession(kuduSchema), tables); + + return runner; + } + catch (Throwable e) { + closeAllSuppress(e, runner); + throw e; + } + } + + private static void installKuduConnector(QueryRunner runner, String schema) + { + String masterAddresses = System.getProperty("kudu.client.master-addresses", "localhost:7051"); + Map properties = ImmutableMap.of( + "kudu.client.master-addresses", masterAddresses); + + runner.installPlugin(new KuduPlugin()); + runner.createCatalog("kudu", "kudu", properties); + + runner.execute("DROP SCHEMA IF EXISTS " + schema); + runner.execute("CREATE SCHEMA " + schema); + } + + public static Session createSession(String schema) + { + return testSessionBuilder() + .setCatalog("kudu") + .setSchema(schema) + .setTimeZoneKey(UTC_KEY) + .setLocale(ENGLISH) + .build(); + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestDec.java b/presto-kudu/src/test/java/io/presto/kudu/TestDec.java new file mode 100644 index 000000000..7b153261b --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestDec.java @@ -0,0 +1,27 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +public class TestDec +{ + final int precision; + final int scale; + + TestDec(int precision, int scale) + { + this.precision = precision; + this.scale = scale; + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestDecimalColumnsTest.java b/presto-kudu/src/test/java/io/presto/kudu/TestDecimalColumnsTest.java new file mode 100644 index 000000000..683a86316 --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestDecimalColumnsTest.java @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.testing.MaterializedResult; +import io.prestosql.testing.QueryRunner; +import io.prestosql.tests.AbstractTestQueryFramework; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class TestDecimalColumnsTest + extends AbstractTestQueryFramework +{ + private QueryRunner queryRunner; + + static final TestDec[] testDecList = { + new TestDec(10, 0), + new TestDec(15, 4), + new TestDec(18, 6), + new TestDec(18, 7), + new TestDec(19, 8), + new TestDec(24, 14), + new TestDec(38, 20), + new TestDec(38, 28), + }; + + public TestDecimalColumnsTest() throws Exception + { + super(() -> KuduQueryRunnerFactory.createKuduQueryRunner("decimal")); + } + + @Test + public void testCreateTableWithDecimalColumn() + { + for (TestDec dec : testDecList) { + doTestCreateTableWithDecimalColumn(dec); + } + } + + private void doTestCreateTableWithDecimalColumn(TestDec dec) + { + String dropTable = "DROP TABLE IF EXISTS test_dec"; + String createTable = "CREATE TABLE test_dec (\n"; + createTable += " id INT,\n"; + createTable += " dec DECIMAL(" + dec.precision + "," + dec.scale + ")\n"; + createTable += ") WITH (\n" + + " column_design = '{\"id\": {\"key\": true}}',\n" + + " partition_design = '{\"hash\":[{\"columns\":[\"id\"], \"buckets\": 2}]}',\n" + + " num_replicas = 1\n" + + ")"; + + queryRunner.execute(dropTable); + queryRunner.execute(createTable); + + String fullPrecisionValue = "1234567890.1234567890123456789012345678"; + int maxScale = dec.precision - 10; + int valuePrecision = dec.precision - maxScale + Math.min(maxScale, dec.scale); + String insertValue = fullPrecisionValue.substring(0, valuePrecision + 1); + queryRunner.execute("INSERT INTO test_dec VALUES(1, DECIMAL '" + insertValue + "')"); + + MaterializedResult result = queryRunner.execute("SELECT id, CAST((dec - (DECIMAL '" + insertValue + "')) as DOUBLE) FROM test_dec"); + assertEquals(result.getRowCount(), 1); + Object obj = result.getMaterializedRows().get(0).getField(1); + assertTrue(obj instanceof Double); + Double actual = (Double) obj; + assertEquals(0, actual, 0.3 * Math.pow(0.1, dec.scale), "p=" + dec.precision + ",s=" + dec.scale + " => " + actual + ",insert = " + insertValue); + } + + @BeforeClass + public void setUp() + { + queryRunner = getQueryRunner(); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + if (queryRunner != null) { + queryRunner.close(); + queryRunner = null; + } + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestInt.java b/presto-kudu/src/test/java/io/presto/kudu/TestInt.java new file mode 100644 index 000000000..7700db6c3 --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestInt.java @@ -0,0 +1,27 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +public class TestInt +{ + final String type; + final int bits; + + TestInt(String type, int bits) + { + this.type = type; + this.bits = bits; + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestIntegerColumnsTest.java b/presto-kudu/src/test/java/io/presto/kudu/TestIntegerColumnsTest.java new file mode 100644 index 000000000..8b4acbf6c --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestIntegerColumnsTest.java @@ -0,0 +1,108 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.testing.MaterializedResult; +import io.prestosql.testing.QueryRunner; +import io.prestosql.testing.assertions.Assert; +import io.prestosql.tests.AbstractTestQueryFramework; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class TestIntegerColumnsTest + extends AbstractTestQueryFramework +{ + private QueryRunner queryRunner; + + static final TestInt[] testList = { + new TestInt("TINYINT", 8), + new TestInt("SMALLINT", 16), + new TestInt("INTEGER", 32), + new TestInt("BIGINT", 64), + }; + + public TestIntegerColumnsTest() throws Exception + { + super(() -> KuduQueryRunnerFactory.createKuduQueryRunner("test_integer")); + } + + @Test + public void testCreateTableWithIntegerColumn() + { + for (TestInt test : testList) { + doTestCreateTableWithIntegerColumn(test); + } + } + + private void doTestCreateTableWithIntegerColumn(TestInt test) + { + String dropTable = "DROP TABLE IF EXISTS test_int"; + String createTable = "CREATE TABLE test_int (\n"; + createTable += " id INT,\n"; + createTable += " intcol " + test.type + "\n"; + createTable += ") WITH (\n" + + " column_design = '{\"id\": {\"key\": true}}',\n" + + " num_replicas = 1\n" + + ")"; + + queryRunner.execute(dropTable); + queryRunner.execute(createTable); + + long maxValue = Long.MAX_VALUE; + long casted = maxValue >> (64 - test.bits); + queryRunner.execute("INSERT INTO test_int VALUES(1, CAST(" + casted + " AS " + test.type + "))"); + + MaterializedResult result = queryRunner.execute("SELECT id, intcol FROM test_int"); + Assert.assertEquals(result.getRowCount(), 1); + Object obj = result.getMaterializedRows().get(0).getField(1); + switch (test.bits) { + case 64: + Assert.assertTrue(obj instanceof Long); + Assert.assertEquals(((Long) obj).longValue(), casted); + break; + case 32: + Assert.assertTrue(obj instanceof Integer); + Assert.assertEquals(((Integer) obj).longValue(), casted); + break; + case 16: + Assert.assertTrue(obj instanceof Short); + Assert.assertEquals(((Short) obj).longValue(), casted); + break; + case 8: + Assert.assertTrue(obj instanceof Byte); + Assert.assertEquals(((Byte) obj).longValue(), casted); + break; + default: + Assert.fail("Unexpected bits: " + test.bits); + break; + } + } + + @BeforeClass + public void setUp() + { + queryRunner = getQueryRunner(); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + if (queryRunner != null) { + queryRunner.close(); + queryRunner = null; + } + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestKuduIntegrationSmokeTest.java b/presto-kudu/src/test/java/io/presto/kudu/TestKuduIntegrationSmokeTest.java new file mode 100644 index 000000000..de8f09a3d --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestKuduIntegrationSmokeTest.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.spi.type.VarcharType; +import io.prestosql.testing.MaterializedResult; +import io.prestosql.testing.MaterializedRow; +import io.prestosql.testing.QueryRunner; +import io.prestosql.tests.AbstractTestIntegrationSmokeTest; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static io.airlift.tpch.TpchTable.ORDERS; +import static io.prestosql.testing.assertions.Assert.assertEquals; + +/** + * Kudu master server is expected to be running on localhost. At least one + * Kudu tablet server must be running, too. + * With Docker, use e.g. + * "docker run --rm -d --name apache-kudu --net=host usuresearch/kudu-docker-slim:release-v1.6.0-2" + */ +public class TestKuduIntegrationSmokeTest + extends AbstractTestIntegrationSmokeTest +{ + public static final String SCHEMA = "tpch"; + + private QueryRunner queryRunner; + + public TestKuduIntegrationSmokeTest() + { + super(() -> KuduQueryRunnerFactory.createKuduQueryRunnerTpch(ORDERS)); + } + + @BeforeClass + public void setUp() + { + queryRunner = getQueryRunner(); + } + + /** + * Overrides original implementation because of usage of 'extra' column. + */ + @Test + @Override + public void testDescribeTable() + { + MaterializedResult actualColumns = this.computeActual("DESC ORDERS").toTestTypes(); + MaterializedResult.Builder builder = MaterializedResult.resultBuilder(this.getQueryRunner().getDefaultSession(), VarcharType.VARCHAR, VarcharType.VARCHAR, VarcharType.VARCHAR, VarcharType.VARCHAR); + for (MaterializedRow row : actualColumns.getMaterializedRows()) { + builder.row(row.getField(0), row.getField(1), "", ""); + } + MaterializedResult filteredActual = builder.build(); + builder = MaterializedResult.resultBuilder(this.getQueryRunner().getDefaultSession(), VarcharType.VARCHAR, VarcharType.VARCHAR, VarcharType.VARCHAR, VarcharType.VARCHAR); + MaterializedResult expectedColumns = builder + .row("orderkey", "bigint", "", "") + .row("custkey", "bigint", "", "") + .row("orderstatus", "varchar", "", "") + .row("totalprice", "double", "", "") + .row("orderdate", "varchar", "", "") + .row("orderpriority", "varchar", "", "") + .row("clerk", "varchar", "", "") + .row("shippriority", "integer", "", "") + .row("comment", "varchar", "", "").build(); + assertEquals(filteredActual, expectedColumns, String.format("%s != %s", filteredActual, expectedColumns)); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + if (queryRunner != null) { + queryRunner.close(); + queryRunner = null; + } + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestKuduPlugin.java b/presto-kudu/src/test/java/io/presto/kudu/TestKuduPlugin.java new file mode 100644 index 000000000..4812d2a5a --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestKuduPlugin.java @@ -0,0 +1,34 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import com.google.common.collect.ImmutableMap; +import io.prestosql.spi.Plugin; +import io.prestosql.spi.connector.ConnectorFactory; +import io.prestosql.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; + +public class TestKuduPlugin +{ + @Test + public void testCreateConnector() + throws Exception + { + Plugin plugin = new KuduPlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + factory.create("test", ImmutableMap.of("kudu.client.master-addresses", "localhost:8050"), new TestingConnectorContext()); + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestRangePartitioningTest.java b/presto-kudu/src/test/java/io/presto/kudu/TestRangePartitioningTest.java new file mode 100644 index 000000000..c57163ad6 --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestRangePartitioningTest.java @@ -0,0 +1,170 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.testing.MaterializedResult; +import io.prestosql.testing.QueryRunner; +import io.prestosql.tests.AbstractTestQueryFramework; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class TestRangePartitioningTest + extends AbstractTestQueryFramework +{ + private QueryRunner queryRunner; + + static final TestRanges[] testRangesList = { + new TestRanges("varchar", + "{\"lower\": null, \"upper\": \"D\"}", + "{\"lower\": \"D\", \"upper\": \"M\"}", + "{\"lower\": \"M\", \"upper\": \"S\"}", + "{\"lower\": \"S\", \"upper\": null}"), + new TestRanges("timestamp", + "{\"lower\": null, \"upper\": \"2017-01-01T02:03:04.567Z\"}", + "{\"lower\": \"2017-01-01 03:03:04.567+01:00\", \"upper\": \"2017-02-01 12:34\"}", + "{\"lower\": \"2017-02-01 12:34\", \"upper\": \"2017-03-01\"}", + "{\"lower\": \"2017-03-01\", \"upper\": null}", + "{\"lower\": null, \"upper\": \"2017-01-01T02:03:04.567Z\"}", + "{\"lower\": \"2017-01-01T02:03:04.567Z\", \"upper\": \"2017-02-01T12:34:00.000Z\"}", + "{\"lower\": \"2017-02-01T12:34:00.000Z\", \"upper\": \"2017-03-01T00:00:00.000Z\"}", + "{\"lower\": \"2017-03-01T00:00:00.000Z\", \"upper\": null}"), + new TestRanges("tinyint", + "{\"lower\": null, \"upper\": -10}", + "{\"lower\": \"-10\", \"upper\": 0}", + "{\"lower\": 0, \"upper\": 10}", + "{\"lower\": 10, \"upper\": 20}", + "{\"lower\": null, \"upper\": -10}", + "{\"lower\": -10, \"upper\": 0}", + "{\"lower\": 0, \"upper\": 10}", + "{\"lower\": 10, \"upper\": 20}"), + new TestRanges("smallint", + "{\"lower\": null, \"upper\": -1000}", + "{\"lower\": -1000, \"upper\": 0}", + "{\"lower\": 0, \"upper\": 1000}", + "{\"lower\": 1000, \"upper\": 2000}"), + new TestRanges("integer", + "{\"lower\": null, \"upper\": -1000000}", + "{\"lower\": -1000000, \"upper\": 0}", + "{\"lower\": 0, \"upper\": 10000}", + "{\"lower\": 10000, \"upper\": 1000000}"), + new TestRanges("bigint", + "{\"lower\": null, \"upper\": \"-123456789012345\"}", + "{\"lower\": \"-123456789012345\", \"upper\": 0}", + "{\"lower\": 0, \"upper\": 123400}", + "{\"lower\": 123400, \"upper\": 123456789012345}", + "{\"lower\": null, \"upper\": -123456789012345}", + "{\"lower\": -123456789012345, \"upper\": 0}", + "{\"lower\": 0, \"upper\": 123400}", + "{\"lower\": 123400, \"upper\": 123456789012345}"), + new TestRanges("varbinary", + "{\"lower\": null, \"upper\": \"YWI=\"}", + "{\"lower\": \"YWI=\", \"upper\": \"ZA==\"}", + "{\"lower\": \"ZA==\", \"upper\": \"bW1t\"}", + "{\"lower\": \"bW1t\", \"upper\": \"eg==\"}"), + new TestRanges(new String[]{"smallint", "varchar"}, + "{\"lower\": null, \"upper\": [1, \"M\"]}", + "{\"lower\": [1, \"M\"], \"upper\": [1, \"T\"]}", + "{\"lower\": [1, \"T\"], \"upper\": [2, \"Z\"]}", + "{\"lower\": [2, \"Z\"], \"upper\": null}"), + }; + + public TestRangePartitioningTest() + { + super(() -> KuduQueryRunnerFactory.createKuduQueryRunner("range_partitioning")); + } + + @Test + public void testCreateAndChangeTableWithRangePartition() + { + for (TestRanges ranges : testRangesList) { + doTestCreateAndChangeTableWithRangePartition(ranges); + } + } + + public void doTestCreateAndChangeTableWithRangePartition(TestRanges ranges) + { + String[] types = ranges.types; + String name = String.join("_", ranges.types); + String tableName = "range_partitioning_" + name; + String createTable = "CREATE TABLE " + tableName + " (\n"; + String columnDesign = ""; + String partitionDesign = ""; + for (int i = 0; i < types.length; i++) { + String type = types[i]; + String columnName = "key" + i; + createTable += " " + columnName + " " + type + ",\n"; + if (i == 0) { + columnDesign += "{"; + partitionDesign += "["; + } + else { + columnDesign += ","; + partitionDesign += ","; + } + columnDesign += "\"" + columnName + "\": {\"key\": true}"; + partitionDesign += "\"" + columnName + "\""; + } + columnDesign += "}"; + partitionDesign += "]"; + + createTable += + " value varchar\n" + + ") WITH (\n" + + " column_design = '" + columnDesign + "',\n" + + " partition_design = '{\"range\": {\"columns\":" + partitionDesign + "}}',\n" + + " range_partitions = '[" + ranges.range1 + "," + ranges.range2 + "]',\n" + + " num_replicas = 1\n" + + ")"; + queryRunner.execute(createTable); + + String schema = queryRunner.getDefaultSession().getSchema().get(); + String addPartition3 = "CALL kudu.system.add_range_partition('" + schema + "','" + tableName + "','" + ranges.range3 + "')"; + queryRunner.execute(addPartition3); + String addPartition4 = "CALL kudu.system.add_range_partition('" + schema + "','" + tableName + "','" + ranges.range4 + "')"; + queryRunner.execute(addPartition4); + + String dropPartition3 = addPartition3.replace(".add_range_partition(", ".drop_range_partition("); + queryRunner.execute(dropPartition3); + + MaterializedResult result = queryRunner.execute("SHOW CREATE TABLE " + tableName); + assertEquals(result.getRowCount(), 1); + String createSQL = result.getMaterializedRows().get(0).getField(0).toString(); + String rangesArray = "'[" + ranges.cmp1 + "," + ranges.cmp2 + "," + ranges.cmp4 + "]'"; + rangesArray = rangesArray.replaceAll("\\s+", ""); + String expectedRanges = "range_partitions = " + rangesArray; + assertTrue(createSQL.contains(expectedRanges), createSQL + "\ncontains\n" + expectedRanges); + String expectedNumReplicas = "num_replicas = 1,"; + assertTrue(createSQL.contains(expectedNumReplicas), createSQL + "\ncontains\n" + expectedNumReplicas); + } + + @BeforeClass + public void setUp() + { + queryRunner = getQueryRunner(); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + if (queryRunner != null) { + queryRunner.close(); + queryRunner = null; + } + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestRanges.java b/presto-kudu/src/test/java/io/presto/kudu/TestRanges.java new file mode 100644 index 000000000..f8dd1ccda --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestRanges.java @@ -0,0 +1,59 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +public class TestRanges +{ + final String[] types; + final String range1; + final String range2; + final String range3; + final String range4; + + final String cmp1; + final String cmp2; + final String cmp3; + final String cmp4; + + TestRanges(String type, String range1, String range2, String range3, String range4) + { + this(new String[]{type}, range1, range2, range3, range4, range1, range2, range3, range4); + } + + TestRanges(String type, String range1, String range2, String range3, String range4, + String cmp1, String cmp2, String cmp3, String cmp4) + { + this(new String[]{type}, range1, range2, range3, range4, cmp1, cmp2, cmp3, cmp4); + } + + TestRanges(String[] types, String range1, String range2, String range3, String range4) + { + this(types, range1, range2, range3, range4, range1, range2, range3, range4); + } + + TestRanges(String[] types, String range1, String range2, String range3, String range4, + String cmp1, String cmp2, String cmp3, String cmp4) + { + this.types = types; + this.range1 = range1; + this.range2 = range2; + this.range3 = range3; + this.range4 = range4; + this.cmp1 = cmp1; + this.cmp2 = cmp2; + this.cmp3 = cmp3; + this.cmp4 = cmp4; + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestSchemaNoExist.java b/presto-kudu/src/test/java/io/presto/kudu/TestSchemaNoExist.java new file mode 100644 index 000000000..aa67f04ca --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/TestSchemaNoExist.java @@ -0,0 +1,74 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu; + +import io.prestosql.testing.QueryRunner; +import io.prestosql.tests.AbstractTestQueryFramework; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; + +public class TestSchemaNoExist + extends AbstractTestQueryFramework +{ + private QueryRunner queryRunner; + + private static final String SCHEMA_NAME = "test_presto_schema"; + + private static final String DROP_SCHEMA = "drop schema if exists kudu." + SCHEMA_NAME; + + private static final String CREATE_TABLE = "create table if not exists kudu." + SCHEMA_NAME + ".test_presto_table " + + "(user_id int, user_name varchar) " + + "with(column_design = '{\"user_id\": {\"key\": true}}'," + + "partition_design = '{\"hash\":[{\"columns\":[\"user_id\"], \"buckets\": 2}]}'," + + "num_replicas = 1)"; + + private static final String DROP_TABLE = "drop table if exists kudu." + SCHEMA_NAME + ".test_presto_table"; + + public TestSchemaNoExist() + { + super(() -> KuduQueryRunnerFactory.createKuduQueryRunner("test_dummy")); + } + + @Test + public void testCreateTableWithoutSchema() + { + try { + queryRunner.execute(CREATE_TABLE); + fail(); + } + catch (Exception e) { + assertEquals("Schema " + SCHEMA_NAME + " not found", e.getMessage()); + } + } + + @BeforeClass + public void setUp() + { + queryRunner = getQueryRunner(); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + queryRunner.execute(DROP_TABLE); + queryRunner.execute(DROP_SCHEMA); + queryRunner.close(); + queryRunner = null; + } +} diff --git a/presto-kudu/src/test/java/io/presto/kudu/properties/RangePartitionSerializationTest.java b/presto-kudu/src/test/java/io/presto/kudu/properties/RangePartitionSerializationTest.java new file mode 100644 index 000000000..b538e9619 --- /dev/null +++ b/presto-kudu/src/test/java/io/presto/kudu/properties/RangePartitionSerializationTest.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.presto.kudu.properties; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.testng.annotations.Test; + +import java.io.IOException; + +import static org.testng.Assert.assertEquals; + +public class RangePartitionSerializationTest +{ + private String[] testInputs = new String[]{ + "{\"lower\":1,\"upper\":null}", + "{\"lower\":12345678901234567890,\"upper\":1.234567890123457E-13}", + "{\"lower\":\"abc\",\"upper\":\"abf\"}", + "{\"lower\":false,\"upper\":true}", + "{\"lower\":\"ABCD\",\"upper\":\"ABCDEF\"}", + "{\"lower\":[\"ABCD\",1,0],\"upper\":[\"ABCD\",13,0]}", + }; + + @Test + public void testDeserializationSerialization() throws IOException + { + ObjectMapper mapper = new ObjectMapper(); + + for (String input : testInputs) { + RangePartition partition = mapper.readValue(input, RangePartition.class); + + String serialized = mapper.writeValueAsString(partition); + assertEquals(serialized, input); + } + } +} diff --git a/presto-main/etc/catalog/kudu.properties b/presto-main/etc/catalog/kudu.properties new file mode 100644 index 000000000..d8eae312d --- /dev/null +++ b/presto-main/etc/catalog/kudu.properties @@ -0,0 +1,54 @@ +connector.name=kudu + +## List of Kudu master addresses, at least one is needed (comma separated) +## Supported formats: example.com, example.com:7051, 192.0.2.1, 192.0.2.1:7051, +## [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1 +kudu.client.master-addresses=localhost:7051 + +#if kudu with kerberos, enable this +is.kerberos.on = false + +#kerberos user for kudu with kerberos +kerberos.user= root@EXAMPLE.COM + +#keytab path +keytab.path = /root/root.keytab + +#the path of hdfs-site.xml is used for kerberos, only is.kerberos.on = true need it? +hdfs-site.xml.path = /root/hdfs-site.xml + +#the path of core-site.xml is used for kerberos, only is.kerberos.on = true need it? +core-site.xml.path = /root/core-site.xml + +#after the service started,the delay time of kerberos re-authentication,this is second time of re-authentication +kudu.reauth.delay.time = 60 + +#after service started,this configuration is re-authentication interval,This configuration is to prevent ticket expiration +kudu.reauth.period.time = 82800 + +## Optional restriction of tablets for specific tenant. +## If a tenant is set, only Kudu tablets starting with `.` will +## be visible in openLookeng +#kudu.session.tenant=root_db + +#If configuration is enabled,you can select database for kudu metadata table +kudu.metadata.on = false + +#for example, if kudu.metadata.on = true ,metadata table in kudu is root_db._schema +kudu.metadata.database = root_Db + +####################### +### Advanced Kudu Java client configuration +####################### + +## Default timeout used for administrative operations (e.g. createTable, deleteTable, etc.) +#kudu.client.defaultAdminOperationTimeout = 30s + +## Default timeout used for user operations +#kudu.client.defaultOperationTimeout = 30s + +## Default timeout to use when waiting on data from a socket +#kudu.client.defaultSocketReadTimeout = 10s + +## Disable Kudu client's collection of statistics. +#kudu.client.disableStatistics = false \ No newline at end of file -- Gitee From 6e3225420d35796a772fc1f5cde36d7648d34a3f Mon Sep 17 00:00:00 2001 From: Zhang Jianming Date: Mon, 31 Jul 2023 17:00:40 +0800 Subject: [PATCH 2/5] add kudu connector --- .../src/main/java/io/presto/kudu/KuduClientConfig.java | 7 ------- presto-kudu/src/main/java/io/presto/kudu/KuduModule.java | 3 --- .../src/main/java/io/presto/kudu/KuduRecordCursor.java | 3 ++- .../main/java/io/presto/kudu/NativeKuduClientSession.java | 5 ----- .../io/presto/kudu/properties/KuduTableProperties.java | 2 +- 5 files changed, 3 insertions(+), 17 deletions(-) diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java b/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java index 3613a9987..79b5efdbb 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java @@ -23,7 +23,6 @@ import io.prestosql.spi.function.Mandatory; import javax.validation.constraints.NotNull; import javax.validation.constraints.Size; -import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeUnit; @@ -127,12 +126,6 @@ public class KuduClientConfig return this; } - public KuduClientConfig setMasterAddresses(String... contactPoints) - { - this.masterAddresses = Arrays.asList(contactPoints); - return this; - } - @Config("kudu.client.defaultAdminOperationTimeout") public KuduClientConfig setDefaultAdminOperationTimeout(Duration timeout) { diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java b/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java index 39cd4c709..c4388b325 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java @@ -103,7 +103,6 @@ public class KuduModule reauth(config, connectorId); KuduClient client; try { - //String name = UserGroupInformation.getLoginUser().getUserName(); client = UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction() { @Override @@ -182,10 +181,8 @@ public class KuduModule return builder.build(); } }); - //log.info("client==+++=======" + client.toString()); KuduClient oldClient = nativeKuduClientSession.getClient(); nativeKuduClientSession.setClient(client); - //oldClient.close(); } catch (Exception e) { log.error(e, "failed to create kudu client"); diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java index 0bfefb258..cf9296f70 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java @@ -51,7 +51,8 @@ public class KuduRecordCursor field.setAccessible(true); } catch (NoSuchFieldException e) { - // ignore + log.error("no such field"); + e.printStackTrace(); } this.rowDataField = field; } diff --git a/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java b/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java index 7d1eb8721..5c4fd2923 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java +++ b/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java @@ -124,7 +124,6 @@ public class NativeKuduClientSession { synchronized (DEFAULT_SCHEMA) { try { - //log.info("client=========" + client.toString()); if (rawSchemasTable == null) { if (client.tableExists(rawSchemasTableName)) { deleteSchemaTables(session); @@ -143,13 +142,11 @@ public class NativeKuduClientSession rawSchemasTable = getSchemasTable(session); } } - //log.info("client=1========" + client.toString()); ColumnSchema tenantColumn = rawSchemasTable.getSchema().getColumnByIndex(0); KuduScanner scanner = client.newScannerBuilder(rawSchemasTable) .addPredicate(KuduPredicate.newComparisonPredicate(tenantColumn, KuduPredicate.ComparisonOp.EQUAL, tenantPrefix)) .setProjectedColumnIndexes(ImmutableList.of(1)) .build(); - //log.info("client=2========" + client.toString()); RowResultIterator iterator = scanner.nextRows(); ArrayList result = new ArrayList<>(); while (iterator != null) { @@ -175,9 +172,7 @@ public class NativeKuduClientSession private KuduTable getSchemasTable(ConnectorSession session) throws KuduException { - //if (rawSchemasTable == null) { rawSchemasTable = client.openTable(rawSchemasTableName); - //} return rawSchemasTable; } diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/KuduTableProperties.java b/presto-kudu/src/main/java/io/presto/kudu/properties/KuduTableProperties.java index 6c0b22cdd..52d8b7734 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/KuduTableProperties.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/KuduTableProperties.java @@ -441,7 +441,7 @@ public final class KuduTableProperties } else { handleInvalidValue(name, type, obj); - return null; + return new byte[0]; } } -- Gitee From 08e0477c5925378f46c77a940d2730e65ae6f330 Mon Sep 17 00:00:00 2001 From: Zhang Jianming Date: Mon, 18 Sep 2023 13:57:08 +0800 Subject: [PATCH 3/5] add kudu connector --- hetu-docs/en/connector/kudu.md | 60 ++++++++++--- hetu-docs/zh/connector/kudu.md | 58 ++++++++++--- .../java/io/presto/kudu/KuduClientConfig.java | 86 ++++++++----------- .../io/presto/kudu/KuduClientSession.java | 1 - .../java/io/presto/kudu/KuduColumnHandle.java | 1 - .../main/java/io/presto/kudu/KuduModule.java | 10 +-- .../java/io/presto/kudu/KuduPageSink.java | 1 - .../io/presto/kudu/KuduPageSinkProvider.java | 1 - .../java/io/presto/kudu/KuduRecordCursor.java | 1 + .../KuduRecordCursorWithVirtualRowId.java | 1 - .../presto/kudu/KuduUpdatablePageSource.java | 1 - .../presto/kudu/NativeKuduClientSession.java | 3 +- .../main/java/io/presto/kudu/RowHelper.java | 1 - .../procedures/RangePartitionProcedures.java | 1 - .../presto/kudu/properties/ColumnDesign.java | 1 - .../properties/HashPartitionDefinition.java | 1 - .../kudu/properties/PartitionDesign.java | 1 - .../kudu/properties/RangeBoundValue.java | 1 - .../RangeBoundValueDeserializer.java | 1 - .../properties/RangeBoundValueSerializer.java | 1 - .../kudu/properties/RangePartition.java | 1 - .../properties/RangePartitionDefinition.java | 1 - .../kudu/client/KeyEncoderAccessor.java | 1 - .../src/test/java/io/presto/kudu/TestDec.java | 1 - .../presto/kudu/TestDecimalColumnsTest.java | 1 - .../src/test/java/io/presto/kudu/TestInt.java | 1 - .../presto/kudu/TestIntegerColumnsTest.java | 1 - .../kudu/TestKuduIntegrationSmokeTest.java | 1 - .../kudu/TestRangePartitioningTest.java | 1 - .../test/java/io/presto/kudu/TestRanges.java | 1 - .../io/presto/kudu/TestSchemaNoExist.java | 1 - .../RangePartitionSerializationTest.java | 1 - presto-main/etc/catalog/kudu.properties | 25 +++--- 33 files changed, 148 insertions(+), 121 deletions(-) diff --git a/hetu-docs/en/connector/kudu.md b/hetu-docs/en/connector/kudu.md index 6cd66ca14..a7020c07f 100644 --- a/hetu-docs/en/connector/kudu.md +++ b/hetu-docs/en/connector/kudu.md @@ -25,20 +25,18 @@ interessed in my binary build project [kudu-rpm](https://github.com/MartinWeinde ## [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1 kudu.client.master-addresses=localhost:7051 - #if kudu with kerberos, enable this - is.kerberos.on = false + #kudu.authentication.type,NONE or KERBEROS + kudu.authentication.type = NONE + #kerberos user for kudu with kerberos - kerberos.user= root@EXAMPLE.COM + kudu.client.principle= root@EXAMPLE.COM #keytab path - keytab.path = /root/root.keytab - - #the path of hdfs-site.xml is used for kerberos, only is.kerberos.on = true need it。 - hdfs-site.xml.path = /root/hdfs-site.xml + kudu.client.keytab = /root/root.keytab - #the path of core-site.xml is used for kerberos, only is.kerberos.on = true need it。 - core-site.xml.path = /root/core-site.xml + #the path of hdfs-site.xml and core-site.xml is used for kerberos, only is.kerberos.on = true need it。 + kudu.config.resources=/root/hdfs-site.xml,/root/core-site.xml #after the service started,the delay time of kerberos re-authentication,this is second time of re-authentication kudu.reauth.delay.time = 60 @@ -48,7 +46,7 @@ interessed in my binary build project [kudu-rpm](https://github.com/MartinWeinde ## Optional restriction of tablets for specific tenant. ## If a tenant is set, only Kudu tablets starting with `.` will - ## be visible in openLookeng + ## be visible in openLooKeng #kudu.session.tenant=root_db #If configuration is enabled,you can select database for kudu metadata table @@ -74,8 +72,35 @@ interessed in my binary build project [kudu-rpm](https://github.com/MartinWeinde #kudu.client.disableStatistics = false ``` when show schemas,kudu connector will create metadata talbe named '_schema'。if you want to connected kudu cluster with kerberos ,you must configure the path of krb5.conf,such as -Djava.security.krb5.conf=/Users/mac/Desktop/krb5.conf + +The following table describes the meanings of the configuration items: + +| Property Name | Default Value | Required | Description | +|----------|---------------|----------|---------------------------------------------------------------------------------------------------------------------------------------------------| +| connector.name| (None) | Yes | Connector name,please fill in "kudu" | +| kudu.client.master-addresses| (None) | Yes | List of Kudu master addresses | +| kudu.authentication.type| NONE | No | kudu.authentication.type,NONE or KERBEROS | +| kudu.client.principle| (None) | No | Kerberos user for kudu with kerberos | +| kudu.client.keytab| (None) | No | Keytab path | +| kudu.config.resources| (None) | No | T·he path of hdfs-site.xml and core-site.xml is used for kerberos, only is.kerberos.on = true need it | +| kudu.reauth.delay.time| (None) | No | After the service started,the delay time of kerberos re-authentication,this is second time of re-authentication | +| kudu.reauth.period.time| (None) | No | After service started,this configuration is re-authentication interval,This configuration is to prevent ticket expiration | +| kudu.session.tenant| (None) | No | Optional restriction of tablets for specific tenant.If a tenant is set, only Kudu tablets starting with `.` willbe visible in openLooKeng | +| kudu.metadata.on| false | No | If configuration is enabled,you can select database for kudu metadata table | +| kudu.metadata.database| (None) | No | For example, if kudu.metadata.on = true ,metadata table in kudu is root_db._schema | +| kudu.client.defaultAdminOperationTimeout| (None) | No | Default timeout used for administrative operations (e.g. createTable, deleteTable, etc.) | +| kudu.client.defaultOperationTimeout| (None) | No | Default timeout used for user operations | +| kudu.client.defaultSocketReadTimeout| (None) | No | Default timeout to use when waiting on data from a socket | +| kudu.client.disableStatistics| false | No | Disable Kudu client's collection of statistics | + + +the minimum configuration to connect kudu as below: + ``` + connector.name=kudu + kudu.client.master-addresses=localhost:7051 + ``` ## Querying Data -A Kudu table named `mytable` is available in openLookeng as table `kudu.default.mytable`. +A Kudu table named `mytable` is available in openLooKeng as table `kudu.default.mytable`. A Kudu table containing a dot is considered as a schema/table combination, e.g. `dev.mytable` is mapped to the openLooKeng table `kudu.dev.mytable. Only Kudu table names in lower case are currently supported. @@ -227,7 +252,20 @@ Example: ``` '{"column1": {"key": true, "encoding": "dictionary", "compression": "LZ4"}, "column2": {...}}' ``` +Encoding: + +| column type (kudu) | encoding | default | +|--------------------------------|----------------------------------------------------|-------------------------------------------------------------------------------------------| +| `int8`,`int16`,`int32`,`int64` | `PLAIN_ENCODING`,`BIT_SHUFFLE`,`RLE` | BIT_SHUFFLE | +| `date`,`unixtime`,`micros` | `PLAIN_ENCODING`,`BIT_SHUFFLE`,`RLE` | BIT_SHUFFLE | +| `float`,`double`,`decimal` | `PLAIN_ENCODING`,`BIT_SHUFFLE` | BIT_SHUFFLE | +| `bool` | `PLAIN_ENCODING`,`RLE` | RLE | +| `string`,`varchar`,`binary` | `PLAIN_ENCODING`,`PREFIX_ENCODING`,`DICT_ENCODING` | DICT_ENCODING | +Compression: +``` +LZ4,Snappy,zlib +``` ### Table property `partition_design` With the partition design table property you define the partition layout. In Apache Kudu you can define multiple hash partitions and at most one range partition. diff --git a/hetu-docs/zh/connector/kudu.md b/hetu-docs/zh/connector/kudu.md index d6536eeec..84be1a591 100644 --- a/hetu-docs/zh/connector/kudu.md +++ b/hetu-docs/zh/connector/kudu.md @@ -6,7 +6,7 @@ ## 兼容性 -kudu 1.15版本已经已经测试并在生产环境使用,如果您需要使用其他版本的kudu,可以下载源码,并修改presto-kudu模块中的pom.xml,修改kudu依赖版本,并重新编译。 +kudu 1.15版本已经测试并在生产环境使用,如果您需要使用其他版本的kudu,可以下载源码,并修改presto-kudu模块中的pom.xml,修改kudu依赖版本,并重新编译。 ### 部署 Kudu server 通过如下向导安装kudu [Apache Kudu](https://kudu.apache.org/)。 @@ -23,20 +23,17 @@ kudu 1.15版本已经已经测试并在生产环境使用,如果您需要使 ## [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1 kudu.client.master-addresses=localhost:7051 - #如果kudu集成了kerberos认证,则开启如下配置 - is.kerberos.on = false + #如果kudu集成了kerberos认证,则开启如下配置,NONE或者为KERBEROS + kudu.authentication.type = NONE #kerberos认证主体 - kerberos.user= root@EXAMPLE.COM + kudu.client.principle= root@EXAMPLE.COM #keytab的路径 - keytab.path = /root/root.keytab + kudu.client.keytab = /root/root.keytab - #使用kerberos认证时,需要配置hdfs-site.xml的路径 - hdfs-site.xml.path = /root/hdfs-site.xml - - #使用kerberos认证时,需要配置core-site.xml的路径 - core-site.xml.path = /root/core-site.xml + #使用kerberos认证时,需要配置hdfs-site.xml的路径和core-site.xml的路径 + kudu.config.resources=/root/hdfs-site.xml,/root/core-site.xml #服务启动后会自动认证,该配置为服务启动后从启动开始到第二次认证的时间间隔 kudu.reauth.delay.time = 60 @@ -71,8 +68,32 @@ kudu 1.15版本已经已经测试并在生产环境使用,如果您需要使 ``` 当执行“show schemas”命令时,kudu连接器将会创建名称为 '_schema'的元数据表。此外,使用kerberos认证的kudu集群,请在jvm.config里配置krb5.conf的路径,例如:-Djava.security.krb5.conf=/Users/mac/Desktop/krb5.conf ## Querying Data -创建表的基本语如下所示,一个名称为"mytable"的表,实际在Openlookeng中为kudu.default.mytable,一个带有"."的前缀,被认为是schema名称,例如dev.mytable表的实际映射为kudu.dev.mytable,仅小写的kudu表名称被支持。 - +创建表的基本语法如下所示,一个名称为"mytable"的表,实际在openLooKeng中为kudu.default.mytable,一个带有"."的前缀,被认为是schema名称,例如dev.mytable表的实际映射为kudu.dev.mytable,仅小写的kudu表名称被支持。 + +各项配置含义及说明如下所示: +| 属性名称| 默认值 | 是否必填 | 说明 | +|----------|-------|------|---------------------------------------------| +| connector.name| (无) | 是 | 连接器名称,请填写kudu | +| kudu.client.master-addresses| (无) | 是 | Kudu集群地址 | +| kudu.authentication.type| NONE | 否 | kudu集群的权限认证类型,可填写NONE或者KERBEROS | +| kudu.client.principle| (无) | 否 | 对于kerbers认证的kudu集群,该值为认证主体 | +| kudu.client.keytab| (无) | 否 | 对于kerbers认证的kudu集群,该值为keytab路径 | +| kudu.config.resources| (无) | 否 | 使用kerberos认证时,需要配置hdfs-site.xml的路径和core-site.xml的路径 | +| kudu.reauth.delay.time| (无) | 否 | 服务启动后会自动认证,该配置为服务启动后从启动开始到第二次认证的时间间隔 | +| kudu.reauth.period.time| (无) | 否 | 第二次认证后的时间间隔,保证票据刷新 | +| kudu.session.tenant| (无) | 否 | 可选项,对每个租户的表的限制,如果被设置后,以其开头的kudu表才可见,若配置为“root_db”,则表名称以“root_db.”开头的表才可见 | +| kudu.metadata.on| false | 否 | 如果配置开启,则可以为元数据表选择数据库去存放 | +| kudu.metadata.database| (无) | 否 | 如果开启kudu.metadata.on,在kudu端看到的元数据表为 root_db._schema,这两种配置目的是为了将“_schemas"表指定到特定数据库,以防止租户保存元数据时无权限 | +| kudu.client.defaultAdminOperationTimeout| (无) | 否 | kudu管理功能操作的超时时间 | +| kudu.client.defaultOperationTimeout| (无) | 否 | kudu用户操作的超时时间 | +| kudu.client.defaultSocketReadTimeout| (无) | 否 | 等待socket的超时时间 | +| kudu.client.disableStatistics| false | 否 | 是否开启kudu客户端的statistics. | + +连接kudu的最小集配置如下: + ``` + connector.name=kudu + kudu.client.master-addresses=localhost:7051 + ``` - 现在可以创建kudu表,表名称为小写且不含“.”。 - 例如你可以创建一个用户表。 @@ -218,7 +239,20 @@ Example: ``` '{"column1": {"key": true, "encoding": "dictionary", "compression": "LZ4"}, "column2": {...}}' ``` +Encoding类型如下: + +| column type (kudu) | encoding | default | +|--------------------------------|----------------------------------------------------|-------------------------------------------------------------------------------------------| +| `int8`,`int16`,`int32`,`int64` | `PLAIN_ENCODING`,`BIT_SHUFFLE`,`RLE` | BIT_SHUFFLE | +| `date`,`unixtime`,`micros` | `PLAIN_ENCODING`,`BIT_SHUFFLE`,`RLE` | BIT_SHUFFLE | +| `float`,`double`,`decimal` | `PLAIN_ENCODING`,`BIT_SHUFFLE` | BIT_SHUFFLE | +| `bool` | `PLAIN_ENCODING`,`RLE` | RLE | +| `string`,`varchar`,`binary` | `PLAIN_ENCODING`,`PREFIX_ENCODING`,`DICT_ENCODING` | DICT_ENCODING | +Compression类型如下: +``` +LZ4,Snappy,zlib +``` ### 表属性 `partition_design` 通过partition design能够定义partition layout。 在kudu中可以定义多个hash partitions和最多一个range partition. diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java b/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java index 79b5efdbb..e650d8b5c 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduClientConfig.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at @@ -38,11 +37,10 @@ public class KuduClientConfig private Duration defaultSocketReadTimeout = new Duration(10, TimeUnit.SECONDS); private boolean disableStatistics; private String tenant; - private String hdfsXmlPath; - private String coreXmlPath; - private String isKerberosOn; - private String keytabPath; - private String kerberosUser; + private String kuduConfigResources; + private String kuduAuthenticationType; + private String kuduClientKeytab; + private String kuduClientPrinciple; private String kuduMetadataOn; private String kuduMetadataDatabase; private String reauthDelayTime; @@ -192,81 +190,67 @@ public class KuduClientConfig return this; } - public String getHdfsXmlPath() + public String getKuduConfigResources() { - return hdfsXmlPath; + return kuduConfigResources; } - @Mandatory(name = "hdfs-site.xml.path", - description = "hdfs-site.xml.path", + @Mandatory(name = "kudu.config.resources", + description = "kudu.config.resources", defaultValue = "", required = false) - @Config("hdfs-site.xml.path") - public KuduClientConfig setHdfsXmlPath(String hdfsXmlPath) + @Config("kudu.config.resources") + public KuduClientConfig setKuduConfigResources(String kuduConfigResources) { - this.hdfsXmlPath = hdfsXmlPath; + this.kuduConfigResources = kuduConfigResources; return this; } - public String getCoreXmlPath() + public String getKuduAuthenticationType() { - return coreXmlPath; + return kuduAuthenticationType; } - @Mandatory(name = "core-site.xml.path", - description = "core-site.xml.path", - defaultValue = "", - required = false) - @Config("core-site.xml.path") - public KuduClientConfig setCoreXmlPath(String coreXmlPath) - { - this.coreXmlPath = coreXmlPath; - return this; - } - - public String getIsKerberosOn() - { - return isKerberosOn; - } - - @Mandatory(name = "is.kerberos.on", - description = "is.kerberos.on", - defaultValue = "false", + @Mandatory(name = "kudu.authentication.type", + description = "kudu.authentication.type", + defaultValue = "NONE", required = false) - @Config("is.kerberos.on") - public KuduClientConfig setIsKerberosOn(String isKerberosOn) + @Config("kudu.authentication.type") + public KuduClientConfig setKuduAuthenticationType(String kuduAuthenticationType) { - this.isKerberosOn = isKerberosOn; + this.kuduAuthenticationType = kuduAuthenticationType; return this; } - public String getKeytabPath() + public String getKuduClientKeytab() { - return keytabPath; + return kuduClientKeytab; } - @Mandatory(name = "keytab.path", - description = "keytab.path", + @Mandatory(name = "kudu.client.keytab", + description = "kudu.client.keytab", defaultValue = "", required = false) - @Config("keytab.path") - public void setKeytabPath(String keytabPath) + @Config("kudu.client.keytab") + public KuduClientConfig setKuduClientKeytab(String kuduClientKeytab) { - this.keytabPath = keytabPath; + this.kuduClientKeytab = kuduClientKeytab; + return this; } - public String getKerberosUser() + public String getKuduClientPrinciple() { - return kerberosUser; + return kuduClientPrinciple; } - @Mandatory(name = "kerberos.user", - description = "kerberos.user", + @Mandatory(name = "kudu.client.principle", + description = " kudu.client.principle", defaultValue = "", required = false) - @Config("kerberos.user") - public void setKerberosUser(String kerberosUser) + @Config("kudu.client.principle") + public KuduClientConfig setKuduClientPrinciple(String kuduClientPrinciple) { - this.kerberosUser = kerberosUser; + this.kuduClientPrinciple = kuduClientPrinciple; + return this; } } diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduClientSession.java b/presto-kudu/src/main/java/io/presto/kudu/KuduClientSession.java index f451b21c3..37c9e8ae6 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduClientSession.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduClientSession.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduColumnHandle.java b/presto-kudu/src/main/java/io/presto/kudu/KuduColumnHandle.java index 1cb97bbef..731a67794 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduColumnHandle.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduColumnHandle.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java b/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java index c4388b325..a65868d4d 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduModule.java @@ -98,7 +98,7 @@ public class KuduModule KuduClientConfig config) { requireNonNull(config, "config is null"); - if ("true".equalsIgnoreCase(config.getIsKerberosOn())) { + if ("kerberos".equalsIgnoreCase(config.getKuduAuthenticationType())) { kbAuth(config); reauth(config, connectorId); KuduClient client; @@ -146,15 +146,15 @@ public class KuduModule try { System.setProperty("javax.security.auth.useSubjectCredsOnly", "false"); Configuration configuration = new Configuration(); - InputStream hdfs = new FileInputStream(config.getHdfsXmlPath()); - InputStream core = new FileInputStream(config.getCoreXmlPath()); + InputStream hdfs = new FileInputStream(config.getKuduConfigResources().split(",")[0]); + InputStream core = new FileInputStream(config.getKuduConfigResources().split(",")[1]); configuration.addResource(hdfs); configuration.addResource(core); UserGroupInformation.setConfiguration(configuration); - UserGroupInformation.loginUserFromKeytab(config.getKerberosUser(), config.getKeytabPath()); + UserGroupInformation.loginUserFromKeytab(config.getKuduClientPrinciple(), config.getKuduClientKeytab()); } catch (Exception e) { - e.printStackTrace(); + log.error("failed to auth:" + e); } } diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduPageSink.java b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSink.java index 620121eda..813a7f206 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduPageSink.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSink.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduPageSinkProvider.java b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSinkProvider.java index b37b5cc3d..31a627c37 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduPageSinkProvider.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduPageSinkProvider.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java index cf9296f70..1487c2652 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursor.java @@ -122,6 +122,7 @@ public class KuduRecordCursor return ((org.apache.kudu.util.Slice) rowDataField.get(currentRow)); } catch (IllegalAccessException e) { + log.error("failed to get current row data:" + e); return null; } } diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursorWithVirtualRowId.java b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursorWithVirtualRowId.java index 0ce9947ea..9d4605c6e 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursorWithVirtualRowId.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduRecordCursorWithVirtualRowId.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/KuduUpdatablePageSource.java b/presto-kudu/src/main/java/io/presto/kudu/KuduUpdatablePageSource.java index ed16bce41..5bddb34ae 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/KuduUpdatablePageSource.java +++ b/presto-kudu/src/main/java/io/presto/kudu/KuduUpdatablePageSource.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java b/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java index 5c4fd2923..acd29ebde 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java +++ b/presto-kudu/src/main/java/io/presto/kudu/NativeKuduClientSession.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at @@ -190,7 +189,7 @@ public class NativeKuduClientSession .key(true).build(); Schema schema = new Schema(ImmutableList.of(tenantColumnSchema, schemaColumnSchema)); CreateTableOptions options = new CreateTableOptions(); - options.setNumReplicas(1); // TODO config + options.setNumReplicas(1); options.addHashPartitions(ImmutableList.of(tenantColumnSchema.getName()), 2); KuduTable schemasTable = client.createTable(rawSchemasTableName, schema, options); KuduSession session = client.newSession(); diff --git a/presto-kudu/src/main/java/io/presto/kudu/RowHelper.java b/presto-kudu/src/main/java/io/presto/kudu/RowHelper.java index 687087821..c72933904 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/RowHelper.java +++ b/presto-kudu/src/main/java/io/presto/kudu/RowHelper.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/procedures/RangePartitionProcedures.java b/presto-kudu/src/main/java/io/presto/kudu/procedures/RangePartitionProcedures.java index 590f71076..58fdf0691 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/procedures/RangePartitionProcedures.java +++ b/presto-kudu/src/main/java/io/presto/kudu/procedures/RangePartitionProcedures.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/ColumnDesign.java b/presto-kudu/src/main/java/io/presto/kudu/properties/ColumnDesign.java index f392c0d7b..7bfc8ce30 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/ColumnDesign.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/ColumnDesign.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/HashPartitionDefinition.java b/presto-kudu/src/main/java/io/presto/kudu/properties/HashPartitionDefinition.java index 73903dd44..f73bf88ce 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/HashPartitionDefinition.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/HashPartitionDefinition.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/PartitionDesign.java b/presto-kudu/src/main/java/io/presto/kudu/properties/PartitionDesign.java index 419d2330a..62ac82040 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/PartitionDesign.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/PartitionDesign.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValue.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValue.java index 2428f1651..b25657137 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValue.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValue.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueDeserializer.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueDeserializer.java index 48dba0fb2..e528fa61a 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueDeserializer.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueDeserializer.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueSerializer.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueSerializer.java index 47bff28fb..2a31ecb17 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueSerializer.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangeBoundValueSerializer.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartition.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartition.java index d41b36039..6c12947e2 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartition.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartition.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartitionDefinition.java b/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartitionDefinition.java index b2c454608..df760fbfd 100644 --- a/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartitionDefinition.java +++ b/presto-kudu/src/main/java/io/presto/kudu/properties/RangePartitionDefinition.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java b/presto-kudu/src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java index e6dbdb1df..779908d20 100644 --- a/presto-kudu/src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java +++ b/presto-kudu/src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestDec.java b/presto-kudu/src/test/java/io/presto/kudu/TestDec.java index 7b153261b..ed1132e69 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/TestDec.java +++ b/presto-kudu/src/test/java/io/presto/kudu/TestDec.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestDecimalColumnsTest.java b/presto-kudu/src/test/java/io/presto/kudu/TestDecimalColumnsTest.java index 683a86316..f3b1b106b 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/TestDecimalColumnsTest.java +++ b/presto-kudu/src/test/java/io/presto/kudu/TestDecimalColumnsTest.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestInt.java b/presto-kudu/src/test/java/io/presto/kudu/TestInt.java index 7700db6c3..db8546080 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/TestInt.java +++ b/presto-kudu/src/test/java/io/presto/kudu/TestInt.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestIntegerColumnsTest.java b/presto-kudu/src/test/java/io/presto/kudu/TestIntegerColumnsTest.java index 8b4acbf6c..043e29283 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/TestIntegerColumnsTest.java +++ b/presto-kudu/src/test/java/io/presto/kudu/TestIntegerColumnsTest.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestKuduIntegrationSmokeTest.java b/presto-kudu/src/test/java/io/presto/kudu/TestKuduIntegrationSmokeTest.java index de8f09a3d..b15d11612 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/TestKuduIntegrationSmokeTest.java +++ b/presto-kudu/src/test/java/io/presto/kudu/TestKuduIntegrationSmokeTest.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestRangePartitioningTest.java b/presto-kudu/src/test/java/io/presto/kudu/TestRangePartitioningTest.java index c57163ad6..e7397a44f 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/TestRangePartitioningTest.java +++ b/presto-kudu/src/test/java/io/presto/kudu/TestRangePartitioningTest.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestRanges.java b/presto-kudu/src/test/java/io/presto/kudu/TestRanges.java index f8dd1ccda..7aaba4ff3 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/TestRanges.java +++ b/presto-kudu/src/test/java/io/presto/kudu/TestRanges.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/TestSchemaNoExist.java b/presto-kudu/src/test/java/io/presto/kudu/TestSchemaNoExist.java index aa67f04ca..2d1008b04 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/TestSchemaNoExist.java +++ b/presto-kudu/src/test/java/io/presto/kudu/TestSchemaNoExist.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-kudu/src/test/java/io/presto/kudu/properties/RangePartitionSerializationTest.java b/presto-kudu/src/test/java/io/presto/kudu/properties/RangePartitionSerializationTest.java index b538e9619..4583b67f0 100644 --- a/presto-kudu/src/test/java/io/presto/kudu/properties/RangePartitionSerializationTest.java +++ b/presto-kudu/src/test/java/io/presto/kudu/properties/RangePartitionSerializationTest.java @@ -1,5 +1,4 @@ /* - * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved. * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at diff --git a/presto-main/etc/catalog/kudu.properties b/presto-main/etc/catalog/kudu.properties index d8eae312d..a32c76a50 100644 --- a/presto-main/etc/catalog/kudu.properties +++ b/presto-main/etc/catalog/kudu.properties @@ -3,28 +3,25 @@ connector.name=kudu ## List of Kudu master addresses, at least one is needed (comma separated) ## Supported formats: example.com, example.com:7051, 192.0.2.1, 192.0.2.1:7051, ## [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1 -kudu.client.master-addresses=localhost:7051 +kudu.client.master-addresses=192.168.1.51:7051 -#if kudu with kerberos, enable this -is.kerberos.on = false +#kudu.authentication.type NONE or KERBEROS +#kudu.authentication.type = NONE #kerberos user for kudu with kerberos -kerberos.user= root@EXAMPLE.COM +#kudu.client.principle= root@EXAMPLE.COM #keytab path -keytab.path = /root/root.keytab +#kudu.client.keytab = /root/root.keytab -#the path of hdfs-site.xml is used for kerberos, only is.kerberos.on = true need it? -hdfs-site.xml.path = /root/hdfs-site.xml - -#the path of core-site.xml is used for kerberos, only is.kerberos.on = true need it? -core-site.xml.path = /root/core-site.xml +#the path of hdfs-site.xml and core-site.xml is used for kerberos, only is.kerberos.on = true need it? +#kudu.config.resources=/root/hdfs-site.xml,/root/core-site.xml #after the service started,the delay time of kerberos re-authentication,this is second time of re-authentication -kudu.reauth.delay.time = 60 +#kudu.reauth.delay.time = 60 #after service started,this configuration is re-authentication interval,This configuration is to prevent ticket expiration -kudu.reauth.period.time = 82800 +#kudu.reauth.period.time = 82800 ## Optional restriction of tablets for specific tenant. ## If a tenant is set, only Kudu tablets starting with `.` will @@ -32,10 +29,10 @@ kudu.reauth.period.time = 82800 #kudu.session.tenant=root_db #If configuration is enabled,you can select database for kudu metadata table -kudu.metadata.on = false +#kudu.metadata.on = false #for example, if kudu.metadata.on = true ,metadata table in kudu is root_db._schema -kudu.metadata.database = root_Db +#kudu.metadata.database = root_db ####################### ### Advanced Kudu Java client configuration -- Gitee From e18f13fd2cdfbe1ab8e3ed93e07dc9acb2ce0504 Mon Sep 17 00:00:00 2001 From: Zhang Jianming Date: Wed, 4 Oct 2023 12:49:47 +0800 Subject: [PATCH 4/5] fix eclipse error --- hetu-heuristic-index/pom.xml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/hetu-heuristic-index/pom.xml b/hetu-heuristic-index/pom.xml index cf0c5c1ec..c5c101dec 100644 --- a/hetu-heuristic-index/pom.xml +++ b/hetu-heuristic-index/pom.xml @@ -55,6 +55,20 @@ org.mapdb mapdb 3.0.8 + + + org.eclipse.collections + eclipse-collections + + + org.eclipse.collections + eclipse-collections-api + + + org.eclipse.collections + eclipse-collections-forkjoin + + org.roaringbitmap -- Gitee From 20bb5f1111aea23286b376d599bf6db949a9dd41 Mon Sep 17 00:00:00 2001 From: Zhang Jianming Date: Sun, 28 Jan 2024 16:32:51 +0800 Subject: [PATCH 5/5] fix mapdb error,the version of 3.0.8 can only compiled in jdk11 --- hetu-heuristic-index/pom.xml | 16 +--------------- .../heuristicindex/index/btree/BTreeIndex.java | 5 ++++- 2 files changed, 5 insertions(+), 16 deletions(-) diff --git a/hetu-heuristic-index/pom.xml b/hetu-heuristic-index/pom.xml index c5c101dec..83fa1e209 100644 --- a/hetu-heuristic-index/pom.xml +++ b/hetu-heuristic-index/pom.xml @@ -54,21 +54,7 @@ org.mapdb mapdb - 3.0.8 - - - org.eclipse.collections - eclipse-collections - - - org.eclipse.collections - eclipse-collections-api - - - org.eclipse.collections - eclipse-collections-forkjoin - - + 3.0.5 org.roaringbitmap diff --git a/hetu-heuristic-index/src/main/java/io/hetu/core/plugin/heuristicindex/index/btree/BTreeIndex.java b/hetu-heuristic-index/src/main/java/io/hetu/core/plugin/heuristicindex/index/btree/BTreeIndex.java index a251a30fe..f7dbbcadb 100644 --- a/hetu-heuristic-index/src/main/java/io/hetu/core/plugin/heuristicindex/index/btree/BTreeIndex.java +++ b/hetu-heuristic-index/src/main/java/io/hetu/core/plugin/heuristicindex/index/btree/BTreeIndex.java @@ -213,7 +213,10 @@ public class BTreeIndex public Properties getProperties() { Properties result = new Properties(); - for (Map.Entry entry : this.properties.entrySet()) { + Set set = this.properties.entrySet(); + Iterator> entryIterator = this.properties.entryIterator(); + while (entryIterator.hasNext()) { + Map.Entry entry = entryIterator.next(); result.put(entry.getKey(), entry.getValue()); } return result; -- Gitee