diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 20a6666..72b98a2 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -36,10 +36,3 @@ jobs: with: flink_version: ${{ matrix.flink }} jdk_version: ${{ matrix.jdk }} - python_test: - strategy: - matrix: - flink: [ 1.17.2, 1.18.1, 1.19-SNAPSHOT ] - uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils - with: - flink_version: ${{ matrix.flink }} \ No newline at end of file diff --git a/.gitmodules b/.gitmodules new file mode 100644 index 0000000..e5d40f3 --- /dev/null +++ b/.gitmodules @@ -0,0 +1,4 @@ +[submodule "tools/releasing/shared"] + path = tools/releasing/shared + url = https://github.com/apache/flink-connector-shared-utils + branch = release_utils diff --git a/.idea/vcs.xml b/.idea/vcs.xml new file mode 100644 index 0000000..31d1734 --- /dev/null +++ b/.idea/vcs.xml @@ -0,0 +1,25 @@ + + + + + + + + + + diff --git a/NOTICE b/NOTICE index c1e8320..c3ce37e 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ -Apache Flink Kudu Connector -Copyright 2014-2024 The Apache Software Foundation +Apache Bahir +Copyright (c) 2016-2024 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). @@ -12,5 +12,3 @@ ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUT DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. - - diff --git a/README.md b/README.md index 1944eca..477aa84 100644 --- a/README.md +++ b/README.md @@ -2,6 +2,10 @@ This repository contains the official Apache Flink Kudu connector. +## Forked from Bahir + +The connector code is forked from [Apache Bahir](https://bahir.apache.org/) project after its retirement. + ## Apache Flink Apache Flink is an open source stream processing framework with powerful stream- and batch-processing capabilities. @@ -65,4 +69,4 @@ This article describes [how to contribute to Apache Flink](https://flink.apache. ## About Apache Flink is an open source project of The Apache Software Foundation (ASF). -The Apache Flink project originated from the [Stratosphere](http://stratosphere.eu) research project. \ No newline at end of file +The Apache Flink project originated from the [Stratosphere](http://stratosphere.eu) research project. diff --git a/flink-connector-kudu/README.md b/flink-connector-kudu/README.md new file mode 100644 index 0000000..7908e08 --- /dev/null +++ b/flink-connector-kudu/README.md @@ -0,0 +1,324 @@ + + +# Flink Kudu Connector + +This connector provides a source (```KuduInputFormat```), a sink/output +(```KuduSink``` and ```KuduOutputFormat```, respectively), +as well a table source (`KuduTableSource`), an upsert table sink (`KuduTableSink`), and a catalog (`KuduCatalog`), +to allow reading and writing to [Kudu](https://kudu.apache.org/). + +To use this connector, add the following dependency to your project: + + + org.apache.bahir + flink-connector-kudu_2.11 + 1.1-SNAPSHOT + +*Version Compatibility*: This module is compatible with Apache Kudu *1.11.1* (last stable version) and Apache Flink 1.10.+. + +Note that the streaming connectors are not part of the binary distribution of Flink. You need to link them into your job jar for cluster execution. +See how to link with them for cluster execution [here](https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/projectsetup/dependencies.html). + +## Installing Kudu + +Follow the instructions from the [Kudu Installation Guide](https://kudu.apache.org/docs/installation.html). +Optionally, you can use the docker images provided in dockers folder. + +## SQL and Table API + +The Kudu connector is fully integrated with the Flink Table and SQL APIs. Once we configure the Kudu catalog (see next section) +we can start querying or inserting into existing Kudu tables using the Flink SQL or Table API. + +For more information about the possible queries please check the [official documentation](https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/sql/) + +### Kudu Catalog + +The connector comes with a catalog implementation to handle metadata about your Kudu setup and perform table management. +By using the Kudu catalog, you can access all the tables already created in Kudu from Flink SQL queries. The Kudu catalog only +allows users to create or access existing Kudu tables. Tables using other data sources must be defined in other catalogs such as +in-memory catalog or Hive catalog. + +When using the SQL CLI you can easily add the Kudu catalog to your environment yaml file: + +``` +catalogs: + - name: kudu + type: kudu + kudu.masters: :7051 +``` +Once the SQL CLI is started you can simply switch to the Kudu catalog by calling `USE CATALOG kudu;` + +You can also create and use the KuduCatalog directly in the Table environment: + +```java +String KUDU_MASTERS="host1:port1,host2:port2" +KuduCatalog catalog = new KuduCatalog(KUDU_MASTERS); +tableEnv.registerCatalog("kudu", catalog); +tableEnv.useCatalog("kudu"); +``` +### DDL operations using SQL + +It is possible to manipulate Kudu tables using SQL DDL. + +When not using the Kudu catalog, the following additional properties must be specified in the `WITH` clause: + +* `'connector.type'='kudu'` +* `'kudu.masters'='host1:port1,host2:port2,...'`: comma-delimitered list of Kudu masters +* `'kudu.table'='...'`: The table's name within the Kudu database. + +If you have registered and are using the Kudu catalog, these properties are handled automatically. + +To create a table, the additional properties `kudu.primary-key-columns` and `kudu.hash-columns` must be specified +as comma-delimited lists. Optionally, you can set the `kudu.replicas` property (defaults to 1). +Other properties, such as range partitioning, cannot be configured here - for more flexibility, please use +`catalog.createTable` as described in [this](#Creating-a-KuduTable-directly-with-KuduCatalog) section or create the table directly in Kudu. + +The `NOT NULL` constraint can be added to any of the column definitions. +By setting a column as a primary key, it will automatically by created with the `NOT NULL` constraint. +Hash columns must be a subset of primary key columns. + +Kudu Catalog + +``` +CREATE TABLE TestTable ( + first STRING, + second STRING, + third INT NOT NULL +) WITH ( + 'kudu.hash-columns' = 'first', + 'kudu.primary-key-columns' = 'first,second' +) +``` +Other catalogs + +``` +CREATE TABLE TestTable ( + first STRING, + second STRING, + third INT NOT NULL +) WITH ( + 'connector.type' = 'kudu', + 'kudu.masters' = '...', + 'kudu.table' = 'TestTable', + 'kudu.hash-columns' = 'first', + 'kudu.primary-key-columns' = 'first,second' +) +``` +Renaming a table: + +``` +ALTER TABLE TestTable RENAME TO TestTableRen +``` +Dropping a table: + +```sql +DROP TABLE TestTableRen +``` +#### Creating a KuduTable directly with KuduCatalog + +The KuduCatalog also exposes a simple `createTable` method that required only the where table configuration, +including schema, partitioning, replication, etc. can be specified using a `KuduTableInfo` object. + +Use the `createTableIfNotExists` method, that takes a `ColumnSchemasFactory` and +a `CreateTableOptionsFactory` parameter, that implement respectively `getColumnSchemas()` +returning a list of Kudu [ColumnSchema](https://kudu.apache.org/apidocs/org/apache/kudu/ColumnSchema.html) objects; +and `getCreateTableOptions()` returning a +[CreateTableOptions](https://kudu.apache.org/apidocs/org/apache/kudu/client/CreateTableOptions.html) object. + +This example shows the creation of a table called `ExampleTable` with two columns, +`first` being a primary key; and configuration of replicas and hash partitioning. + +```java +KuduTableInfo tableInfo = KuduTableInfo + .forTable("ExampleTable") + .createTableIfNotExists( + () -> + Lists.newArrayList( + new ColumnSchema + .ColumnSchemaBuilder("first", Type.INT32) + .key(true) + .build(), + new ColumnSchema + .ColumnSchemaBuilder("second", Type.STRING) + .build() + ), + () -> new CreateTableOptions() + .setNumReplicas(1) + .addHashPartitions(Lists.newArrayList("first"), 2)); + +catalog.createTable(tableInfo, false); +``` +The example uses lambda expressions to implement the functional interfaces. + +Read more about Kudu schema design in the [Kudu docs](https://kudu.apache.org/docs/schema_design.html). + +### Supported data types + + +| Flink/SQL | Kudu | +| ---------------- | :---------------: | +| `STRING` | STRING | +| `BOOLEAN` | BOOL | +| `TINYINT` | INT8 | +| `SMALLINT` | INT16 | +| `INT` | INT32 | +| `BIGINT` | INT64 | +| `FLOAT` | FLOAT | +| `DOUBLE` | DOUBLE | +| `BYTES` | BINARY | +| `TIMESTAMP(3)` | UNIXTIME_MICROS | + +Note: + +* `TIMESTAMP`s are fixed to a precision of 3, and the corresponding Java conversion class is `java.sql.Timestamp` +* `BINARY` and `VARBINARY` are not yet supported - use `BYTES`, which is a `VARBINARY(2147483647)` +* `CHAR` and `VARCHAR` are not yet supported - use `STRING`, which is a `VARCHAR(2147483647)` +* `DECIMAL` types are not yet supported + +### Lookup Cache + +Kudu connector can be used in temporal join as a lookup source (aka. dimension table). Currently, only sync lookup mode is supported. + +By default, lookup cache is not enabled. You can enable it by setting both `lookup.cache.max-rows` and `lookup.cache.ttl`. + +The lookup cache is used to improve performance of temporal join theKudu connector. By default, lookup cache is not enabled, so all the requests are sent to external database. When lookup cache is enabled, each process (i.e. TaskManager) will hold a cache. Flink will lookup the cache first, and only send requests to external database when cache missing, and update cache with the rows returned. The oldest rows in cache will be expired when the cache hit to the max cached rows `kudu.lookup.cache.max-rows` or when the row exceeds the max time to live `kudu.lookup.cache.ttl`. The cached rows might not be the latest, users can tune `kudu.lookup.cache.ttl` to a smaller value to have a better fresh data, but this may increase the number of requests send to database. So this is a balance between throughput and correctness. + +Reference :[Flink Jdbc Connector](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/jdbc/#lookup-cache) + + +### Known limitations + +* Data type limitations (see above). +* SQL Create table: primary keys can only be set by the `kudu.primary-key-columns` property, using the + `PRIMARY KEY` constraint is not yet possible. +* SQL Create table: range partitioning is not supported. +* When getting a table through the Catalog, NOT NULL and PRIMARY KEY constraints are ignored. All columns + are described as being nullable, and not being primary keys. +* Kudu tables cannot be altered through the catalog other than simple renaming + +## DataStream API + +It is also possible to use the Kudu connector directly from the DataStream API however we +encourage all users to explore the Table API as it provides a lot of useful tooling when working +with Kudu data. + +### Reading tables into a DataStreams + +There are 2 main ways of reading a Kudu Table into a DataStream + +1. Using the `KuduCatalog` and the Table API +2. Using the `KuduRowInputFormat` directly + +Using the `KuduCatalog` and Table API is the recommended way of reading tables as it automatically +guarantees type safety and takes care of configuration of our readers. + +This is how it works in practice: + +```java +StreamTableEnvironment tableEnv = StreamTableEnvironment.create(streamEnv, tableSettings); + +tableEnv.registerCatalog("kudu", new KuduCatalog("master:port")); +tableEnv.useCatalog("kudu"); + +Table table = tableEnv.sqlQuery("SELECT * FROM MyKuduTable"); +DataStream rows = tableEnv.toAppendStream(table, Row.class); +``` +The second way of achieving the same thing is by using the `KuduRowInputFormat` directly. +In this case we have to manually provide all information about our table: + +```java +KuduTableInfo tableInfo = ... +KuduReaderConfig readerConfig = ... +KuduRowInputFormat inputFormat = new KuduRowInputFormat(readerConfig, tableInfo); + +DataStream rowStream = env.createInput(inputFormat, rowTypeInfo); +``` +At the end of the day the `KuduTableSource` is just a convenient wrapper around the `KuduRowInputFormat`. + +### Kudu Sink + +The connector provides a `KuduSink` class that can be used to consume DataStreams +and write the results into a Kudu table. + +The constructor takes 3 or 4 arguments. + +* `KuduWriterConfig` is used to specify the Kudu masters and the flush mode. +* `KuduTableInfo` identifies the table to be written +* `KuduOperationMapper` maps the records coming from the DataStream to a list of Kudu operations. +* `KuduFailureHandler` (optional): If you want to provide your own logic for handling writing failures. + +The example below shows the creation of a sink for Row type records of 3 fields. It Upserts each record. +It is assumed that a Kudu table with columns `col1, col2, col3` called `AlreadyExistingTable` exists. Note that if this were not the case, +we could pass a `KuduTableInfo` as described in the [Catalog - Creating a table](#creating-a-table) section, +and the sink would create the table with the provided configuration. + +```java +KuduWriterConfig writerConfig = KuduWriterConfig.Builder.setMasters(KUDU_MASTERS).build(); + +KuduSink sink = new KuduSink<>( + writerConfig, + KuduTableInfo.forTable("AlreadyExistingTable"), + new RowOperationMapper<>( + new String[]{"col1", "col2", "col3"}, + AbstractSingleOperationMapper.KuduOperation.UPSERT) +) +``` +#### KuduOperationMapper + +This section describes the Operation mapping logic in more detail. + +The connector supports insert, upsert, update, and delete operations. +The operation to be performed can vary dynamically based on the record. +To allow for more flexibility, it is also possible for one record to trigger +0, 1, or more operations. +For the highest level of control, implement the `KuduOperationMapper` interface. + +If one record from the DataStream corresponds to one table operation, +extend the `AbstractSingleOperationMapper` class. An array of column +names must be provided. This must match the Kudu table's schema. + +The `getField` method must be overridden, which extracts the value for the table column whose name is +at the `i`th place in the `columnNames` array. +If the operation is one of (`CREATE, UPSERT, UPDATE, DELETE`) +and doesn't depend on the input record (constant during the life of the sink), it can be set in the constructor +of `AbstractSingleOperationMapper`. +It is also possible to implement your own logic by overriding the +`createBaseOperation` method that returns a Kudu [Operation](https://kudu.apache.org/apidocs/org/apache/kudu/client/Operation.html). + +There are pre-defined operation mappers for Pojo, Flink Row, and Flink Tuple types for constant operation, 1-to-1 sinks. + +* `PojoOperationMapper`: Each table column must correspond to a POJO field + with the same name. The `columnNames` array should contain those fields of the POJO that + are present as table columns (the POJO fields can be a superset of table columns). +* `RowOperationMapper` and `TupleOperationMapper`: the mapping is based on position. The + `i`th field of the Row/Tuple corresponds to the column of the table at the `i`th + position in the `columnNames` array. + +## Building the connector + +The connector can be easily built by using maven: + +``` +cd bahir-flink +mvn clean install +``` +### Running the tests + +The integration tests rely on the Kudu test harness which requires the current user to be able to ssh to localhost. + +This might not work out of the box on some operating systems (such as Mac OS X). +To solve this problem go to *System Preferences/Sharing* and enable Remote login for your user. diff --git a/flink-connector-kudu/pom.xml b/flink-connector-kudu/pom.xml new file mode 100644 index 0000000..22817ff --- /dev/null +++ b/flink-connector-kudu/pom.xml @@ -0,0 +1,97 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connector-kudu-parent + 2.0-SNAPSHOT + + + flink-connector-kudu + Flink : Connectors : Kudu + jar + + + + org.apache.flink + flink-clients + + + + org.apache.flink + flink-streaming-java + + + + org.apache.flink + flink-table-api-java-bridge + + + + org.apache.flink + flink-table-common + + + + org.apache.flink + flink-table-planner-loader + + + + org.apache.flink + flink-table-runtime + + + + org.apache.kudu + kudu-client + + + + org.apache.kudu + kudu-test-utils + test + + + + + org.junit.jupiter + junit-jupiter-migrationsupport + test + + + + org.mockito + mockito-all + test + + + + org.testcontainers + testcontainers + test + + + diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/ColumnSchemasFactory.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/ColumnSchemasFactory.java new file mode 100644 index 0000000..95fe97d --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/ColumnSchemasFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector; + +import org.apache.flink.annotation.PublicEvolving; + +import org.apache.kudu.ColumnSchema; + +import java.io.Serializable; +import java.util.List; + +/** + * Factory for creating {@link ColumnSchema}s to be used when creating a table that does not + * currently exist in Kudu. Usable through {@link KuduTableInfo#createTableIfNotExists}. + * + *

This factory implementation must be Serializable as it will be used directly in the Flink + * sources and sinks. + */ +@PublicEvolving +public interface ColumnSchemasFactory extends Serializable { + + /** + * Creates the columns of the Kudu table that will be used during the createTable operation. + * + * @return List of columns. + */ + List getColumnSchemas(); +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/CreateTableOptionsFactory.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/CreateTableOptionsFactory.java new file mode 100644 index 0000000..08b4f36 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/CreateTableOptionsFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector; + +import org.apache.flink.annotation.PublicEvolving; + +import org.apache.kudu.client.CreateTableOptions; + +import java.io.Serializable; + +/** + * Factory for creating {@link CreateTableOptions} to be used when creating a table that does not + * currently exist in Kudu. Usable through {@link KuduTableInfo#createTableIfNotExists}. + * + *

This factory implementation must be Serializable as it will be used directly in the Flink + * sources and sinks. + */ +@PublicEvolving +public interface CreateTableOptionsFactory extends Serializable { + + /** + * Creates the {@link CreateTableOptions} that will be used during the createTable operation. + * + * @return CreateTableOptions for creating the table. + */ + CreateTableOptions getCreateTableOptions(); +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/KuduFilterInfo.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/KuduFilterInfo.java new file mode 100644 index 0000000..97d8496 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/KuduFilterInfo.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.data.binary.BinaryStringData; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Schema; +import org.apache.kudu.client.KuduPredicate; + +import java.io.Serializable; +import java.util.List; + +/** Helper class that groups necessary data to define a Kudu column filter. */ +@PublicEvolving +public class KuduFilterInfo implements Serializable { + + private String column; + private FilterType type; + private Object value; + + private KuduFilterInfo() {} + + public KuduPredicate toPredicate(Schema schema) { + return toPredicate(schema.getColumn(this.column)); + } + + public KuduPredicate toPredicate(ColumnSchema column) { + KuduPredicate predicate; + switch (this.type) { + case IS_IN: + predicate = KuduPredicate.newInListPredicate(column, (List) this.value); + break; + case IS_NULL: + predicate = KuduPredicate.newIsNullPredicate(column); + break; + case IS_NOT_NULL: + predicate = KuduPredicate.newIsNotNullPredicate(column); + break; + default: + predicate = predicateComparator(column); + break; + } + return predicate; + } + + private KuduPredicate predicateComparator(ColumnSchema column) { + + KuduPredicate.ComparisonOp comparison = this.type.comparator; + + KuduPredicate predicate; + + switch (column.getType()) { + case STRING: + predicate = + KuduPredicate.newComparisonPredicate( + column, + comparison, + (this.value instanceof BinaryStringData) + ? this.value.toString() + : (String) this.value); + break; + case FLOAT: + predicate = + KuduPredicate.newComparisonPredicate( + column, comparison, (float) this.value); + break; + case INT8: + predicate = + KuduPredicate.newComparisonPredicate(column, comparison, (byte) this.value); + break; + case INT16: + predicate = + KuduPredicate.newComparisonPredicate( + column, comparison, (short) this.value); + break; + case INT32: + predicate = + KuduPredicate.newComparisonPredicate(column, comparison, (int) this.value); + break; + case INT64: + predicate = + KuduPredicate.newComparisonPredicate(column, comparison, (long) this.value); + break; + case DOUBLE: + predicate = + KuduPredicate.newComparisonPredicate( + column, comparison, (double) this.value); + break; + case BOOL: + predicate = + KuduPredicate.newComparisonPredicate( + column, comparison, (boolean) this.value); + break; + case UNIXTIME_MICROS: + Long time = (Long) this.value; + predicate = KuduPredicate.newComparisonPredicate(column, comparison, time * 1000); + break; + case BINARY: + predicate = + KuduPredicate.newComparisonPredicate( + column, comparison, (byte[]) this.value); + break; + default: + throw new IllegalArgumentException("Illegal var type: " + column.getType()); + } + return predicate; + } + + /** Possible filter types and their appropriate Kudu comparison operator. */ + public enum FilterType { + GREATER(KuduPredicate.ComparisonOp.GREATER), + GREATER_EQUAL(KuduPredicate.ComparisonOp.GREATER_EQUAL), + EQUAL(KuduPredicate.ComparisonOp.EQUAL), + LESS(KuduPredicate.ComparisonOp.LESS), + LESS_EQUAL(KuduPredicate.ComparisonOp.LESS_EQUAL), + IS_NOT_NULL(null), + IS_NULL(null), + IS_IN(null); + + final KuduPredicate.ComparisonOp comparator; + + FilterType(KuduPredicate.ComparisonOp comparator) { + this.comparator = comparator; + } + } + + /** KuduFilterInfo builder. */ + public static class Builder { + private final KuduFilterInfo filter; + + private Builder(String column) { + this.filter = new KuduFilterInfo(); + this.filter.column = column; + } + + public static Builder create(String column) { + return new Builder(column); + } + + public Builder greaterThan(Object value) { + return filter(FilterType.GREATER, value); + } + + public Builder lessThan(Object value) { + return filter(FilterType.LESS, value); + } + + public Builder equalTo(Object value) { + return filter(FilterType.EQUAL, value); + } + + public Builder greaterOrEqualTo(Object value) { + return filter(FilterType.GREATER_EQUAL, value); + } + + public Builder lessOrEqualTo(Object value) { + return filter(FilterType.LESS_EQUAL, value); + } + + public Builder isNotNull() { + return filter(FilterType.IS_NOT_NULL, null); + } + + public Builder isNull() { + return filter(FilterType.IS_NULL, null); + } + + public Builder isIn(List values) { + return filter(FilterType.IS_IN, values); + } + + public Builder filter(FilterType type, Object value) { + this.filter.type = type; + this.filter.value = value; + return this; + } + + public KuduFilterInfo build() { + return filter; + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/KuduTableInfo.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/KuduTableInfo.java new file mode 100644 index 0000000..f8ad4fa --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/KuduTableInfo.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector; + +import org.apache.flink.annotation.PublicEvolving; + +import org.apache.kudu.Schema; +import org.apache.kudu.client.CreateTableOptions; + +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Describes which table should be used in sources and sinks along with specifications on how to + * create it if it does not exist. + * + *

For sources and sinks reading from already existing tables, simply use @{@link + * KuduTableInfo#forTable(String)} and if you want the system to create the table if it does not + * exist you need to specify the column and options factories through {@link + * KuduTableInfo#createTableIfNotExists} + */ +@PublicEvolving +public class KuduTableInfo implements Serializable { + + private String name; + private CreateTableOptionsFactory createTableOptionsFactory = null; + private ColumnSchemasFactory schemasFactory = null; + + private KuduTableInfo(String name) { + this.name = checkNotNull(name); + } + + /** + * Creates a new {@link KuduTableInfo} that is sufficient for reading/writing to existing Kudu + * Tables. For creating new tables call {@link #createTableIfNotExists} afterward. + * + * @param name Table name in Kudu + * @return KuduTableInfo for the given table name + */ + public static KuduTableInfo forTable(String name) { + return new KuduTableInfo(name); + } + + /** + * Defines table parameters to be used when creating the Kudu table if it does not exist (read + * or write). + * + * @param schemasFactory factory for defining columns + * @param createTableOptionsFactory factory for defining create table options + * @return KuduTableInfo that will create tables that does not exist with the given settings. + */ + public KuduTableInfo createTableIfNotExists( + ColumnSchemasFactory schemasFactory, + CreateTableOptionsFactory createTableOptionsFactory) { + this.createTableOptionsFactory = checkNotNull(createTableOptionsFactory); + this.schemasFactory = checkNotNull(schemasFactory); + return this; + } + + /** + * Returns the {@link Schema} of the table. Only works if {@link #createTableIfNotExists} was + * specified otherwise throws an error. + * + * @return Schema of the target table. + */ + public Schema getSchema() { + if (!getCreateTableIfNotExists()) { + throw new RuntimeException( + "Cannot access schema for KuduTableInfo. Use createTableIfNotExists to specify the columns."); + } + + return new Schema(schemasFactory.getColumnSchemas()); + } + + /** @return Name of the table. */ + public String getName() { + return name; + } + + /** @return True if table creation is enabled if target table does not exist. */ + public boolean getCreateTableIfNotExists() { + return createTableOptionsFactory != null; + } + + /** @return CreateTableOptions if {@link #createTableIfNotExists} was specified. */ + public CreateTableOptions getCreateTableOptions() { + if (!getCreateTableIfNotExists()) { + throw new RuntimeException( + "Cannot access CreateTableOptions for KuduTableInfo. Use createTableIfNotExists to specify."); + } + return createTableOptionsFactory.getCreateTableOptions(); + } + + @Override + public int hashCode() { + return Objects.hash(name); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KuduTableInfo that = (KuduTableInfo) o; + return Objects.equals(this.name, that.name); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultConverter.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultConverter.java new file mode 100644 index 0000000..ee07f9f --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultConverter.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.converter; + +import org.apache.flink.annotation.Internal; + +import org.apache.kudu.client.RowResult; + +import java.io.Serializable; + +/** + * Interface to convert Kudu RowResult objects to internal Flink row representations. + * + * @param Target Flink representation type + */ +@Internal +public interface RowResultConverter extends Serializable { + + /** + * Convert Kudu RowResult to the corresponding format. + * + * @param row Kudu RowResult Type + * @return {@link T} + */ + T convert(RowResult row); +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultRowConverter.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultRowConverter.java new file mode 100644 index 0000000..dc2f814 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultRowConverter.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.converter; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; + +import org.apache.kudu.Schema; +import org.apache.kudu.client.RowResult; + +/** Transforms a Kudu RowResult object into a Flink Row object. */ +@Internal +public class RowResultRowConverter implements RowResultConverter { + @Override + public Row convert(RowResult row) { + Schema schema = row.getColumnProjection(); + + Row values = new Row(schema.getColumnCount()); + schema.getColumns() + .forEach( + column -> { + String name = column.getName(); + int pos = schema.getColumnIndex(name); + if (row.isNull(name)) { + return; + } + values.setField(pos, row.getObject(name)); + }); + return values; + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultRowDataConverter.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultRowDataConverter.java new file mode 100644 index 0000000..07002a7 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/converter/RowResultRowDataConverter.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.converter; + +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; + +import org.apache.kudu.Schema; +import org.apache.kudu.Type; +import org.apache.kudu.client.RowResult; + +import java.math.BigDecimal; +import java.util.Objects; + +/** Transforms a Kudu RowResult object into a Flink RowData object. */ +public class RowResultRowDataConverter implements RowResultConverter { + @Override + public RowData convert(RowResult row) { + Schema schema = row.getColumnProjection(); + GenericRowData values = new GenericRowData(schema.getColumnCount()); + schema.getColumns() + .forEach( + column -> { + String name = column.getName(); + Type type = column.getType(); + int pos = schema.getColumnIndex(name); + if (Objects.isNull(type)) { + throw new IllegalArgumentException("columnName:" + name); + } + if (row.isNull(name)) { + return; + } + switch (type) { + case DECIMAL: + BigDecimal decimal = row.getDecimal(name); + values.setField( + pos, + DecimalData.fromBigDecimal( + decimal, decimal.precision(), decimal.scale())); + break; + case UNIXTIME_MICROS: + values.setField( + pos, + TimestampData.fromTimestamp(row.getTimestamp(name))); + break; + case DOUBLE: + values.setField(pos, row.getDouble(name)); + break; + case STRING: + Object value = row.getObject(name); + values.setField( + pos, + StringData.fromString( + Objects.nonNull(value) + ? value.toString() + : "")); + break; + case BINARY: + values.setField(pos, row.getBinary(name)); + break; + case FLOAT: + values.setField(pos, row.getFloat(name)); + break; + case INT64: + values.setField(pos, row.getLong(name)); + break; + case INT32: + case INT16: + case INT8: + values.setField(pos, row.getInt(name)); + break; + case BOOL: + values.setField(pos, row.getBoolean(name)); + break; + default: + throw new IllegalArgumentException( + "columnName:" + + name + + ",type:" + + type.getName() + + "not support!"); + } + }); + return values; + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/failure/DefaultKuduFailureHandler.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/failure/DefaultKuduFailureHandler.java new file mode 100644 index 0000000..785b4e9 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/failure/DefaultKuduFailureHandler.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.failure; + +import org.apache.kudu.client.RowError; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +/** Default failure handling logic that doesn't do any handling but throws an error. */ +public class DefaultKuduFailureHandler implements KuduFailureHandler { + + @Override + public void onFailure(List failure) throws IOException { + String errors = + failure.stream() + .map(error -> error.toString() + System.lineSeparator()) + .collect(Collectors.joining()); + + throw new IOException("Error while sending value. \n " + errors); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/failure/KuduFailureHandler.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/failure/KuduFailureHandler.java new file mode 100644 index 0000000..6da1631 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/failure/KuduFailureHandler.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.failure; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.kudu.connector.writer.KuduWriter; + +import org.apache.kudu.client.RowError; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + +/** + * Custom handling logic for errors resulting from trying to execute Kudu operations in the {@link + * KuduWriter}. + */ +@PublicEvolving +public interface KuduFailureHandler extends Serializable { + + /** + * Handle a failure. + * + * @param failure the cause of failure + * @throws IOException if the sink should fail on this failure, the implementation should + * rethrow the throwable or a custom one + */ + void onFailure(List failure) throws IOException; + + /** + * Handle a ClassCastException. Default implementation rethrows the exception. + * + * @param e the cause of failure + * @throws IOException if the casting failed + */ + default void onTypeMismatch(ClassCastException e) throws IOException { + throw new IOException("Class casting failed \n", e); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduInputSplit.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduInputSplit.java new file mode 100644 index 0000000..c45fcb9 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduInputSplit.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.LocatableInputSplit; + +/** Kudu input split that holds a scan token to identify the split position. */ +@Internal +public class KuduInputSplit extends LocatableInputSplit { + + private static final long serialVersionUID = 1L; + + private final byte[] scanToken; + + /** + * Creates a new KuduInputSplit. + * + * @param splitNumber the number of the input split + * @param hostnames the names of the hosts storing the data this input split refers to + */ + public KuduInputSplit(byte[] scanToken, final int splitNumber, final String[] hostnames) { + super(splitNumber, hostnames); + + this.scanToken = scanToken; + } + + public byte[] getScanToken() { + return scanToken; + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReader.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReader.java new file mode 100644 index 0000000..0a8f0b2 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReader.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kudu.connector.KuduFilterInfo; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.converter.RowResultConverter; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduSession; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.LocatedTablet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** Reader to read data from a Kudu table. */ +@Internal +public class KuduReader implements AutoCloseable { + + private final Logger log = LoggerFactory.getLogger(getClass()); + + private final KuduTableInfo tableInfo; + private final KuduReaderConfig readerConfig; + private List tableFilters; + private List tableProjections; + private final RowResultConverter rowResultConverter; + + private final transient KuduClient client; + private final transient KuduSession session; + private final transient KuduTable table; + + public KuduReader( + KuduTableInfo tableInfo, + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter) + throws IOException { + this(tableInfo, readerConfig, rowResultConverter, new ArrayList<>(), null); + } + + public KuduReader( + KuduTableInfo tableInfo, + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + List tableFilters) + throws IOException { + this(tableInfo, readerConfig, rowResultConverter, tableFilters, null); + } + + public KuduReader( + KuduTableInfo tableInfo, + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + List tableFilters, + List tableProjections) + throws IOException { + this.tableInfo = tableInfo; + this.readerConfig = readerConfig; + this.tableFilters = tableFilters; + this.tableProjections = tableProjections; + this.rowResultConverter = rowResultConverter; + this.client = obtainClient(); + this.session = obtainSession(); + this.table = obtainTable(); + } + + public void setTableFilters(List tableFilters) { + this.tableFilters = tableFilters; + } + + public void setTableProjections(List tableProjections) { + this.tableProjections = tableProjections; + } + + private KuduClient obtainClient() { + return new KuduClient.KuduClientBuilder(readerConfig.getMasters()).build(); + } + + private KuduSession obtainSession() { + return client.newSession(); + } + + private KuduTable obtainTable() throws IOException { + String tableName = tableInfo.getName(); + if (client.tableExists(tableName)) { + return client.openTable(tableName); + } + if (tableInfo.getCreateTableIfNotExists()) { + return client.createTable( + tableName, tableInfo.getSchema(), tableInfo.getCreateTableOptions()); + } + throw new RuntimeException("Table " + tableName + " does not exist."); + } + + public KuduReaderIterator scanner(byte[] token) throws IOException { + return new KuduReaderIterator<>( + KuduScanToken.deserializeIntoScanner(token, client), rowResultConverter); + } + + public List scanTokens( + List tableFilters, List tableProjections, Integer rowLimit) { + KuduScanToken.KuduScanTokenBuilder tokenBuilder = client.newScanTokenBuilder(table); + + if (tableProjections != null) { + tokenBuilder.setProjectedColumnNames(tableProjections); + } + + if (CollectionUtils.isNotEmpty(tableFilters)) { + tableFilters.stream() + .map(filter -> filter.toPredicate(table.getSchema())) + .forEach(tokenBuilder::addPredicate); + } + + if (rowLimit != null && rowLimit > 0) { + tokenBuilder.limit(rowLimit); + } + + return tokenBuilder.build(); + } + + public KuduInputSplit[] createInputSplits(int minNumSplits) throws IOException { + + List tokens = + scanTokens(tableFilters, tableProjections, readerConfig.getRowLimit()); + + KuduInputSplit[] splits = new KuduInputSplit[tokens.size()]; + + for (int i = 0; i < tokens.size(); i++) { + KuduScanToken token = tokens.get(i); + + List locations = new ArrayList<>(token.getTablet().getReplicas().size()); + + for (LocatedTablet.Replica replica : token.getTablet().getReplicas()) { + locations.add(getLocation(replica.getRpcHost(), replica.getRpcPort())); + } + + KuduInputSplit split = + new KuduInputSplit( + token.serialize(), i, locations.toArray(new String[locations.size()])); + splits[i] = split; + } + + if (splits.length < minNumSplits) { + log.warn( + " The minimum desired number of splits with your configured parallelism level " + + "is {}. Current kudu splits = {}. {} instances will remain idle.", + minNumSplits, + splits.length, + (minNumSplits - splits.length)); + } + + return splits; + } + + private String getLocation(String host, Integer port) { + return host + ":" + port; + } + + @Override + public void close() throws IOException { + try { + if (session != null) { + session.close(); + } + } catch (KuduException e) { + log.error("Error while closing session.", e); + } + try { + if (client != null) { + client.close(); + } + } catch (Exception e) { + log.error("Error while closing client.", e); + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReaderConfig.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReaderConfig.java new file mode 100644 index 0000000..71d56de --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReaderConfig.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.reader; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.kudu.format.KuduRowInputFormat; + +import org.apache.commons.lang3.builder.ToStringBuilder; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Configuration used by {@link KuduRowInputFormat}. Specifies connection and other necessary + * properties. + */ +@PublicEvolving +public class KuduReaderConfig implements Serializable { + + private final String masters; + private final int rowLimit; + + private KuduReaderConfig(String masters, int rowLimit) { + + this.masters = checkNotNull(masters, "Kudu masters cannot be null"); + this.rowLimit = checkNotNull(rowLimit, "Kudu rowLimit cannot be null"); + } + + public String getMasters() { + return masters; + } + + public int getRowLimit() { + return rowLimit; + } + + @Override + public String toString() { + return new ToStringBuilder(this) + .append("masters", masters) + .append("rowLimit", rowLimit) + .toString(); + } + + /** Builder for the {@link KuduReaderConfig}. */ + public static class Builder { + private static final int DEFAULT_ROW_LIMIT = 0; + + private final String masters; + private final int rowLimit; + + private Builder(String masters) { + this(masters, DEFAULT_ROW_LIMIT); + } + + private Builder(String masters, Integer rowLimit) { + this.masters = masters; + this.rowLimit = rowLimit; + } + + public static Builder setMasters(String masters) { + return new Builder(masters); + } + + public Builder setRowLimit(int rowLimit) { + return new Builder(masters, rowLimit); + } + + public KuduReaderConfig build() { + return new KuduReaderConfig(masters, rowLimit); + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReaderIterator.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReaderIterator.java new file mode 100644 index 0000000..0253b21 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/reader/KuduReaderIterator.java @@ -0,0 +1,68 @@ +/* + * Licensed serialize the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file serialize You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed serialize 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.flink.connector.kudu.connector.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kudu.connector.converter.RowResultConverter; + +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.io.Serializable; + +/** An iterator that helps to iterate on Kudu record rows. */ +@Internal +public class KuduReaderIterator implements Serializable { + + private final KuduScanner scanner; + private final RowResultConverter rowResultConverter; + private RowResultIterator rowIterator; + + public KuduReaderIterator(KuduScanner scanner, RowResultConverter rowResultConverter) + throws KuduException { + this.scanner = scanner; + this.rowResultConverter = rowResultConverter; + nextRows(); + } + + public void close() throws KuduException { + scanner.close(); + } + + public boolean hasNext() throws KuduException { + if (rowIterator.hasNext()) { + return true; + } else if (scanner.hasMoreRows()) { + nextRows(); + return true; + } else { + return false; + } + } + + public T next() { + RowResult row = this.rowIterator.next(); + return rowResultConverter.convert(row); + } + + private void nextRows() throws KuduException { + this.rowIterator = scanner.nextRows(); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/AbstractSingleOperationMapper.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/AbstractSingleOperationMapper.java new file mode 100644 index 0000000..9a29207 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/AbstractSingleOperationMapper.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.writer; + +import org.apache.flink.annotation.PublicEvolving; + +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.Operation; +import org.apache.kudu.client.PartialRow; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * Base implementation for {@link KuduOperationMapper}s that have one-to-one input to Kudu operation + * mapping. It requires a fixed table schema to be provided at construction time and only requires + * users to implement a getter for a specific column index (relative to the ones provided in the + * constructor).
+ * Supports both fixed operation type per record by specifying the {@link KuduOperation} or a custom + * implementation for creating the base {@link Operation} through the {@link + * #createBaseOperation(Object, KuduTable)} method. + * + * @param Input type + */ +@PublicEvolving +public abstract class AbstractSingleOperationMapper implements KuduOperationMapper { + + protected final String[] columnNames; + private final KuduOperation operation; + + protected AbstractSingleOperationMapper(String[] columnNames) { + this(columnNames, null); + } + + public AbstractSingleOperationMapper(String[] columnNames, KuduOperation operation) { + this.columnNames = columnNames; + this.operation = operation; + } + + /** + * Returns the object corresponding to the given column index. + * + * @param input Input element + * @param i Column index + * @return Column value + */ + public abstract Object getField(T input, int i); + + public Optional createBaseOperation(T input, KuduTable table) { + if (operation == null) { + throw new UnsupportedOperationException( + "createBaseOperation must be overridden if no operation specified in constructor"); + } + switch (operation) { + case INSERT: + return Optional.of(table.newInsert()); + case UPDATE: + return Optional.of(table.newUpdate()); + case UPSERT: + return Optional.of(table.newUpsert()); + case DELETE: + return Optional.of(table.newDelete()); + default: + throw new RuntimeException("Unknown operation " + operation); + } + } + + @Override + public List createOperations(T input, KuduTable table) { + Optional operationOpt = createBaseOperation(input, table); + if (!operationOpt.isPresent()) { + return Collections.emptyList(); + } + + Operation operation = operationOpt.get(); + PartialRow partialRow = operation.getRow(); + + for (int i = 0; i < columnNames.length; i++) { + partialRow.addObject(columnNames[i], getField(input, i)); + } + + return Collections.singletonList(operation); + } + + /** Kudu operation types. */ + public enum KuduOperation { + INSERT, + UPDATE, + UPSERT, + DELETE + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduOperationMapper.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduOperationMapper.java new file mode 100644 index 0000000..1f1be1c --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduOperationMapper.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.writer; + +import org.apache.flink.annotation.PublicEvolving; + +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.Operation; + +import java.io.Serializable; +import java.util.List; + +/** + * Encapsulates the logic of mapping input records (of a DataStream) to operations executed in Kudu. + * By allowing to return a list of operations we give flexibility to the implementers to provide + * more sophisticated logic. + * + * @param Type of the input data + */ +@PublicEvolving +public interface KuduOperationMapper extends Serializable { + + /** + * Create a list of operations to be executed by the {@link KuduWriter} for the current input. + * + * @param input input element + * @param table table for which the operations should be created + * @return List of operations to be executed on the table + */ + List createOperations(T input, KuduTable table); +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduWriter.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduWriter.java new file mode 100644 index 0000000..318ba2e --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduWriter.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.writer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.failure.DefaultKuduFailureHandler; +import org.apache.flink.connector.kudu.connector.failure.KuduFailureHandler; + +import org.apache.kudu.client.DeleteTableResponse; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduSession; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.Operation; +import org.apache.kudu.client.OperationResponse; +import org.apache.kudu.client.RowError; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +/** Writer to write data to a Kudu table. */ +@Internal +public class KuduWriter implements AutoCloseable { + + private final Logger log = LoggerFactory.getLogger(getClass()); + + private final KuduTableInfo tableInfo; + private final KuduWriterConfig writerConfig; + private final KuduFailureHandler failureHandler; + private final KuduOperationMapper operationMapper; + + private transient KuduClient client; + private transient KuduSession session; + private transient KuduTable table; + + public KuduWriter( + KuduTableInfo tableInfo, + KuduWriterConfig writerConfig, + KuduOperationMapper operationMapper) + throws IOException { + this(tableInfo, writerConfig, operationMapper, new DefaultKuduFailureHandler()); + } + + public KuduWriter( + KuduTableInfo tableInfo, + KuduWriterConfig writerConfig, + KuduOperationMapper operationMapper, + KuduFailureHandler failureHandler) + throws IOException { + this.tableInfo = tableInfo; + this.writerConfig = writerConfig; + this.failureHandler = failureHandler; + + this.client = obtainClient(); + this.session = obtainSession(); + this.table = obtainTable(); + this.operationMapper = operationMapper; + } + + private KuduClient obtainClient() { + return new KuduClient.KuduClientBuilder(writerConfig.getMasters()).build(); + } + + private KuduSession obtainSession() { + KuduSession session = client.newSession(); + session.setFlushMode(writerConfig.getFlushMode()); + session.setTimeoutMillis(writerConfig.getOperationTimeout()); + session.setMutationBufferSpace(writerConfig.getMaxBufferSize()); + session.setFlushInterval(writerConfig.getFlushInterval()); + session.setIgnoreAllDuplicateRows(writerConfig.isIgnoreDuplicate()); + session.setIgnoreAllNotFoundRows(writerConfig.isIgnoreNotFound()); + return session; + } + + private KuduTable obtainTable() throws IOException { + String tableName = tableInfo.getName(); + if (client.tableExists(tableName)) { + return client.openTable(tableName); + } + if (tableInfo.getCreateTableIfNotExists()) { + return client.createTable( + tableName, tableInfo.getSchema(), tableInfo.getCreateTableOptions()); + } + throw new RuntimeException("Table " + tableName + " does not exist."); + } + + public void write(T input) throws IOException { + checkAsyncErrors(); + + for (Operation operation : operationMapper.createOperations(input, table)) { + checkErrors(session.apply(operation)); + } + } + + public void flushAndCheckErrors() throws IOException { + checkAsyncErrors(); + flush(); + checkAsyncErrors(); + } + + @VisibleForTesting + public DeleteTableResponse deleteTable() throws IOException { + String tableName = table.getName(); + return client.deleteTable(tableName); + } + + @Override + public void close() throws IOException { + try { + flushAndCheckErrors(); + } finally { + try { + if (session != null) { + session.close(); + } + } catch (Exception e) { + log.error("Error while closing session.", e); + } + try { + if (client != null) { + client.close(); + } + } catch (Exception e) { + log.error("Error while closing client.", e); + } + } + } + + private void flush() throws IOException { + session.flush(); + } + + private void checkErrors(OperationResponse response) throws IOException { + if (response != null && response.hasRowError()) { + failureHandler.onFailure(Arrays.asList(response.getRowError())); + } else { + checkAsyncErrors(); + } + } + + private void checkAsyncErrors() throws IOException { + if (session.countPendingErrors() == 0) { + return; + } + + List errors = Arrays.asList(session.getPendingErrors().getRowErrors()); + failureHandler.onFailure(errors); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduWriterConfig.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduWriterConfig.java new file mode 100644 index 0000000..38db17a --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/KuduWriterConfig.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.writer; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.kudu.format.KuduOutputFormat; +import org.apache.flink.connector.kudu.streaming.KuduSink; + +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.kudu.client.AsyncKuduClient; + +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.kudu.client.SessionConfiguration.FlushMode; + +/** + * Configuration used by {@link KuduSink} and {@link KuduOutputFormat}. Specifies connection and + * other necessary properties. + */ +@PublicEvolving +public class KuduWriterConfig implements Serializable { + + private final String masters; + private final FlushMode flushMode; + private final long operationTimeout; + private int maxBufferSize; + private int flushInterval; + private boolean ignoreNotFound; + private boolean ignoreDuplicate; + + private KuduWriterConfig( + String masters, + FlushMode flushMode, + long operationTimeout, + int maxBufferSize, + int flushInterval, + boolean ignoreNotFound, + boolean ignoreDuplicate) { + + this.masters = checkNotNull(masters, "Kudu masters cannot be null"); + this.flushMode = checkNotNull(flushMode, "Kudu flush mode cannot be null"); + this.operationTimeout = operationTimeout; + this.maxBufferSize = maxBufferSize; + this.flushInterval = flushInterval; + this.ignoreNotFound = ignoreNotFound; + this.ignoreDuplicate = ignoreDuplicate; + } + + public String getMasters() { + return masters; + } + + public FlushMode getFlushMode() { + return flushMode; + } + + public long getOperationTimeout() { + return operationTimeout; + } + + public int getMaxBufferSize() { + return maxBufferSize; + } + + public int getFlushInterval() { + return flushInterval; + } + + public boolean isIgnoreNotFound() { + return ignoreNotFound; + } + + public boolean isIgnoreDuplicate() { + return ignoreDuplicate; + } + + @Override + public String toString() { + return new ToStringBuilder(this) + .append("masters", masters) + .append("flushMode", flushMode) + .toString(); + } + + /** Builder for the {@link KuduWriterConfig}. */ + public static class Builder { + private String masters; + private FlushMode flushMode = FlushMode.AUTO_FLUSH_BACKGROUND; + // Reference from AsyncKuduClientBuilder defaultOperationTimeoutMs. + private long timeout = AsyncKuduClient.DEFAULT_OPERATION_TIMEOUT_MS; + // Reference from AsyncKuduSession mutationBufferMaxOps 1000. + private int maxBufferSize = 1000; + // Reference from AsyncKuduSession flushIntervalMillis 1000. + private int flushInterval = 1000; + // Reference from AsyncKuduSession ignoreAllNotFoundRows false. + private boolean ignoreNotFound = false; + // Reference from AsyncKuduSession ignoreAllDuplicateRows false. + private boolean ignoreDuplicate = false; + + private Builder(String masters) { + this.masters = masters; + } + + public static Builder setMasters(String masters) { + return new Builder(masters); + } + + public Builder setConsistency(FlushMode flushMode) { + this.flushMode = flushMode; + return this; + } + + public Builder setEventualConsistency() { + return setConsistency(FlushMode.AUTO_FLUSH_BACKGROUND); + } + + public Builder setStrongConsistency() { + return setConsistency(FlushMode.AUTO_FLUSH_SYNC); + } + + public Builder setMaxBufferSize(int maxBufferSize) { + this.maxBufferSize = maxBufferSize; + return this; + } + + public Builder setFlushInterval(int flushInterval) { + this.flushInterval = flushInterval; + return this; + } + + public Builder setOperationTimeout(long timeout) { + this.timeout = timeout; + return this; + } + + public Builder setIgnoreNotFound(boolean ignoreNotFound) { + this.ignoreNotFound = ignoreNotFound; + return this; + } + + public Builder setIgnoreDuplicate(boolean ignoreDuplicate) { + this.ignoreDuplicate = ignoreDuplicate; + return this; + } + + public KuduWriterConfig build() { + return new KuduWriterConfig( + masters, + flushMode, + timeout, + maxBufferSize, + flushInterval, + ignoreNotFound, + ignoreDuplicate); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + masters, + flushMode, + timeout, + maxBufferSize, + flushInterval, + ignoreNotFound, + ignoreDuplicate); + return result; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder that = (Builder) o; + return Objects.equals(masters, that.masters) + && Objects.equals(flushMode, that.flushMode) + && Objects.equals(timeout, that.timeout) + && Objects.equals(maxBufferSize, that.maxBufferSize) + && Objects.equals(flushInterval, that.flushInterval) + && Objects.equals(ignoreNotFound, that.ignoreNotFound) + && Objects.equals(ignoreDuplicate, that.ignoreDuplicate); + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/PojoOperationMapper.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/PojoOperationMapper.java new file mode 100644 index 0000000..226cecf --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/PojoOperationMapper.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.writer; + +import org.apache.flink.annotation.PublicEvolving; + +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** Logic to map a given POJO to a Kudu-compatible format. */ +@PublicEvolving +public class PojoOperationMapper extends AbstractSingleOperationMapper { + + private final Field[] fields; + + public PojoOperationMapper(Class pojoClass, String[] columnNames, KuduOperation operation) { + super(columnNames, operation); + fields = initFields(pojoClass, columnNames); + } + + public static List getAllFields(List fields, Class type) { + fields.addAll(Arrays.asList(type.getDeclaredFields())); + + if (type.getSuperclass() != null) { + getAllFields(fields, type.getSuperclass()); + } + + return fields; + } + + private Field[] initFields(Class pojoClass, String[] columnNames) { + Map allFields = new HashMap<>(); + getAllFields(new ArrayList<>(), pojoClass) + .forEach( + f -> { + if (!allFields.containsKey(f.getName())) { + allFields.put(f.getName(), f); + } + }); + + Field[] fields = new Field[columnNames.length]; + + for (int i = 0; i < columnNames.length; i++) { + Field f = allFields.get(columnNames[i]); + if (f == null) { + throw new RuntimeException( + "Cannot find field " + + columnNames[i] + + ". List of detected fields: " + + allFields.keySet()); + } + f.setAccessible(true); + fields[i] = f; + } + + return fields; + } + + @Override + public Object getField(T input, int i) { + try { + return fields[i].get(input); + } catch (IllegalAccessException e) { + throw new RuntimeException("This is a bug"); + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/RowDataUpsertOperationMapper.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/RowDataUpsertOperationMapper.java new file mode 100644 index 0000000..5b0f56a --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/RowDataUpsertOperationMapper.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.writer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; + +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.Operation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Optional; + +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision; + +/** Logic to map Flink UPSERT RowData to a Kudu-compatible format. */ +@Internal +public class RowDataUpsertOperationMapper extends AbstractSingleOperationMapper { + + private static final Logger LOG = LoggerFactory.getLogger(RowDataUpsertOperationMapper.class); + + private static final int MIN_TIME_PRECISION = 0; + private static final int MAX_TIME_PRECISION = 3; + private static final int MIN_TIMESTAMP_PRECISION = 0; + private static final int MAX_TIMESTAMP_PRECISION = 6; + + private LogicalType[] logicalTypes; + + public RowDataUpsertOperationMapper(TableSchema schema) { + super(schema.getFieldNames()); + logicalTypes = + Arrays.stream(schema.getFieldDataTypes()) + .map(DataType::getLogicalType) + .toArray(LogicalType[]::new); + } + + @Override + public Object getField(RowData input, int i) { + return getFieldValue(input, i); + } + + public Object getFieldValue(RowData input, int i) { + if (input == null || input.isNullAt(i)) { + return null; + } + LogicalType fieldType = logicalTypes[i]; + switch (fieldType.getTypeRoot()) { + case CHAR: + case VARCHAR: + { + StringData data = input.getString(i); + if (data != null) { + return data.toString(); + } + return null; + } + case BOOLEAN: + return input.getBoolean(i); + case BINARY: + case VARBINARY: + return input.getBinary(i); + case DECIMAL: + { + DecimalType decimalType = (DecimalType) fieldType; + final int precision = decimalType.getPrecision(); + final int scale = decimalType.getScale(); + DecimalData data = input.getDecimal(i, precision, scale); + if (data != null) { + return data.toBigDecimal(); + } else { + return null; + } + } + case TINYINT: + return input.getByte(i); + case SMALLINT: + return input.getShort(i); + case INTEGER: + case DATE: + case INTERVAL_YEAR_MONTH: + return input.getInt(i); + case TIME_WITHOUT_TIME_ZONE: + final int timePrecision = getPrecision(fieldType); + if (timePrecision < MIN_TIME_PRECISION || timePrecision > MAX_TIME_PRECISION) { + throw new UnsupportedOperationException( + String.format( + "The precision %s of TIME type is out of the range [%s, %s] supported by " + + "kudu connector", + timePrecision, MIN_TIME_PRECISION, MAX_TIME_PRECISION)); + } + return input.getInt(i); + case BIGINT: + case INTERVAL_DAY_TIME: + return input.getLong(i); + case FLOAT: + return input.getFloat(i); + case DOUBLE: + return input.getDouble(i); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + final int timestampPrecision = getPrecision(fieldType); + if (timestampPrecision < MIN_TIMESTAMP_PRECISION + || timestampPrecision > MAX_TIMESTAMP_PRECISION) { + throw new UnsupportedOperationException( + String.format( + "The precision %s of TIMESTAMP type is out of the range [%s, %s] supported " + + "by kudu connector", + timestampPrecision, + MIN_TIMESTAMP_PRECISION, + MAX_TIMESTAMP_PRECISION)); + } + return input.getTimestamp(i, timestampPrecision).toTimestamp(); + default: + throw new UnsupportedOperationException("Unsupported type: " + fieldType); + } + } + + @Override + public Optional createBaseOperation(RowData input, KuduTable table) { + Optional operation = Optional.empty(); + switch (input.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + operation = Optional.of(table.newUpsert()); + break; + case DELETE: + operation = Optional.of(table.newDelete()); + break; + } + return operation; + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/RowOperationMapper.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/RowOperationMapper.java new file mode 100644 index 0000000..6b2e073 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/RowOperationMapper.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.writer; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.types.Row; + +/** Logic to map Flink Row to a "Kudu upsert"-compatible format. */ +@PublicEvolving +public class RowOperationMapper extends AbstractSingleOperationMapper { + + protected RowOperationMapper(String[] columnNames) { + super(columnNames); + } + + public RowOperationMapper(String[] columnNames, KuduOperation operation) { + super(columnNames, operation); + } + + @Override + public Object getField(Row input, int i) { + return input.getField(i); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/TupleOperationMapper.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/TupleOperationMapper.java new file mode 100644 index 0000000..bb80a14 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/connector/writer/TupleOperationMapper.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector.writer; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.tuple.Tuple; + +/** Logic to map a Flink Tuple to a Kudu-compatible format. */ +@PublicEvolving +public class TupleOperationMapper extends AbstractSingleOperationMapper { + + protected TupleOperationMapper(String[] columnNames) { + super(columnNames); + } + + public TupleOperationMapper(String[] columnNames, KuduOperation operation) { + super(columnNames, operation); + } + + @Override + public Object getField(T input, int i) { + return input.getField(i); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/AbstractKuduInputFormat.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/AbstractKuduInputFormat.java new file mode 100644 index 0000000..f1d8e6b --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/AbstractKuduInputFormat.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.format; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.io.LocatableInputSplitAssigner; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kudu.connector.KuduFilterInfo; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.converter.RowResultConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduInputSplit; +import org.apache.flink.connector.kudu.connector.reader.KuduReader; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderIterator; +import org.apache.flink.connector.kudu.table.KuduCatalog; +import org.apache.flink.core.io.InputSplitAssigner; + +import org.apache.kudu.client.KuduException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Input format for reading the contents of a Kudu table (defined by the provided {@link + * KuduTableInfo}) in both batch and stream programs. Rows of the Kudu table are mapped to {@link T} + * instances that can converted to other data types by the user later if necessary. + * + *

For programmatic access to the schema of the input rows users can use the {@link KuduCatalog} + * or overwrite the column order manually by providing a list of projected column names. + */ +@PublicEvolving +public abstract class AbstractKuduInputFormat extends RichInputFormat + implements ResultTypeQueryable { + + private final Logger log = LoggerFactory.getLogger(getClass()); + + private final KuduReaderConfig readerConfig; + private final KuduTableInfo tableInfo; + private final List tableFilters; + private final List tableProjections; + private final RowResultConverter rowResultConverter; + private boolean endReached; + private transient KuduReader kuduReader; + private transient KuduReaderIterator resultIterator; + + public AbstractKuduInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo) { + this(readerConfig, rowResultConverter, tableInfo, new ArrayList<>(), null); + } + + public AbstractKuduInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo, + List tableProjections) { + this(readerConfig, rowResultConverter, tableInfo, new ArrayList<>(), tableProjections); + } + + public AbstractKuduInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo, + List tableFilters, + List tableProjections) { + + this.readerConfig = checkNotNull(readerConfig, "readerConfig could not be null"); + this.rowResultConverter = + checkNotNull(rowResultConverter, "rowResultConvertor could not be null"); + this.tableInfo = checkNotNull(tableInfo, "tableInfo could not be null"); + this.tableFilters = checkNotNull(tableFilters, "tableFilters could not be null"); + this.tableProjections = tableProjections; + + this.endReached = false; + } + + @Override + public void configure(Configuration parameters) {} + + @Override + public void open(KuduInputSplit split) throws IOException { + endReached = false; + startKuduReader(); + + resultIterator = kuduReader.scanner(split.getScanToken()); + } + + private void startKuduReader() throws IOException { + if (kuduReader == null) { + kuduReader = + new KuduReader<>( + tableInfo, + readerConfig, + rowResultConverter, + tableFilters, + tableProjections); + } + } + + private void closeKuduReader() throws IOException { + if (kuduReader != null) { + kuduReader.close(); + kuduReader = null; + } + } + + @Override + public void close() throws IOException { + if (resultIterator != null) { + try { + resultIterator.close(); + } catch (KuduException e) { + log.error("Error while closing reader iterator.", e); + } + } + closeKuduReader(); + } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { + return cachedStatistics; + } + + @Override + public InputSplitAssigner getInputSplitAssigner(KuduInputSplit[] inputSplits) { + return new LocatableInputSplitAssigner(inputSplits); + } + + @Override + public KuduInputSplit[] createInputSplits(int minNumSplits) throws IOException { + try { + startKuduReader(); + return kuduReader.createInputSplits(minNumSplits); + } finally { + closeKuduReader(); + } + } + + @Override + public boolean reachedEnd() { + return endReached; + } + + @Override + public T nextRecord(T reuse) throws IOException { + // check that current iterator has next rows + if (this.resultIterator.hasNext()) { + return resultIterator.next(); + } else { + endReached = true; + return null; + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduOutputFormat.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduOutputFormat.java new file mode 100644 index 0000000..824be30 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduOutputFormat.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.format; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.failure.DefaultKuduFailureHandler; +import org.apache.flink.connector.kudu.connector.failure.KuduFailureHandler; +import org.apache.flink.connector.kudu.connector.writer.KuduOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.KuduWriter; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Output format for writing data into a Kudu table (defined by the provided {@link KuduTableInfo}) + * in both batch and stream programs. + */ +@PublicEvolving +public class KuduOutputFormat extends RichOutputFormat implements CheckpointedFunction { + + private final Logger log = LoggerFactory.getLogger(getClass()); + + private final KuduTableInfo tableInfo; + private final KuduWriterConfig writerConfig; + private final KuduFailureHandler failureHandler; + private final KuduOperationMapper opsMapper; + + private transient KuduWriter kuduWriter; + + public KuduOutputFormat( + KuduWriterConfig writerConfig, + KuduTableInfo tableInfo, + KuduOperationMapper opsMapper) { + this(writerConfig, tableInfo, opsMapper, new DefaultKuduFailureHandler()); + } + + public KuduOutputFormat( + KuduWriterConfig writerConfig, + KuduTableInfo tableInfo, + KuduOperationMapper opsMapper, + KuduFailureHandler failureHandler) { + this.tableInfo = checkNotNull(tableInfo, "tableInfo could not be null"); + this.writerConfig = checkNotNull(writerConfig, "config could not be null"); + this.opsMapper = checkNotNull(opsMapper, "opsMapper could not be null"); + this.failureHandler = checkNotNull(failureHandler, "failureHandler could not be null"); + } + + @Override + public void configure(Configuration parameters) {} + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + kuduWriter = new KuduWriter(tableInfo, writerConfig, opsMapper, failureHandler); + } + + @Override + public void writeRecord(IN row) throws IOException { + kuduWriter.write(row); + } + + @Override + public void close() throws IOException { + if (kuduWriter != null) { + kuduWriter.close(); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception { + kuduWriter.flushAndCheckErrors(); + } + + @Override + public void initializeState(FunctionInitializationContext functionInitializationContext) + throws Exception {} +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduRowDataInputFormat.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduRowDataInputFormat.java new file mode 100644 index 0000000..6a100c5 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduRowDataInputFormat.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.format; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.kudu.connector.KuduFilterInfo; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.converter.RowResultConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.table.data.RowData; + +import java.util.List; + +/** InputFormat based on the rowData object type. */ +@PublicEvolving +public class KuduRowDataInputFormat extends AbstractKuduInputFormat { + + public KuduRowDataInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo) { + super(readerConfig, rowResultConverter, tableInfo); + } + + public KuduRowDataInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo, + List tableProjections) { + super(readerConfig, rowResultConverter, tableInfo, tableProjections); + } + + public KuduRowDataInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo, + List tableFilters, + List tableProjections) { + super(readerConfig, rowResultConverter, tableInfo, tableFilters, tableProjections); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(RowData.class); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduRowInputFormat.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduRowInputFormat.java new file mode 100644 index 0000000..5f0dc12 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/format/KuduRowInputFormat.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.format; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.kudu.connector.KuduFilterInfo; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.converter.RowResultConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.types.Row; + +import java.util.List; + +/** InputFormat based on the row object type. */ +@PublicEvolving +public class KuduRowInputFormat extends AbstractKuduInputFormat { + + public KuduRowInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo) { + super(readerConfig, rowResultConverter, tableInfo); + } + + public KuduRowInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo, + List tableProjections) { + super(readerConfig, rowResultConverter, tableInfo, tableProjections); + } + + public KuduRowInputFormat( + KuduReaderConfig readerConfig, + RowResultConverter rowResultConverter, + KuduTableInfo tableInfo, + List tableFilters, + List tableProjections) { + super(readerConfig, rowResultConverter, tableInfo, tableFilters, tableProjections); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(Row.class); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/streaming/KuduSink.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/streaming/KuduSink.java new file mode 100644 index 0000000..2943a19 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/streaming/KuduSink.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.streaming; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.failure.DefaultKuduFailureHandler; +import org.apache.flink.connector.kudu.connector.failure.KuduFailureHandler; +import org.apache.flink.connector.kudu.connector.writer.KuduOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.KuduWriter; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Streaming Sink that executes Kudu operations based on the incoming elements. The target Kudu + * table is defined in the {@link KuduTableInfo} object together with parameters for table creation + * in case the table does not exist. + * + *

Incoming records are mapped to Kudu table operations using the provided {@link + * KuduOperationMapper} logic. While failures resulting from the operations are handled by the + * {@link KuduFailureHandler} instance. + * + * @param Type of the input records + */ +@PublicEvolving +public class KuduSink extends RichSinkFunction implements CheckpointedFunction { + + private final Logger log = LoggerFactory.getLogger(getClass()); + + private final KuduTableInfo tableInfo; + private final KuduWriterConfig writerConfig; + private final KuduFailureHandler failureHandler; + private final KuduOperationMapper opsMapper; + private transient KuduWriter kuduWriter; + + /** + * Creates a new {@link KuduSink} that will execute operations against the specified Kudu table + * (defined in {@link KuduTableInfo}) for the incoming stream elements. + * + * @param writerConfig Writer configuration + * @param tableInfo Table information for the target table + * @param opsMapper Mapping logic from inputs to Kudu operations + */ + public KuduSink( + KuduWriterConfig writerConfig, + KuduTableInfo tableInfo, + KuduOperationMapper opsMapper) { + this(writerConfig, tableInfo, opsMapper, new DefaultKuduFailureHandler()); + } + + /** + * Creates a new {@link KuduSink} that will execute operations against the specified Kudu table + * (defined in {@link KuduTableInfo}) for the incoming stream elements. + * + * @param writerConfig Writer configuration + * @param tableInfo Table information for the target table + * @param opsMapper Mapping logic from inputs to Kudu operations + * @param failureHandler Custom failure handler instance + */ + public KuduSink( + KuduWriterConfig writerConfig, + KuduTableInfo tableInfo, + KuduOperationMapper opsMapper, + KuduFailureHandler failureHandler) { + this.tableInfo = checkNotNull(tableInfo, "tableInfo could not be null"); + this.writerConfig = checkNotNull(writerConfig, "config could not be null"); + this.opsMapper = checkNotNull(opsMapper, "opsMapper could not be null"); + this.failureHandler = checkNotNull(failureHandler, "failureHandler could not be null"); + } + + @Override + public void open(Configuration parameters) throws Exception { + kuduWriter = new KuduWriter(tableInfo, writerConfig, opsMapper, failureHandler); + } + + @Override + public void invoke(IN value) throws Exception { + try { + kuduWriter.write(value); + } catch (ClassCastException e) { + failureHandler.onTypeMismatch(e); + } + } + + @Override + public void close() throws Exception { + if (kuduWriter != null) { + kuduWriter.close(); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception { + kuduWriter.flushAndCheckErrors(); + } + + @Override + public void initializeState(FunctionInitializationContext functionInitializationContext) + throws Exception {} +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/AbstractReadOnlyCatalog.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/AbstractReadOnlyCatalog.java new file mode 100644 index 0000000..f3d01cf --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/AbstractReadOnlyCatalog.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; + +/** Read-only catalog. */ +@Internal +public abstract class AbstractReadOnlyCatalog extends AbstractCatalog { + + private static final CatalogException UNSUPPORTED_ERR = + new CatalogException("This action is not supported for read-only catalogs"); + + public AbstractReadOnlyCatalog(String name, String defaultDatabase) { + super(name, defaultDatabase); + } + + @Override + public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException { + throw UNSUPPORTED_ERR; + } + + @Override + public void alterTable( + ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) + throws TableNotExistException { + throw UNSUPPORTED_ERR; + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, PartitionAlreadyExistsException, + CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void createFunction( + ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) + throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void alterFunction( + ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException, TablePartitionedException { + throw UNSUPPORTED_ERR; + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw UNSUPPORTED_ERR; + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException { + throw UNSUPPORTED_ERR; + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException { + throw UNSUPPORTED_ERR; + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduCatalog.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduCatalog.java new file mode 100644 index 0000000..37e554d --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduCatalog.java @@ -0,0 +1,373 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory; +import org.apache.flink.connector.kudu.table.dynamic.catalog.KuduDynamicCatalog; +import org.apache.flink.connector.kudu.table.utils.KuduTableUtils; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.TableFactory; +import org.apache.flink.util.StringUtils; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.client.AlterTableOptions; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.shaded.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Catalog for reading and creating Kudu tables. + * + * @deprecated After this class based on {@link KuduTableFactory}, but flink upgrade {@link + * KuduDynamicTableSourceSinkFactory} {@link KuduCatalog} underlying the use of TableFactory + * also needs to update,So this class is replaced by the {@link KuduDynamicCatalog} class + */ +@PublicEvolving +@Deprecated +public class KuduCatalog extends AbstractReadOnlyCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(KuduCatalog.class); + private final KuduTableFactory tableFactory = new KuduTableFactory(); + private final String kuduMasters; + private KuduClient kuduClient; + + /** + * Create a new {@link KuduCatalog} with the specified catalog name and kudu master addresses. + * + * @param catalogName Name of the catalog (used by the table environment) + * @param kuduMasters Connection address to Kudu + */ + public KuduCatalog(String catalogName, String kuduMasters) { + super(catalogName, "default_database"); + this.kuduMasters = kuduMasters; + this.kuduClient = createClient(); + } + + /** + * Create a new {@link KuduCatalog} with the specified kudu master addresses. + * + * @param kuduMasters Connection address to Kudu + */ + public KuduCatalog(String kuduMasters) { + this("kudu", kuduMasters); + } + + public Optional getTableFactory() { + return Optional.of(getKuduTableFactory()); + } + + public KuduTableFactory getKuduTableFactory() { + return tableFactory; + } + + private KuduClient createClient() { + return new KuduClient.KuduClientBuilder(kuduMasters).build(); + } + + @Override + public void open() {} + + @Override + public void close() { + try { + if (kuduClient != null) { + kuduClient.close(); + } + } catch (KuduException e) { + LOG.error("Error while closing kudu client", e); + } + } + + public ObjectPath getObjectPath(String tableName) { + return new ObjectPath(getDefaultDatabase(), tableName); + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "databaseName cannot be null or empty"); + + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } + + try { + return kuduClient.getTablesList().getTablesList(); + } catch (Throwable t) { + throw new CatalogException("Could not list tables", t); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) { + checkNotNull(tablePath); + try { + return kuduClient.tableExists(tablePath.getObjectName()); + } catch (KuduException e) { + throw new CatalogException(e); + } + } + + @Override + public CatalogTable getTable(ObjectPath tablePath) throws TableNotExistException { + checkNotNull(tablePath); + + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + + String tableName = tablePath.getObjectName(); + + try { + KuduTable kuduTable = kuduClient.openTable(tableName); + + CatalogTableImpl table = + new CatalogTableImpl( + KuduTableUtils.kuduToFlinkSchema(kuduTable.getSchema()), + createTableProperties( + tableName, kuduTable.getSchema().getPrimaryKeyColumns()), + tableName); + + return table; + } catch (KuduException e) { + throw new CatalogException(e); + } + } + + protected Map createTableProperties( + String tableName, List primaryKeyColumns) { + Map props = new HashMap<>(); + props.put(KuduTableFactory.KUDU_MASTERS, kuduMasters); + String primaryKeyNames = + primaryKeyColumns.stream() + .map(ColumnSchema::getName) + .collect(Collectors.joining(",")); + props.put(KuduTableFactory.KUDU_PRIMARY_KEY_COLS, primaryKeyNames); + props.put(KuduTableFactory.KUDU_TABLE, tableName); + return props; + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException { + String tableName = tablePath.getObjectName(); + try { + if (tableExists(tablePath)) { + kuduClient.deleteTable(tableName); + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } catch (KuduException e) { + throw new CatalogException("Could not delete table " + tableName, e); + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException { + String tableName = tablePath.getObjectName(); + try { + if (tableExists(tablePath)) { + kuduClient.alterTable(tableName, new AlterTableOptions().renameTable(newTableName)); + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } catch (KuduException e) { + throw new CatalogException("Could not rename table " + tableName, e); + } + } + + public void createTable(KuduTableInfo tableInfo, boolean ignoreIfExists) + throws CatalogException, TableAlreadyExistException { + ObjectPath path = getObjectPath(tableInfo.getName()); + if (tableExists(path)) { + if (ignoreIfExists) { + return; + } else { + throw new TableAlreadyExistException(getName(), path); + } + } + + try { + kuduClient.createTable( + tableInfo.getName(), tableInfo.getSchema(), tableInfo.getCreateTableOptions()); + } catch (KuduException e) { + throw new CatalogException("Could not create table " + tableInfo.getName(), e); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException { + Map tableProperties = table.getOptions(); + TableSchema tableSchema = table.getSchema(); + + Set optionalProperties = + new HashSet<>(Arrays.asList(KuduTableFactory.KUDU_REPLICAS)); + Set requiredProperties = + new HashSet<>(Arrays.asList(KuduTableFactory.KUDU_HASH_COLS)); + + if (!tableSchema.getPrimaryKey().isPresent()) { + requiredProperties.add(KuduTableFactory.KUDU_PRIMARY_KEY_COLS); + } + + if (!tableProperties.keySet().containsAll(requiredProperties)) { + throw new CatalogException( + "Missing required property. The following properties must be provided: " + + requiredProperties.toString()); + } + + Set permittedProperties = Sets.union(requiredProperties, optionalProperties); + if (!permittedProperties.containsAll(tableProperties.keySet())) { + throw new CatalogException( + "Unpermitted properties were given. The following properties are allowed:" + + permittedProperties.toString()); + } + + String tableName = tablePath.getObjectName(); + + KuduTableInfo tableInfo = + KuduTableUtils.createTableInfo(tableName, tableSchema, tableProperties); + + createTable(tableInfo, ignoreIfExists); + } + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public List listDatabases() throws CatalogException { + return Collections.singletonList(getDefaultDatabase()); + } + + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + if (databaseName.equals(getDefaultDatabase())) { + return new CatalogDatabaseImpl(new HashMap<>(), null); + } else { + throw new DatabaseNotExistException(getName(), databaseName); + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + return listDatabases().contains(databaseName); + } + + @Override + public List listViews(String databaseName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitions(ObjectPath tablePath) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List filters) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + return false; + } + + @Override + public List listFunctions(String dbName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) + throws FunctionNotExistException, CatalogException { + throw new FunctionNotExistException(getName(), functionPath); + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + return false; + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableFactory.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableFactory.java new file mode 100644 index 0000000..943030f --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableFactory.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.connector.kudu.table.utils.KuduTableUtils; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.SchemaValidator; +import org.apache.flink.table.factories.TableSinkFactory; +import org.apache.flink.table.factories.TableSourceFactory; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_CLASS; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_FROM; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_SERIALIZED; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_TYPE; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_CLASS; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_DELAY; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_SERIALIZED; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_TYPE; +import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_DATA_TYPE; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_FROM; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_PROCTIME; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Factory for Kudu table source/sink impls. */ +public class KuduTableFactory + implements TableSourceFactory, TableSinkFactory> { + + public static final String KUDU_TABLE = "kudu.table"; + public static final String KUDU_MASTERS = "kudu.masters"; + public static final String KUDU_HASH_COLS = "kudu.hash-columns"; + public static final String KUDU_PRIMARY_KEY_COLS = "kudu.primary-key-columns"; + public static final String KUDU_REPLICAS = "kudu.replicas"; + public static final String KUDU_MAX_BUFFER_SIZE = "kudu.max-buffer-size"; + public static final String KUDU_FLUSH_INTERVAL = "kudu.flush-interval"; + public static final String KUDU_OPERATION_TIMEOUT = "kudu.operation-timeout"; + public static final String KUDU_IGNORE_NOT_FOUND = "kudu.ignore-not-found"; + public static final String KUDU_IGNORE_DUPLICATE = "kudu.ignore-duplicate"; + public static final String KUDU = "kudu"; + + @Override + public Map requiredContext() { + Map context = new HashMap<>(); + context.put(CONNECTOR_TYPE, KUDU); + return context; + } + + @Override + public List supportedProperties() { + List properties = new ArrayList<>(); + properties.add(KUDU_TABLE); + properties.add(KUDU_MASTERS); + properties.add(KUDU_HASH_COLS); + properties.add(KUDU_PRIMARY_KEY_COLS); + properties.add(KUDU_MAX_BUFFER_SIZE); + properties.add(KUDU_FLUSH_INTERVAL); + properties.add(KUDU_OPERATION_TIMEOUT); + properties.add(KUDU_IGNORE_NOT_FOUND); + properties.add(KUDU_IGNORE_DUPLICATE); + // schema + properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); + properties.add(SCHEMA + ".#." + SCHEMA_TYPE); + properties.add(SCHEMA + ".#." + SCHEMA_NAME); + properties.add(SCHEMA + ".#." + SCHEMA_FROM); + // computed column + properties.add(SCHEMA + ".#." + EXPR); + + // time attributes + properties.add(SCHEMA + ".#." + SCHEMA_PROCTIME); + properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_TYPE); + properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_FROM); + properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_CLASS); + properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_SERIALIZED); + properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_TYPE); + properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_CLASS); + properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_SERIALIZED); + properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_DELAY); + + // watermark + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_ROWTIME); + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_EXPR); + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_DATA_TYPE); + return properties; + } + + private DescriptorProperties validateTable(CatalogTable table) { + Map properties = table.toProperties(); + checkNotNull(properties.get(KUDU_MASTERS), "Missing required property " + KUDU_MASTERS); + + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + new SchemaValidator(true, false, false).validate(descriptorProperties); + return descriptorProperties; + } + + @Override + public KuduTableSource createTableSource(ObjectPath tablePath, CatalogTable table) { + validateTable(table); + String tableName = table.toProperties().getOrDefault(KUDU_TABLE, tablePath.getObjectName()); + return createTableSource(tableName, table.getSchema(), table.getOptions()); + } + + private KuduTableSource createTableSource( + String tableName, TableSchema schema, Map props) { + String masterAddresses = props.get(KUDU_MASTERS); + TableSchema physicalSchema = KuduTableUtils.getSchemaWithSqlTimestamp(schema); + KuduTableInfo tableInfo = KuduTableUtils.createTableInfo(tableName, schema, props); + + KuduReaderConfig.Builder configBuilder = + KuduReaderConfig.Builder.setMasters(masterAddresses); + + return new KuduTableSource(configBuilder, tableInfo, physicalSchema, null, null); + } + + @Override + public KuduTableSink createTableSink(ObjectPath tablePath, CatalogTable table) { + validateTable(table); + String tableName = table.toProperties().getOrDefault(KUDU_TABLE, tablePath.getObjectName()); + return createTableSink(tableName, table.getSchema(), table.toProperties()); + } + + private KuduTableSink createTableSink( + String tableName, TableSchema schema, Map props) { + DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(props); + String masterAddresses = props.get(KUDU_MASTERS); + TableSchema physicalSchema = KuduTableUtils.getSchemaWithSqlTimestamp(schema); + KuduTableInfo tableInfo = KuduTableUtils.createTableInfo(tableName, schema, props); + + KuduWriterConfig.Builder configBuilder = + KuduWriterConfig.Builder.setMasters(masterAddresses); + + Optional operationTimeout = properties.getOptionalLong(KUDU_OPERATION_TIMEOUT); + Optional flushInterval = properties.getOptionalInt(KUDU_FLUSH_INTERVAL); + Optional bufferSize = properties.getOptionalInt(KUDU_MAX_BUFFER_SIZE); + Optional ignoreNotFound = properties.getOptionalBoolean(KUDU_IGNORE_NOT_FOUND); + Optional ignoreDuplicate = properties.getOptionalBoolean(KUDU_IGNORE_DUPLICATE); + + operationTimeout.ifPresent(time -> configBuilder.setOperationTimeout(time)); + flushInterval.ifPresent(interval -> configBuilder.setFlushInterval(interval)); + bufferSize.ifPresent(size -> configBuilder.setMaxBufferSize(size)); + ignoreNotFound.ifPresent(i -> configBuilder.setIgnoreNotFound(i)); + ignoreDuplicate.ifPresent(i -> configBuilder.setIgnoreDuplicate(i)); + + return new KuduTableSink(configBuilder, tableInfo, physicalSchema); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableSink.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableSink.java new file mode 100644 index 0000000..4fc0984 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableSink.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.connector.kudu.streaming.KuduSink; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sinks.UpsertStreamTableSink; +import org.apache.flink.table.utils.TableConnectorUtils; +import org.apache.flink.types.Row; + +import java.util.Objects; + +/** Table API Kudu sink implementation. */ +public class KuduTableSink implements UpsertStreamTableSink { + + private final KuduWriterConfig.Builder writerConfigBuilder; + private final TableSchema flinkSchema; + private final KuduTableInfo tableInfo; + + public KuduTableSink( + KuduWriterConfig.Builder configBuilder, + KuduTableInfo tableInfo, + TableSchema flinkSchema) { + this.writerConfigBuilder = configBuilder; + this.tableInfo = tableInfo; + this.flinkSchema = flinkSchema; + } + + @Override + public void setKeyFields(String[] keyFields) { + /* this has no effect */ + } + + @Override + public void setIsAppendOnly(Boolean isAppendOnly) { + /* this has no effect */ + } + + @Override + public TypeInformation getRecordType() { + return flinkSchema.toRowType(); + } + + @Override + public DataStreamSink consumeDataStream(DataStream> dataStreamTuple) { + KuduSink upsertKuduSink = + new KuduSink( + writerConfigBuilder.build(), + tableInfo, + new UpsertOperationMapper(getTableSchema().getFieldNames())); + + return dataStreamTuple + .addSink(upsertKuduSink) + .setParallelism(dataStreamTuple.getParallelism()) + .name( + TableConnectorUtils.generateRuntimeName( + this.getClass(), getTableSchema().getFieldNames())); + } + + @Override + public TableSink> configure( + String[] fieldNames, TypeInformation[] fieldTypes) { + return new KuduTableSink(writerConfigBuilder, tableInfo, flinkSchema); + } + + @Override + public TableSchema getTableSchema() { + return flinkSchema; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || o.getClass() != this.getClass()) { + return false; + } + KuduTableSink that = (KuduTableSink) o; + return this.writerConfigBuilder.equals(that.writerConfigBuilder) + && this.flinkSchema.equals(that.flinkSchema) + && this.tableInfo.equals(that.tableInfo); + } + + @Override + public int hashCode() { + return Objects.hash(writerConfigBuilder, flinkSchema, tableInfo); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableSource.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableSource.java new file mode 100644 index 0000000..82ed1e9 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/KuduTableSource.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.kudu.connector.KuduFilterInfo; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.converter.RowResultRowConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.format.KuduRowInputFormat; +import org.apache.flink.connector.kudu.table.utils.KuduTableUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.sources.FilterableTableSource; +import org.apache.flink.table.sources.LimitableTableSource; +import org.apache.flink.table.sources.ProjectableTableSource; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.table.sources.TableSource; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.ListIterator; +import java.util.Optional; + +/** DataStream source implementation for Kudu. */ +public class KuduTableSource + implements StreamTableSource, + LimitableTableSource, + ProjectableTableSource, + FilterableTableSource { + + private static final Logger LOG = LoggerFactory.getLogger(KuduTableSource.class); + + private final KuduReaderConfig.Builder configBuilder; + private final KuduTableInfo tableInfo; + private final TableSchema flinkSchema; + private final String[] projectedFields; + // predicate expression to apply + @Nullable private final List predicates; + private boolean isFilterPushedDown; + + private KuduRowInputFormat kuduRowInputFormat; + + public KuduTableSource( + KuduReaderConfig.Builder configBuilder, + KuduTableInfo tableInfo, + TableSchema flinkSchema, + List predicates, + String[] projectedFields) { + this.configBuilder = configBuilder; + this.tableInfo = tableInfo; + this.flinkSchema = flinkSchema; + this.predicates = predicates; + this.projectedFields = projectedFields; + if (predicates != null && predicates.size() != 0) { + this.isFilterPushedDown = true; + } + this.kuduRowInputFormat = + new KuduRowInputFormat( + configBuilder.build(), + new RowResultRowConverter(), + tableInfo, + predicates == null ? Collections.emptyList() : predicates, + projectedFields == null ? null : Arrays.asList(projectedFields)); + } + + @Override + public boolean isBounded() { + return true; + } + + @Override + public DataStream getDataStream(StreamExecutionEnvironment env) { + KuduRowInputFormat inputFormat = + new KuduRowInputFormat( + configBuilder.build(), + new RowResultRowConverter(), + tableInfo, + predicates == null ? Collections.emptyList() : predicates, + projectedFields == null ? null : Arrays.asList(projectedFields)); + return env.createInput( + inputFormat, + (TypeInformation) + TypeConversions.fromDataTypeToLegacyInfo(getProducedDataType())) + .name(explainSource()); + } + + @Override + public TableSchema getTableSchema() { + return flinkSchema; + } + + @Override + public boolean isFilterPushedDown() { + return this.isFilterPushedDown; + } + + @Override + public DataType getProducedDataType() { + if (projectedFields == null) { + return flinkSchema.toRowDataType(); + } else { + DataTypes.Field[] fields = new DataTypes.Field[projectedFields.length]; + for (int i = 0; i < fields.length; i++) { + String fieldName = projectedFields[i]; + fields[i] = + DataTypes.FIELD( + fieldName, flinkSchema.getTableColumn(fieldName).get().getType()); + } + return DataTypes.ROW(fields); + } + } + + @Override + public boolean isLimitPushedDown() { + return true; + } + + @Override + public TableSource applyLimit(long l) { + return new KuduTableSource( + configBuilder.setRowLimit((int) l), + tableInfo, + flinkSchema, + predicates, + projectedFields); + } + + @Override + public TableSource projectFields(int[] ints) { + String[] fieldNames = new String[ints.length]; + RowType producedDataType = (RowType) getProducedDataType().getLogicalType(); + List prevFieldNames = producedDataType.getFieldNames(); + for (int i = 0; i < ints.length; i++) { + fieldNames[i] = prevFieldNames.get(ints[i]); + } + return new KuduTableSource(configBuilder, tableInfo, flinkSchema, predicates, fieldNames); + } + + @Override + public TableSource applyPredicate(List predicates) { + List kuduPredicates = new ArrayList<>(); + ListIterator predicatesIter = predicates.listIterator(); + while (predicatesIter.hasNext()) { + Expression predicate = predicatesIter.next(); + Optional kuduPred = KuduTableUtils.toKuduFilterInfo(predicate); + if (kuduPred != null && kuduPred.isPresent()) { + LOG.debug( + "Predicate [{}] converted into KuduFilterInfo and pushed into " + + "KuduTable [{}].", + predicate, + tableInfo.getName()); + kuduPredicates.add(kuduPred.get()); + predicatesIter.remove(); + } else { + LOG.debug( + "Predicate [{}] could not be pushed into KuduFilterInfo for KuduTable [{}].", + predicate, + tableInfo.getName()); + } + } + return new KuduTableSource( + configBuilder, tableInfo, flinkSchema, kuduPredicates, projectedFields); + } + + @Override + public String explainSource() { + return "KuduTableSource[schema=" + + Arrays.toString(getTableSchema().getFieldNames()) + + ", filter=" + + predicateString() + + (projectedFields != null + ? ", projectFields=" + Arrays.toString(projectedFields) + "]" + : "]"); + } + + private String predicateString() { + if (predicates == null || predicates.size() == 0) { + return "No predicates push down"; + } else { + return "AND(" + predicates + ")"; + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/UpsertOperationMapper.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/UpsertOperationMapper.java new file mode 100644 index 0000000..9fe5597 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/UpsertOperationMapper.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kudu.connector.writer.AbstractSingleOperationMapper; +import org.apache.flink.types.Row; + +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.Operation; + +import java.util.Optional; + +/** Logic to map a Flink DataStream upsert record to a Kudu-compatible format. */ +@Internal +public class UpsertOperationMapper extends AbstractSingleOperationMapper> { + + public UpsertOperationMapper(String[] columnNames) { + super(columnNames); + } + + @Override + public Object getField(Tuple2 input, int i) { + return input.f1.getField(i); + } + + @Override + public Optional createBaseOperation(Tuple2 input, KuduTable table) { + return Optional.of(input.f0 ? table.newUpsert() : table.newDelete()); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSink.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSink.java new file mode 100644 index 0000000..999f90f --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSink.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.dynamic; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.connector.kudu.connector.writer.RowDataUpsertOperationMapper; +import org.apache.flink.connector.kudu.streaming.KuduSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Preconditions; + +import java.util.Objects; + +/** A {@link KuduDynamicTableSink} for Kudu. */ +public class KuduDynamicTableSink implements DynamicTableSink { + private final KuduWriterConfig.Builder writerConfigBuilder; + private final TableSchema flinkSchema; + private final KuduTableInfo tableInfo; + + public KuduDynamicTableSink( + KuduWriterConfig.Builder writerConfigBuilder, + TableSchema flinkSchema, + KuduTableInfo tableInfo) { + this.writerConfigBuilder = writerConfigBuilder; + this.flinkSchema = flinkSchema; + this.tableInfo = tableInfo; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + this.validatePrimaryKey(requestedMode); + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.DELETE) + .addContainedKind(RowKind.UPDATE_AFTER) + .build(); + } + + private void validatePrimaryKey(ChangelogMode requestedMode) { + Preconditions.checkState( + ChangelogMode.insertOnly().equals(requestedMode) + || this.tableInfo.getSchema().getPrimaryKeyColumnCount() != 0, + "please declare primary key for sink table when query contains update/delete record."); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + KuduSink upsertKuduSink = + new KuduSink<>( + writerConfigBuilder.build(), + tableInfo, + new RowDataUpsertOperationMapper(flinkSchema)); + return SinkFunctionProvider.of(upsertKuduSink); + } + + @Override + public DynamicTableSink copy() { + return new KuduDynamicTableSink(this.writerConfigBuilder, this.flinkSchema, this.tableInfo); + } + + @Override + public String asSummaryString() { + return "kudu"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KuduDynamicTableSink that = (KuduDynamicTableSink) o; + return Objects.equals(writerConfigBuilder, that.writerConfigBuilder) + && Objects.equals(flinkSchema, that.flinkSchema) + && Objects.equals(tableInfo, that.tableInfo); + } + + @Override + public int hashCode() { + return Objects.hash(writerConfigBuilder, flinkSchema, tableInfo); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSource.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSource.java new file mode 100644 index 0000000..bc19696 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSource.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.dynamic; + +import org.apache.flink.connector.kudu.connector.KuduFilterInfo; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.converter.RowResultRowDataConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.format.KuduRowDataInputFormat; +import org.apache.flink.connector.kudu.table.function.lookup.KuduLookupOptions; +import org.apache.flink.connector.kudu.table.function.lookup.KuduRowDataLookupFunction; +import org.apache.flink.connector.kudu.table.utils.KuduTableUtils; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.InputFormatProvider; +import org.apache.flink.table.connector.source.LookupTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.TableFunctionProvider; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.FieldsDataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.kudu.shaded.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.flink.table.utils.TableSchemaUtils.containsPhysicalColumnsOnly; + +/** A {@link DynamicTableSource} for Kudu. */ +public class KuduDynamicTableSource + implements ScanTableSource, + SupportsProjectionPushDown, + SupportsLimitPushDown, + LookupTableSource, + SupportsFilterPushDown { + + private static final Logger LOG = LoggerFactory.getLogger(KuduDynamicTableSource.class); + private final KuduTableInfo tableInfo; + private final KuduLookupOptions kuduLookupOptions; + private final KuduRowDataInputFormat kuduRowDataInputFormat; + private final transient List predicates = Lists.newArrayList(); + private KuduReaderConfig.Builder configBuilder; + private TableSchema physicalSchema; + private String[] projectedFields; + private transient List filters; + + public KuduDynamicTableSource( + KuduReaderConfig.Builder configBuilder, + KuduTableInfo tableInfo, + TableSchema physicalSchema, + String[] projectedFields, + KuduLookupOptions kuduLookupOptions) { + this.configBuilder = configBuilder; + this.tableInfo = tableInfo; + this.physicalSchema = physicalSchema; + this.projectedFields = projectedFields; + this.kuduRowDataInputFormat = + new KuduRowDataInputFormat( + configBuilder.build(), + new RowResultRowDataConverter(), + tableInfo, + predicates, + projectedFields == null ? null : Lists.newArrayList(projectedFields)); + this.kuduLookupOptions = kuduLookupOptions; + } + + @Override + public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { + int keysLen = context.getKeys().length; + String[] keyNames = new String[keysLen]; + for (int i = 0; i < keyNames.length; ++i) { + int[] innerKeyArr = context.getKeys()[i]; + Preconditions.checkArgument( + innerKeyArr.length == 1, "Kudu only support non-nested look up keys"); + keyNames[i] = this.physicalSchema.getFieldNames()[innerKeyArr[0]]; + } + KuduRowDataLookupFunction rowDataLookupFunction = + KuduRowDataLookupFunction.Builder.options() + .keyNames(keyNames) + .kuduReaderConfig(configBuilder.build()) + .projectedFields(projectedFields) + .tableInfo(tableInfo) + .kuduLookupOptions(kuduLookupOptions) + .build(); + return TableFunctionProvider.of(rowDataLookupFunction); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + if (CollectionUtils.isNotEmpty(this.filters)) { + for (ResolvedExpression filter : this.filters) { + Optional kuduFilterInfo = KuduTableUtils.toKuduFilterInfo(filter); + if (kuduFilterInfo != null && kuduFilterInfo.isPresent()) { + this.predicates.add(kuduFilterInfo.get()); + } + } + } + KuduRowDataInputFormat inputFormat = + new KuduRowDataInputFormat( + configBuilder.build(), + new RowResultRowDataConverter(), + tableInfo, + this.predicates, + projectedFields == null ? null : Lists.newArrayList(projectedFields)); + return InputFormatProvider.of(inputFormat); + } + + @Override + public DynamicTableSource copy() { + return new KuduDynamicTableSource( + this.configBuilder, + this.tableInfo, + this.physicalSchema, + this.projectedFields, + this.kuduLookupOptions); + } + + @Override + public String asSummaryString() { + return "kudu"; + } + + @Override + public boolean supportsNestedProjection() { + // planner doesn't support nested projection push down yet. + return false; + } + + @Override + public void applyProjection(int[][] projectedFields, DataType producedDataType) { + // parser projectFields + this.physicalSchema = projectSchema(this.physicalSchema, projectedFields); + this.projectedFields = physicalSchema.getFieldNames(); + } + + private TableSchema projectSchema(TableSchema tableSchema, int[][] projectedFields) { + Preconditions.checkArgument( + containsPhysicalColumnsOnly(tableSchema), + "Projection is only supported for physical columns."); + TableSchema.Builder builder = TableSchema.builder(); + + FieldsDataType fields = + (FieldsDataType) + DataTypeUtils.projectRow(tableSchema.toRowDataType(), projectedFields); + RowType topFields = (RowType) fields.getLogicalType(); + for (int i = 0; i < topFields.getFieldCount(); i++) { + builder.field(topFields.getFieldNames().get(i), fields.getChildren().get(i)); + } + return builder.build(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KuduDynamicTableSource that = (KuduDynamicTableSource) o; + return Objects.equals(configBuilder, that.configBuilder) + && Objects.equals(tableInfo, that.tableInfo) + && Objects.equals(physicalSchema, that.physicalSchema) + && Arrays.equals(projectedFields, that.projectedFields) + && Objects.equals(kuduLookupOptions, that.kuduLookupOptions) + && Objects.equals(kuduRowDataInputFormat, that.kuduRowDataInputFormat) + && Objects.equals(filters, that.filters) + && Objects.equals(predicates, that.predicates); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + configBuilder, + tableInfo, + physicalSchema, + kuduLookupOptions, + kuduRowDataInputFormat, + filters, + predicates); + result = 31 * result + Arrays.hashCode(projectedFields); + return result; + } + + @Override + public void applyLimit(long limit) { + this.configBuilder = this.configBuilder.setRowLimit((int) limit); + } + + @Override + public Result applyFilters(List filters) { + this.filters = filters; + return Result.of(Collections.emptyList(), filters); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSourceSinkFactory.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSourceSinkFactory.java new file mode 100644 index 0000000..9bf3816 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicTableSourceSinkFactory.java @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.dynamic; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.connector.kudu.table.function.lookup.KuduLookupOptions; +import org.apache.flink.connector.kudu.table.utils.KuduTableUtils; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import org.apache.kudu.shaded.com.google.common.collect.Sets; + +import java.util.Optional; +import java.util.Set; + +/** + * Factory for creating configured instances of {@link KuduDynamicTableSource}/{@link + * KuduDynamicTableSink} in a stream environment. + */ +public class KuduDynamicTableSourceSinkFactory + implements DynamicTableSourceFactory, DynamicTableSinkFactory { + public static final String IDENTIFIER = "kudu"; + public static final ConfigOption KUDU_TABLE = + ConfigOptions.key("kudu.table") + .stringType() + .noDefaultValue() + .withDescription("kudu's table name"); + + public static final ConfigOption KUDU_MASTERS = + ConfigOptions.key("kudu.masters") + .stringType() + .noDefaultValue() + .withDescription("kudu's master server address"); + + public static final ConfigOption KUDU_HASH_COLS = + ConfigOptions.key("kudu.hash-columns") + .stringType() + .noDefaultValue() + .withDescription("kudu's hash columns"); + + public static final ConfigOption KUDU_REPLICAS = + ConfigOptions.key("kudu.replicas") + .intType() + .defaultValue(3) + .withDescription("kudu's replica nums"); + + public static final ConfigOption KUDU_MAX_BUFFER_SIZE = + ConfigOptions.key("kudu.max-buffer-size") + .intType() + .noDefaultValue() + .withDescription("kudu's max buffer size"); + + public static final ConfigOption KUDU_FLUSH_INTERVAL = + ConfigOptions.key("kudu.flush-interval") + .intType() + .noDefaultValue() + .withDescription("kudu's data flush interval"); + + public static final ConfigOption KUDU_OPERATION_TIMEOUT = + ConfigOptions.key("kudu.operation-timeout") + .longType() + .noDefaultValue() + .withDescription("kudu's operation timeout"); + + public static final ConfigOption KUDU_IGNORE_NOT_FOUND = + ConfigOptions.key("kudu.ignore-not-found") + .booleanType() + .noDefaultValue() + .withDescription("if true, ignore all not found rows"); + + public static final ConfigOption KUDU_IGNORE_DUPLICATE = + ConfigOptions.key("kudu.ignore-not-found") + .booleanType() + .noDefaultValue() + .withDescription("if true, ignore all dulicate rows"); + + public static final ConfigOption KUDU_HASH_PARTITION_NUMS = + ConfigOptions.key("kudu.hash-partition-nums") + .intType() + .defaultValue(KUDU_REPLICAS.defaultValue() * 2) + .withDescription( + "kudu's hash partition bucket nums, defaultValue is 2 * replica nums"); + + public static final ConfigOption KUDU_PRIMARY_KEY_COLS = + ConfigOptions.key("kudu.primary-key-columns") + .stringType() + .noDefaultValue() + .withDescription("kudu's primary key, primary key must be ordered"); + + public static final ConfigOption KUDU_SCAN_ROW_SIZE = + ConfigOptions.key("kudu.scan.row-size") + .intType() + .defaultValue(0) + .withDescription("kudu's scan row size"); + + public static final ConfigOption KUDU_LOOKUP_CACHE_MAX_ROWS = + ConfigOptions.key("kudu.lookup.cache.max-rows") + .longType() + .defaultValue(-1L) + .withDescription( + "the max number of rows of lookup cache, over this value, the oldest rows will " + + "be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any" + + " of them is " + + "specified. Cache is not enabled as default."); + + public static final ConfigOption KUDU_LOOKUP_CACHE_TTL = + ConfigOptions.key("kudu.lookup.cache.ttl") + .longType() + .defaultValue(-1L) + .withDescription("the cache time to live."); + + public static final ConfigOption KUDU_LOOKUP_MAX_RETRIES = + ConfigOptions.key("kudu.lookup.max-retries") + .intType() + .defaultValue(3) + .withDescription("the max retry times if lookup database failed."); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + ReadableConfig config = getReadableConfig(context); + String masterAddresses = config.get(KUDU_MASTERS); + String tableName = config.get(KUDU_TABLE); + Optional operationTimeout = config.getOptional(KUDU_OPERATION_TIMEOUT); + Optional flushInterval = config.getOptional(KUDU_FLUSH_INTERVAL); + Optional bufferSize = config.getOptional(KUDU_MAX_BUFFER_SIZE); + Optional ignoreNotFound = config.getOptional(KUDU_IGNORE_NOT_FOUND); + Optional ignoreDuplicate = config.getOptional(KUDU_IGNORE_DUPLICATE); + TableSchema schema = context.getCatalogTable().getSchema(); + TableSchema physicalSchema = KuduTableUtils.getSchemaWithSqlTimestamp(schema); + KuduTableInfo tableInfo = + KuduTableUtils.createTableInfo( + tableName, schema, context.getCatalogTable().toProperties()); + + KuduWriterConfig.Builder configBuilder = + KuduWriterConfig.Builder.setMasters(masterAddresses); + operationTimeout.ifPresent(configBuilder::setOperationTimeout); + flushInterval.ifPresent(configBuilder::setFlushInterval); + bufferSize.ifPresent(configBuilder::setMaxBufferSize); + ignoreNotFound.ifPresent(configBuilder::setIgnoreNotFound); + ignoreDuplicate.ifPresent(configBuilder::setIgnoreDuplicate); + return new KuduDynamicTableSink(configBuilder, physicalSchema, tableInfo); + } + + private ReadableConfig getReadableConfig(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + return helper.getOptions(); + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + ReadableConfig config = getReadableConfig(context); + String masterAddresses = config.get(KUDU_MASTERS); + + int scanRowSize = config.get(KUDU_SCAN_ROW_SIZE); + long kuduCacheMaxRows = config.get(KUDU_LOOKUP_CACHE_MAX_ROWS); + long kuduCacheTtl = config.get(KUDU_LOOKUP_CACHE_TTL); + int kuduMaxReties = config.get(KUDU_LOOKUP_MAX_RETRIES); + + // build kudu lookup options + KuduLookupOptions kuduLookupOptions = + KuduLookupOptions.Builder.options() + .withCacheMaxSize(kuduCacheMaxRows) + .withCacheExpireMs(kuduCacheTtl) + .withMaxRetryTimes(kuduMaxReties) + .build(); + + TableSchema schema = context.getCatalogTable().getSchema(); + TableSchema physicalSchema = KuduTableUtils.getSchemaWithSqlTimestamp(schema); + KuduTableInfo tableInfo = + KuduTableUtils.createTableInfo( + config.get(KUDU_TABLE), schema, context.getCatalogTable().toProperties()); + + KuduReaderConfig.Builder configBuilder = + KuduReaderConfig.Builder.setMasters(masterAddresses).setRowLimit(scanRowSize); + return new KuduDynamicTableSource( + configBuilder, + tableInfo, + physicalSchema, + physicalSchema.getFieldNames(), + kuduLookupOptions); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Sets.newHashSet(KUDU_TABLE, KUDU_MASTERS); + } + + @Override + public Set> optionalOptions() { + return Sets.newHashSet( + KUDU_HASH_COLS, + KUDU_HASH_PARTITION_NUMS, + KUDU_PRIMARY_KEY_COLS, + KUDU_SCAN_ROW_SIZE, + KUDU_REPLICAS, + KUDU_MAX_BUFFER_SIZE, + KUDU_MAX_BUFFER_SIZE, + KUDU_OPERATION_TIMEOUT, + KUDU_IGNORE_NOT_FOUND, + KUDU_IGNORE_DUPLICATE, + // lookup + KUDU_LOOKUP_CACHE_MAX_ROWS, + KUDU_LOOKUP_CACHE_TTL, + KUDU_LOOKUP_MAX_RETRIES); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/catalog/KuduCatalogFactory.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/catalog/KuduCatalogFactory.java new file mode 100644 index 0000000..21f5e4c --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/catalog/KuduCatalogFactory.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.dynamic.catalog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.Set; + +import static org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory.IDENTIFIER; +import static org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory.KUDU_MASTERS; +import static org.apache.flink.table.factories.FactoryUtil.PROPERTY_VERSION; + +/** Factory for {@link KuduDynamicCatalog}. */ +@Internal +public class KuduCatalogFactory implements CatalogFactory { + + private static final Logger LOG = LoggerFactory.getLogger(KuduCatalogFactory.class); + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> optionalOptions() { + final Set> options = new HashSet<>(); + options.add(PROPERTY_VERSION); + return options; + } + + @Override + public Set> requiredOptions() { + final Set> options = new HashSet<>(); + options.add(KUDU_MASTERS); + return options; + } + + @Override + public Catalog createCatalog(Context context) { + final FactoryUtil.CatalogFactoryHelper helper = + FactoryUtil.createCatalogFactoryHelper(this, context); + helper.validate(); + return new KuduDynamicCatalog(context.getName(), helper.getOptions().get(KUDU_MASTERS)); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/catalog/KuduDynamicCatalog.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/catalog/KuduDynamicCatalog.java new file mode 100644 index 0000000..5f51e06 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/dynamic/catalog/KuduDynamicCatalog.java @@ -0,0 +1,375 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.dynamic.catalog; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.table.AbstractReadOnlyCatalog; +import org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory; +import org.apache.flink.connector.kudu.table.utils.KuduTableUtils; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.Factory; +import org.apache.flink.util.StringUtils; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.client.AlterTableOptions; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.shaded.com.google.common.collect.Lists; +import org.apache.kudu.shaded.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory.KUDU_HASH_COLS; +import static org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory.KUDU_HASH_PARTITION_NUMS; +import static org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory.KUDU_PRIMARY_KEY_COLS; +import static org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory.KUDU_REPLICAS; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Catalog for reading and creating Kudu tables. */ +public class KuduDynamicCatalog extends AbstractReadOnlyCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(KuduDynamicCatalog.class); + private final KuduDynamicTableSourceSinkFactory tableFactory = + new KuduDynamicTableSourceSinkFactory(); + private final String kuduMasters; + private final KuduClient kuduClient; + + /** + * Create a new {@link KuduDynamicCatalog} with the specified catalog name and kudu master + * addresses. + * + * @param catalogName Name of the catalog (used by the table environment) + * @param kuduMasters Connection address to Kudu + */ + public KuduDynamicCatalog(String catalogName, String kuduMasters) { + super(catalogName, "default_database"); + this.kuduMasters = kuduMasters; + this.kuduClient = createClient(); + } + + /** + * Create a new {@link KuduDynamicCatalog} with the specified kudu master addresses. + * + * @param kuduMasters Connection address to Kudu + */ + public KuduDynamicCatalog(String kuduMasters) { + this("kudu", kuduMasters); + } + + @Override + public Optional getFactory() { + return Optional.of(getKuduTableFactory()); + } + + public KuduDynamicTableSourceSinkFactory getKuduTableFactory() { + return tableFactory; + } + + private KuduClient createClient() { + return new KuduClient.KuduClientBuilder(kuduMasters).build(); + } + + @Override + public void open() {} + + @Override + public void close() { + try { + if (kuduClient != null) { + kuduClient.close(); + } + } catch (KuduException e) { + LOG.error("Error while closing kudu client", e); + } + } + + public ObjectPath getObjectPath(String tableName) { + return new ObjectPath(getDefaultDatabase(), tableName); + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "databaseName cannot be null or empty"); + + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } + + try { + return kuduClient.getTablesList().getTablesList(); + } catch (Throwable t) { + throw new CatalogException("Could not list tables", t); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) { + checkNotNull(tablePath); + try { + return kuduClient.tableExists(tablePath.getObjectName()); + } catch (KuduException e) { + throw new CatalogException(e); + } + } + + @Override + public CatalogTable getTable(ObjectPath tablePath) throws TableNotExistException { + checkNotNull(tablePath); + + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + + String tableName = tablePath.getObjectName(); + + try { + KuduTable kuduTable = kuduClient.openTable(tableName); + // fixme base on TableSchema, TableSchema needs to be upgraded to ResolvedSchema + CatalogTableImpl table = + new CatalogTableImpl( + KuduTableUtils.kuduToFlinkSchema(kuduTable.getSchema()), + createTableProperties( + tableName, kuduTable.getSchema().getPrimaryKeyColumns()), + tableName); + + return table; + } catch (KuduException e) { + throw new CatalogException(e); + } + } + + protected Map createTableProperties( + String tableName, List primaryKeyColumns) { + Map props = new HashMap<>(); + props.put(KuduDynamicTableSourceSinkFactory.KUDU_MASTERS.key(), kuduMasters); + String primaryKeyNames = + primaryKeyColumns.stream() + .map(ColumnSchema::getName) + .collect(Collectors.joining(",")); + props.put(KUDU_PRIMARY_KEY_COLS.key(), primaryKeyNames); + props.put(KuduDynamicTableSourceSinkFactory.KUDU_TABLE.key(), tableName); + return props; + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException { + String tableName = tablePath.getObjectName(); + try { + if (tableExists(tablePath)) { + kuduClient.deleteTable(tableName); + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } catch (KuduException e) { + throw new CatalogException("Could not delete table " + tableName, e); + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException { + String tableName = tablePath.getObjectName(); + try { + if (tableExists(tablePath)) { + kuduClient.alterTable(tableName, new AlterTableOptions().renameTable(newTableName)); + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } catch (KuduException e) { + throw new CatalogException("Could not rename table " + tableName, e); + } + } + + public void createTable(KuduTableInfo tableInfo, boolean ignoreIfExists) + throws CatalogException, TableAlreadyExistException { + ObjectPath path = getObjectPath(tableInfo.getName()); + if (tableExists(path)) { + if (ignoreIfExists) { + return; + } else { + throw new TableAlreadyExistException(getName(), path); + } + } + + try { + kuduClient.createTable( + tableInfo.getName(), tableInfo.getSchema(), tableInfo.getCreateTableOptions()); + } catch (KuduException e) { + throw new CatalogException("Could not create table " + tableInfo.getName(), e); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException { + Map tableProperties = table.getOptions(); + TableSchema tableSchema = table.getSchema(); + + Set optionalProperties = + new HashSet<>( + Arrays.asList( + KUDU_REPLICAS.key(), + KUDU_HASH_PARTITION_NUMS.key(), + KUDU_HASH_COLS.key())); + Set requiredProperties = new HashSet<>(); + + if (!tableSchema.getPrimaryKey().isPresent()) { + requiredProperties.add(KUDU_PRIMARY_KEY_COLS.key()); + } + + if (!tableProperties.keySet().containsAll(requiredProperties)) { + throw new CatalogException( + "Missing required property. The following properties must be provided: " + + requiredProperties.toString()); + } + + Set permittedProperties = Sets.union(requiredProperties, optionalProperties); + if (!permittedProperties.containsAll(tableProperties.keySet())) { + throw new CatalogException( + "Unpermitted properties were given. The following properties are allowed:" + + permittedProperties.toString()); + } + + String tableName = tablePath.getObjectName(); + + KuduTableInfo tableInfo = + KuduTableUtils.createTableInfo(tableName, tableSchema, tableProperties); + + createTable(tableInfo, ignoreIfExists); + } + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public List listDatabases() throws CatalogException { + return Lists.newArrayList(getDefaultDatabase()); + } + + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + if (databaseName.equals(getDefaultDatabase())) { + return new CatalogDatabaseImpl(new HashMap<>(), null); + } else { + throw new DatabaseNotExistException(getName(), databaseName); + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + return listDatabases().contains(databaseName); + } + + @Override + public List listViews(String databaseName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitions(ObjectPath tablePath) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List filters) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + return false; + } + + @Override + public List listFunctions(String dbName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) + throws FunctionNotExistException, CatalogException { + throw new FunctionNotExistException(getName(), functionPath); + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + return false; + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/function/lookup/KuduLookupOptions.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/function/lookup/KuduLookupOptions.java new file mode 100644 index 0000000..f05b988 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/function/lookup/KuduLookupOptions.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.function.lookup; + +/** Options for the Kudu lookup. */ +public class KuduLookupOptions { + private final long cacheMaxSize; + private final long cacheExpireMs; + private final int maxRetryTimes; + + public static Builder builder() { + return new Builder(); + } + + public KuduLookupOptions(long cacheMaxSize, long cacheExpireMs, int maxRetryTimes) { + this.cacheMaxSize = cacheMaxSize; + this.cacheExpireMs = cacheExpireMs; + this.maxRetryTimes = maxRetryTimes; + } + + public long getCacheMaxSize() { + return cacheMaxSize; + } + + public long getCacheExpireMs() { + return cacheExpireMs; + } + + public int getMaxRetryTimes() { + return maxRetryTimes; + } + + /** Builder for KuduLookupOptions. */ + public static final class Builder { + private long cacheMaxSize; + private long cacheExpireMs; + private int maxRetryTimes; + + public static Builder options() { + return new Builder(); + } + + public Builder withCacheMaxSize(long cacheMaxSize) { + this.cacheMaxSize = cacheMaxSize; + return this; + } + + public Builder withCacheExpireMs(long cacheExpireMs) { + this.cacheExpireMs = cacheExpireMs; + return this; + } + + public Builder withMaxRetryTimes(int maxRetryTimes) { + this.maxRetryTimes = maxRetryTimes; + return this; + } + + public KuduLookupOptions build() { + return new KuduLookupOptions(cacheMaxSize, cacheExpireMs, maxRetryTimes); + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/function/lookup/KuduRowDataLookupFunction.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/function/lookup/KuduRowDataLookupFunction.java new file mode 100644 index 0000000..4ae4121 --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/function/lookup/KuduRowDataLookupFunction.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.function.lookup; + +import org.apache.flink.connector.kudu.connector.KuduFilterInfo; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.converter.RowResultConverter; +import org.apache.flink.connector.kudu.connector.converter.RowResultRowDataConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduInputSplit; +import org.apache.flink.connector.kudu.connector.reader.KuduReader; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderIterator; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.TableFunction; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.compress.utils.Lists; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.kudu.shaded.com.google.common.cache.Cache; +import org.apache.kudu.shaded.com.google.common.cache.CacheBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** LookupFunction based on the RowData object type. */ +public class KuduRowDataLookupFunction extends TableFunction { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(KuduRowDataLookupFunction.class); + + private final KuduTableInfo tableInfo; + private final KuduReaderConfig kuduReaderConfig; + private final String[] keyNames; + private final String[] projectedFields; + private final long cacheMaxSize; + private final long cacheExpireMs; + private final int maxRetryTimes; + private final RowResultConverter convertor; + + private transient Cache> cache; + private transient KuduReader kuduReader; + + private KuduRowDataLookupFunction( + String[] keyNames, + KuduTableInfo tableInfo, + KuduReaderConfig kuduReaderConfig, + String[] projectedFields, + KuduLookupOptions kuduLookupOptions) { + this.tableInfo = tableInfo; + this.convertor = new RowResultRowDataConverter(); + this.projectedFields = projectedFields; + this.keyNames = keyNames; + this.kuduReaderConfig = kuduReaderConfig; + this.cacheMaxSize = kuduLookupOptions.getCacheMaxSize(); + this.cacheExpireMs = kuduLookupOptions.getCacheExpireMs(); + this.maxRetryTimes = kuduLookupOptions.getMaxRetryTimes(); + } + + public RowData buildCacheKey(Object... keys) { + return GenericRowData.of(keys); + } + + /** + * invoke entry point of lookup function. + * + * @param keys join keys + */ + public void eval(Object... keys) { + if (keys.length != keyNames.length) { + throw new RuntimeException("The join keys are of unequal lengths"); + } + // cache key + RowData keyRow = buildCacheKey(keys); + if (this.cache != null) { + List cacheRows = this.cache.getIfPresent(keyRow); + if (CollectionUtils.isNotEmpty(cacheRows)) { + for (RowData cacheRow : cacheRows) { + collect(cacheRow); + } + return; + } + } + + for (int retry = 1; retry <= maxRetryTimes; retry++) { + try { + List kuduFilterInfos = buildKuduFilterInfo(keys); + this.kuduReader.setTableFilters(kuduFilterInfos); + KuduInputSplit[] inputSplits = kuduReader.createInputSplits(1); + ArrayList rows = new ArrayList<>(); + for (KuduInputSplit inputSplit : inputSplits) { + KuduReaderIterator scanner = + kuduReader.scanner(inputSplit.getScanToken()); + // not use cache + if (cache == null) { + while (scanner.hasNext()) { + collect(scanner.next()); + } + } else { + while (scanner.hasNext()) { + RowData row = scanner.next(); + rows.add(row); + collect(row); + } + rows.trimToSize(); + } + } + if (cache != null) { + cache.put(keyRow, rows); + } + break; + } catch (Exception e) { + LOG.error(String.format("Kudu scan error, retry times = %d", retry), e); + if (retry >= maxRetryTimes) { + throw new RuntimeException("Execution of Kudu scan failed.", e); + } + try { + Thread.sleep(1000L * retry); + } catch (InterruptedException e1) { + throw new RuntimeException(e1); + } + } + } + } + + private List buildKuduFilterInfo(Object... keyValS) { + List kuduFilterInfos = Lists.newArrayList(); + for (int i = 0; i < keyNames.length; i++) { + KuduFilterInfo kuduFilterInfo = + KuduFilterInfo.Builder.create(keyNames[i]).equalTo(keyValS[i]).build(); + kuduFilterInfos.add(kuduFilterInfo); + } + return kuduFilterInfos; + } + + @Override + public void open(FunctionContext context) { + try { + super.open(context); + this.kuduReader = + new KuduReader<>(this.tableInfo, this.kuduReaderConfig, this.convertor); + // build kudu cache + this.kuduReader.setTableProjections( + ArrayUtils.isNotEmpty(projectedFields) ? Arrays.asList(projectedFields) : null); + this.cache = + this.cacheMaxSize == -1 || this.cacheExpireMs == -1 + ? null + : CacheBuilder.newBuilder() + .expireAfterWrite(this.cacheExpireMs, TimeUnit.MILLISECONDS) + .maximumSize(this.cacheMaxSize) + .build(); + } catch (Exception ioe) { + LOG.error("Exception while creating connection to Kudu.", ioe); + throw new RuntimeException("Cannot create connection to Kudu.", ioe); + } + } + + @Override + public void close() { + if (null != this.kuduReader) { + try { + this.kuduReader.close(); + if (cache != null) { + this.cache.cleanUp(); + // help gc + this.cache = null; + } + this.kuduReader = null; + } catch (IOException e) { + // ignore exception when close. + LOG.warn("exception when close table", e); + } + } + } + + /** Builder for KuduRowDataLookupFunction. */ + public static class Builder { + private KuduTableInfo tableInfo; + private KuduReaderConfig kuduReaderConfig; + private String[] keyNames; + private String[] projectedFields; + private KuduLookupOptions kuduLookupOptions; + + public static Builder options() { + return new Builder(); + } + + public Builder tableInfo(KuduTableInfo tableInfo) { + this.tableInfo = tableInfo; + return this; + } + + public Builder kuduReaderConfig(KuduReaderConfig kuduReaderConfig) { + this.kuduReaderConfig = kuduReaderConfig; + return this; + } + + public Builder keyNames(String[] keyNames) { + this.keyNames = keyNames; + return this; + } + + public Builder projectedFields(String[] projectedFields) { + this.projectedFields = projectedFields; + return this; + } + + public Builder kuduLookupOptions(KuduLookupOptions kuduLookupOptions) { + this.kuduLookupOptions = kuduLookupOptions; + return this; + } + + public KuduRowDataLookupFunction build() { + return new KuduRowDataLookupFunction( + keyNames, tableInfo, kuduReaderConfig, projectedFields, kuduLookupOptions); + } + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/utils/KuduTableUtils.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/utils/KuduTableUtils.java new file mode 100644 index 0000000..63d721e --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/utils/KuduTableUtils.java @@ -0,0 +1,318 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.utils; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kudu.connector.ColumnSchemasFactory; +import org.apache.flink.connector.kudu.connector.CreateTableOptionsFactory; +import org.apache.flink.connector.kudu.connector.KuduFilterInfo; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.utils.TableSchemaUtils; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.ColumnTypeAttributes; +import org.apache.kudu.Schema; +import org.apache.kudu.client.CreateTableOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** Kudu table utilities. */ +public class KuduTableUtils { + + private static final Logger LOG = LoggerFactory.getLogger(KuduTableUtils.class); + + public static KuduTableInfo createTableInfo( + String tableName, TableSchema schema, Map props) { + // Since KUDU_HASH_COLS is a required property for table creation, we use it to infer + // whether to create table + boolean createIfMissing = + props.containsKey(KuduDynamicTableSourceSinkFactory.KUDU_PRIMARY_KEY_COLS.key()) + || schema.getPrimaryKey().isPresent(); + KuduTableInfo tableInfo = KuduTableInfo.forTable(tableName); + + if (createIfMissing) { + + List> columns = + getSchemaWithSqlTimestamp(schema).getTableColumns().stream() + .map(tc -> Tuple2.of(tc.getName(), tc.getType())) + .collect(Collectors.toList()); + + List keyColumns = getPrimaryKeyColumns(props, schema); + ColumnSchemasFactory schemasFactory = () -> toKuduConnectorColumns(columns, keyColumns); + int replicas = + Optional.ofNullable( + props.get( + KuduDynamicTableSourceSinkFactory.KUDU_REPLICAS.key())) + .map(Integer::parseInt) + .orElse(1); + // if hash partitions nums not exists,default 3; + int hashPartitionNums = + Optional.ofNullable( + props.get( + KuduDynamicTableSourceSinkFactory + .KUDU_HASH_PARTITION_NUMS + .key())) + .map(Integer::parseInt) + .orElse(3); + CreateTableOptionsFactory optionsFactory = + () -> + new CreateTableOptions() + .setNumReplicas(replicas) + .addHashPartitions(getHashColumns(props), hashPartitionNums); + tableInfo.createTableIfNotExists(schemasFactory, optionsFactory); + } else { + LOG.debug( + "Property {} is missing, assuming the table is already created.", + KuduDynamicTableSourceSinkFactory.KUDU_HASH_COLS.key()); + } + + return tableInfo; + } + + public static List toKuduConnectorColumns( + List> columns, Collection keyColumns) { + return columns.stream() + .map( + t -> { + ColumnSchema.ColumnSchemaBuilder builder = + new ColumnSchema.ColumnSchemaBuilder( + t.f0, KuduTypeUtils.toKuduType(t.f1)) + .key(keyColumns.contains(t.f0)) + .nullable( + !keyColumns.contains(t.f0) + && t.f1.getLogicalType().isNullable()); + if (t.f1.getLogicalType() instanceof DecimalType) { + DecimalType decimalType = ((DecimalType) t.f1.getLogicalType()); + builder.typeAttributes( + new ColumnTypeAttributes.ColumnTypeAttributesBuilder() + .precision(decimalType.getPrecision()) + .scale(decimalType.getScale()) + .build()); + } + return builder.build(); + }) + .collect(Collectors.toList()); + } + + public static TableSchema kuduToFlinkSchema(Schema schema) { + TableSchema.Builder builder = TableSchema.builder(); + + for (ColumnSchema column : schema.getColumns()) { + DataType flinkType = + KuduTypeUtils.toFlinkType(column.getType(), column.getTypeAttributes()) + .nullable(); + builder.field(column.getName(), flinkType); + } + + return builder.build(); + } + + public static List getPrimaryKeyColumns( + Map tableProperties, TableSchema tableSchema) { + return tableProperties.containsKey( + KuduDynamicTableSourceSinkFactory.KUDU_PRIMARY_KEY_COLS.key()) + ? Arrays.asList( + tableProperties + .get(KuduDynamicTableSourceSinkFactory.KUDU_PRIMARY_KEY_COLS.key()) + .split(",")) + : tableSchema.getPrimaryKey().get().getColumns(); + } + + public static List getHashColumns(Map tableProperties) { + return Arrays.asList( + tableProperties + .get(KuduDynamicTableSourceSinkFactory.KUDU_HASH_COLS.key()) + .split(",")); + } + + public static TableSchema getSchemaWithSqlTimestamp(TableSchema schema) { + TableSchema.Builder builder = new TableSchema.Builder(); + TableSchemaUtils.getPhysicalSchema(schema) + .getTableColumns() + .forEach( + tableColumn -> { + if (tableColumn.getType().getLogicalType() instanceof TimestampType) { + builder.field( + tableColumn.getName(), + tableColumn.getType().bridgedTo(Timestamp.class)); + } else { + builder.field(tableColumn.getName(), tableColumn.getType()); + } + }); + return builder.build(); + } + + /** Converts Flink Expression to KuduFilterInfo. */ + @Nullable + public static Optional toKuduFilterInfo(Expression predicate) { + LOG.debug( + "predicate summary: [{}], class: [{}], children: [{}]", + predicate.asSummaryString(), + predicate.getClass(), + predicate.getChildren()); + if (predicate instanceof CallExpression) { + CallExpression callExpression = (CallExpression) predicate; + FunctionDefinition functionDefinition = callExpression.getFunctionDefinition(); + List children = callExpression.getChildren(); + if (children.size() == 1) { + return convertUnaryIsNullExpression(functionDefinition, children); + } else if (children.size() == 2 + && !functionDefinition.equals(BuiltInFunctionDefinitions.OR)) { + return convertBinaryComparison(functionDefinition, children); + } else if (children.size() > 0 + && functionDefinition.equals(BuiltInFunctionDefinitions.OR)) { + return convertIsInExpression(children); + } + } + return Optional.empty(); + } + + private static boolean isFieldReferenceExpression(Expression exp) { + return exp instanceof FieldReferenceExpression; + } + + private static boolean isValueLiteralExpression(Expression exp) { + return exp instanceof ValueLiteralExpression; + } + + private static Optional convertUnaryIsNullExpression( + FunctionDefinition functionDefinition, List children) { + FieldReferenceExpression fieldReferenceExpression; + if (isFieldReferenceExpression(children.get(0))) { + fieldReferenceExpression = (FieldReferenceExpression) children.get(0); + } else { + return Optional.empty(); + } + // IS_NULL IS_NOT_NULL + String columnName = fieldReferenceExpression.getName(); + KuduFilterInfo.Builder builder = KuduFilterInfo.Builder.create(columnName); + if (functionDefinition.equals(BuiltInFunctionDefinitions.IS_NULL)) { + return Optional.of(builder.isNull().build()); + } else if (functionDefinition.equals(BuiltInFunctionDefinitions.IS_NOT_NULL)) { + return Optional.of(builder.isNotNull().build()); + } + return Optional.empty(); + } + + private static Optional convertBinaryComparison( + FunctionDefinition functionDefinition, List children) { + FieldReferenceExpression fieldReferenceExpression; + ValueLiteralExpression valueLiteralExpression; + if (isFieldReferenceExpression(children.get(0)) + && isValueLiteralExpression(children.get(1))) { + fieldReferenceExpression = (FieldReferenceExpression) children.get(0); + valueLiteralExpression = (ValueLiteralExpression) children.get(1); + } else if (isValueLiteralExpression(children.get(0)) + && isFieldReferenceExpression(children.get(1))) { + fieldReferenceExpression = (FieldReferenceExpression) children.get(1); + valueLiteralExpression = (ValueLiteralExpression) children.get(0); + } else { + return Optional.empty(); + } + String columnName = fieldReferenceExpression.getName(); + Object value = extractValueLiteral(fieldReferenceExpression, valueLiteralExpression); + if (value == null) { + return Optional.empty(); + } + KuduFilterInfo.Builder builder = KuduFilterInfo.Builder.create(columnName); + // GREATER GREATER_EQUAL EQUAL LESS LESS_EQUAL + if (functionDefinition.equals(BuiltInFunctionDefinitions.GREATER_THAN)) { + return Optional.of(builder.greaterThan(value).build()); + } else if (functionDefinition.equals(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL)) { + return Optional.of(builder.greaterOrEqualTo(value).build()); + } else if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS)) { + return Optional.of(builder.equalTo(value).build()); + } else if (functionDefinition.equals(BuiltInFunctionDefinitions.LESS_THAN)) { + return Optional.of(builder.lessThan(value).build()); + } else if (functionDefinition.equals(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL)) { + return Optional.of(builder.lessOrEqualTo(value).build()); + } + return Optional.empty(); + } + + private static Optional convertIsInExpression(List children) { + // IN operation will be: or(equals(field, value1), equals(field, value2), ...) in blink + // For FilterType IS_IN, all internal CallExpression's function need to be equals and + // fields need to be same + List values = new ArrayList<>(children.size()); + String columnName = ""; + for (int i = 0; i < children.size(); i++) { + if (children.get(i) instanceof CallExpression) { + CallExpression callExpression = (CallExpression) children.get(i); + FunctionDefinition functionDefinition = callExpression.getFunctionDefinition(); + List subChildren = callExpression.getChildren(); + FieldReferenceExpression fieldReferenceExpression; + ValueLiteralExpression valueLiteralExpression; + if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS) + && subChildren.size() == 2 + && isFieldReferenceExpression(subChildren.get(0)) + && isValueLiteralExpression(subChildren.get(1))) { + fieldReferenceExpression = (FieldReferenceExpression) subChildren.get(0); + valueLiteralExpression = (ValueLiteralExpression) subChildren.get(1); + String fieldName = fieldReferenceExpression.getName(); + if (i != 0 && !columnName.equals(fieldName)) { + return Optional.empty(); + } else { + columnName = fieldName; + } + Object value = + extractValueLiteral(fieldReferenceExpression, valueLiteralExpression); + if (value == null) { + return Optional.empty(); + } + values.add(i, value); + } else { + return Optional.empty(); + } + } else { + return Optional.empty(); + } + } + KuduFilterInfo.Builder builder = KuduFilterInfo.Builder.create(columnName); + return Optional.of(builder.isIn(values).build()); + } + + private static Object extractValueLiteral( + FieldReferenceExpression fieldReferenceExpression, + ValueLiteralExpression valueLiteralExpression) { + DataType fieldType = fieldReferenceExpression.getOutputDataType(); + return valueLiteralExpression.getValueAs(fieldType.getConversionClass()).orElse(null); + } +} diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/utils/KuduTypeUtils.java b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/utils/KuduTypeUtils.java new file mode 100644 index 0000000..55a40ac --- /dev/null +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/table/utils/KuduTypeUtils.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.utils; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.AtomicDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; + +import org.apache.kudu.ColumnTypeAttributes; +import org.apache.kudu.Type; + +import java.sql.Timestamp; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Kudu type utilities. */ +public class KuduTypeUtils { + + public static DataType toFlinkType(Type type, ColumnTypeAttributes typeAttributes) { + switch (type) { + case STRING: + return DataTypes.STRING(); + case FLOAT: + return DataTypes.FLOAT(); + case INT8: + return DataTypes.TINYINT(); + case INT16: + return DataTypes.SMALLINT(); + case INT32: + return DataTypes.INT(); + case INT64: + return DataTypes.BIGINT(); + case DOUBLE: + return DataTypes.DOUBLE(); + case DECIMAL: + return DataTypes.DECIMAL(typeAttributes.getPrecision(), typeAttributes.getScale()); + case BOOL: + return DataTypes.BOOLEAN(); + case BINARY: + return DataTypes.BYTES(); + case UNIXTIME_MICROS: + return new AtomicDataType(new TimestampType(3), Timestamp.class); + + default: + throw new IllegalArgumentException("Illegal var type: " + type); + } + } + + public static Type toKuduType(DataType dataType) { + checkNotNull(dataType, "type cannot be null"); + LogicalType logicalType = dataType.getLogicalType(); + return logicalType.accept(new KuduTypeLogicalTypeVisitor(dataType)); + } + + private static class KuduTypeLogicalTypeVisitor extends LogicalTypeDefaultVisitor { + + private final DataType dataType; + + KuduTypeLogicalTypeVisitor(DataType dataType) { + this.dataType = dataType; + } + + @Override + public Type visit(BooleanType booleanType) { + return Type.BOOL; + } + + @Override + public Type visit(TinyIntType tinyIntType) { + return Type.INT8; + } + + @Override + public Type visit(SmallIntType smallIntType) { + return Type.INT16; + } + + @Override + public Type visit(IntType intType) { + return Type.INT32; + } + + @Override + public Type visit(BigIntType bigIntType) { + return Type.INT64; + } + + @Override + public Type visit(FloatType floatType) { + return Type.FLOAT; + } + + @Override + public Type visit(DoubleType doubleType) { + return Type.DOUBLE; + } + + @Override + public Type visit(DecimalType decimalType) { + return Type.DECIMAL; + } + + @Override + public Type visit(TimestampType timestampType) { + return Type.UNIXTIME_MICROS; + } + + @Override + public Type visit(VarCharType varCharType) { + return Type.STRING; + } + + @Override + public Type visit(VarBinaryType varBinaryType) { + return Type.BINARY; + } + + @Override + protected Type defaultMethod(LogicalType logicalType) { + throw new UnsupportedOperationException( + String.format( + "Flink doesn't support converting type %s to Kudu type yet.", + dataType.toString())); + } + } +} diff --git a/flink-connector-kudu/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connector-kudu/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000..48d5746 --- /dev/null +++ b/flink-connector-kudu/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.connector.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory diff --git a/flink-connector-kudu/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connector-kudu/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory new file mode 100644 index 0000000..ca6fe21 --- /dev/null +++ b/flink-connector-kudu/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -0,0 +1,17 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.connector.kudu.table.KuduTableFactory +org.apache.flink.connector.kudu.table.dynamic.catalog.KuduCatalogFactory diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/connector/KuduFilterInfoTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/connector/KuduFilterInfoTest.java new file mode 100644 index 0000000..b1164da --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/connector/KuduFilterInfoTest.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector; + +import org.apache.flink.table.data.binary.BinaryStringData; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Type; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +/** Tests for {@link KuduFilterInfo}. */ +public class KuduFilterInfoTest { + + @Test + void testKuduFilterInfoWithBinaryStringData() { + String filterValue = "someValue"; + + KuduFilterInfo kuduFilterInfo = + KuduFilterInfo.Builder.create("col") + .equalTo(BinaryStringData.fromString(filterValue)) + .build(); + + ColumnSchema colSchema = new ColumnSchema.ColumnSchemaBuilder("col", Type.STRING).build(); + assertDoesNotThrow(() -> kuduFilterInfo.toPredicate(colSchema)); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/connector/KuduTestBase.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/connector/KuduTestBase.java new file mode 100644 index 0000000..7ebb062 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/connector/KuduTestBase.java @@ -0,0 +1,419 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.connector; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.connector.kudu.connector.converter.RowResultRowConverter; +import org.apache.flink.connector.kudu.connector.converter.RowResultRowDataConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduInputSplit; +import org.apache.flink.connector.kudu.connector.reader.KuduReader; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderIterator; +import org.apache.flink.connector.kudu.connector.writer.AbstractSingleOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.KuduWriter; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.connector.kudu.connector.writer.RowOperationMapper; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.types.Row; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Schema; +import org.apache.kudu.Type; +import org.apache.kudu.client.CreateTableOptions; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.shaded.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.shaded.com.google.common.collect.ImmutableList; +import org.testcontainers.shaded.com.google.common.io.Closer; +import org.testcontainers.shaded.com.google.common.net.HostAndPort; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Base class for integration tests. */ +public class KuduTestBase { + + private static final String DOCKER_IMAGE = "apache/kudu:1.13.0"; + private static final Integer KUDU_MASTER_PORT = 7051; + private static final Integer KUDU_TSERVER_PORT = 7050; + private static final Integer NUMBER_OF_REPLICA = 3; + private static final Object[][] booksTableData = { + {1001, "Java for dummies", "Tan Ah Teck", 11.11, 11}, + {1002, "More Java for dummies", "Tan Ah Teck", 22.22, 22}, + {1003, "More Java for more dummies", "Mohammad Ali", 33.33, 33}, + {1004, "A Cup of Java", "Kumar", 44.44, 44}, + {1005, "A Teaspoon of Java", "Kevin Jones", 55.55, 55} + }; + public static String[] columns = new String[] {"id", "title", "author", "price", "quantity"}; + private static GenericContainer master; + private static List> tServers; + private static HostAndPort masterAddress; + private static KuduClient kuduClient; + + @BeforeAll + public static void beforeClass() throws Exception { + Network network = Network.newNetwork(); + + ImmutableList.Builder> tServersBuilder = ImmutableList.builder(); + master = + new GenericContainer<>(DOCKER_IMAGE) + .withExposedPorts(KUDU_MASTER_PORT, 8051) + .withCommand("master") + .withNetwork(network) + .withNetworkAliases("kudu-master"); + master.start(); + masterAddress = + HostAndPort.fromParts(master.getHost(), master.getMappedPort(KUDU_MASTER_PORT)); + + for (int instance = 1; instance <= NUMBER_OF_REPLICA; instance++) { + String instanceName = "kudu-tserver-" + instance; + GenericContainer tableServer = + new GenericContainer<>(DOCKER_IMAGE) + .withExposedPorts(KUDU_TSERVER_PORT) + .withCommand("tserver") + .withEnv("KUDU_MASTERS", "kudu-master:" + KUDU_MASTER_PORT) + .withEnv( + "TSERVER_ARGS", + "--fs_wal_dir=/var/lib/kudu/tserver --logtostderr " + + " --use_hybrid_clock=false --rpc_advertised_addresses=" + + instanceName) + .withNetwork(network) + .withNetworkAliases(instanceName) + .dependsOn(master); + tableServer.start(); + tServersBuilder.add(tableServer); + } + tServers = tServersBuilder.build(); + + kuduClient = new KuduClient.KuduClientBuilder(masterAddress.toString()).build(); + } + + @AfterAll + public static void afterClass() throws Exception { + kuduClient.close(); + try (Closer closer = Closer.create()) { + closer.register(master::stop); + tServers.forEach(tabletServer -> closer.register(tabletServer::stop)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static KuduTableInfo booksTableInfo(String tableName, boolean createIfNotExist) { + + KuduTableInfo tableInfo = KuduTableInfo.forTable(tableName); + + if (createIfNotExist) { + ColumnSchemasFactory schemasFactory = + () -> { + List schemas = new ArrayList<>(); + schemas.add( + new ColumnSchema.ColumnSchemaBuilder("id", Type.INT32) + .key(true) + .build()); + schemas.add( + new ColumnSchema.ColumnSchemaBuilder("title", Type.STRING).build()); + schemas.add( + new ColumnSchema.ColumnSchemaBuilder("author", Type.STRING) + .build()); + schemas.add( + new ColumnSchema.ColumnSchemaBuilder("price", Type.DOUBLE) + .nullable(true) + .build()); + schemas.add( + new ColumnSchema.ColumnSchemaBuilder("quantity", Type.INT32) + .nullable(true) + .build()); + return schemas; + }; + + tableInfo.createTableIfNotExists( + schemasFactory, + () -> + new CreateTableOptions() + .setNumReplicas(1) + .addHashPartitions(Lists.newArrayList("id"), 2)); + } + + return tableInfo; + } + + public static List> booksDataTuple() { + return Arrays.stream(booksTableData) + .map( + row -> { + Integer rowId = (Integer) row[0]; + if (rowId % 2 == 1) { + Tuple5 values = + Tuple5.of( + (Integer) row[0], + (String) row[1], + (String) row[2], + (Double) row[3], + (Integer) row[4]); + return values; + } else { + Tuple5 values = + Tuple5.of( + (Integer) row[0], + (String) row[1], + (String) row[2], + null, + null); + return values; + } + }) + .collect(Collectors.toList()); + } + + public static TableSchema booksTableSchema() { + return TableSchema.builder() + .field("id", DataTypes.INT()) + .field("title", DataTypes.STRING()) + .field("author", DataTypes.STRING()) + .field("price", DataTypes.DOUBLE()) + .field("quantity", DataTypes.INT()) + .build(); + } + + public static List booksRowData() { + return Arrays.stream(booksTableData) + .map( + row -> { + Integer rowId = (Integer) row[0]; + if (rowId % 2 == 1) { + GenericRowData values = new GenericRowData(5); + values.setField(0, row[0]); + values.setField(1, StringData.fromString(row[1].toString())); + values.setField(2, StringData.fromString(row[2].toString())); + values.setField(3, row[3]); + values.setField(4, row[4]); + return values; + } else { + GenericRowData values = new GenericRowData(5); + values.setField(0, row[0]); + values.setField(1, StringData.fromString(row[1].toString())); + values.setField(2, StringData.fromString(row[2].toString())); + return values; + } + }) + .collect(Collectors.toList()); + } + + public static List booksDataRow() { + return Arrays.stream(booksTableData) + .map( + row -> { + Integer rowId = (Integer) row[0]; + if (rowId % 2 == 1) { + Row values = new Row(5); + values.setField(0, row[0]); + values.setField(1, row[1]); + values.setField(2, row[2]); + values.setField(3, row[3]); + values.setField(4, row[4]); + return values; + } else { + Row values = new Row(5); + values.setField(0, row[0]); + values.setField(1, row[1]); + values.setField(2, row[2]); + return values; + } + }) + .collect(Collectors.toList()); + } + + public static List booksDataPojo() { + return Arrays.stream(booksTableData) + .map( + row -> + new BookInfo( + (int) row[0], + (String) row[1], + (String) row[2], + (Double) row[3], + (int) row[4])) + .collect(Collectors.toList()); + } + + public String getMasterAddress() { + return masterAddress.toString(); + } + + public KuduClient getClient() { + return kuduClient; + } + + protected void setUpDatabase(KuduTableInfo tableInfo) { + try { + String masterAddresses = getMasterAddress(); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses).build(); + KuduWriter kuduWriter = + new KuduWriter( + tableInfo, + writerConfig, + new RowOperationMapper( + columns, AbstractSingleOperationMapper.KuduOperation.INSERT)); + booksDataRow() + .forEach( + row -> { + try { + kuduWriter.write(row); + } catch (Exception e) { + e.printStackTrace(); + } + }); + kuduWriter.close(); + } catch (Exception e) { + Assertions.fail(e.getMessage()); + } + } + + protected void cleanDatabase(KuduTableInfo tableInfo) { + try { + String masterAddresses = getMasterAddress(); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses).build(); + KuduWriter kuduWriter = + new KuduWriter( + tableInfo, + writerConfig, + new RowOperationMapper( + columns, AbstractSingleOperationMapper.KuduOperation.INSERT)); + kuduWriter.deleteTable(); + kuduWriter.close(); + } catch (Exception e) { + Assertions.fail(); + } + } + + protected List readRows(KuduTableInfo tableInfo) throws Exception { + String masterAddresses = getMasterAddress(); + KuduReaderConfig readerConfig = + KuduReaderConfig.Builder.setMasters(masterAddresses).build(); + KuduReader reader = + new KuduReader<>(tableInfo, readerConfig, new RowResultRowConverter()); + + KuduInputSplit[] splits = reader.createInputSplits(1); + List rows = new ArrayList<>(); + for (KuduInputSplit split : splits) { + KuduReaderIterator resultIterator = reader.scanner(split.getScanToken()); + while (resultIterator.hasNext()) { + Row row = resultIterator.next(); + if (row != null) { + rows.add(row); + } + } + } + reader.close(); + + return rows; + } + + protected List readRowDatas(KuduTableInfo tableInfo) throws Exception { + String masterAddresses = getMasterAddress(); + KuduReaderConfig readerConfig = + KuduReaderConfig.Builder.setMasters(masterAddresses).build(); + KuduReader reader = + new KuduReader<>(tableInfo, readerConfig, new RowResultRowDataConverter()); + + KuduInputSplit[] splits = reader.createInputSplits(1); + List rows = new ArrayList<>(); + for (KuduInputSplit split : splits) { + KuduReaderIterator resultIterator = reader.scanner(split.getScanToken()); + while (resultIterator.hasNext()) { + RowData row = resultIterator.next(); + if (row != null) { + rows.add(row); + } + } + } + reader.close(); + + return rows; + } + + protected void kuduRowsTest(List rows) { + for (Row row : rows) { + Integer rowId = (Integer) row.getField(0); + if (rowId % 2 == 1) { + Assertions.assertNotEquals(null, row.getField(3)); + Assertions.assertNotEquals(null, row.getField(4)); + } else { + Assertions.assertNull(row.getField(3)); + Assertions.assertNull(row.getField(4)); + } + } + } + + protected void validateSingleKey(String tableName) throws Exception { + KuduTable kuduTable = getClient().openTable(tableName); + Schema schema = kuduTable.getSchema(); + + assertEquals(1, schema.getPrimaryKeyColumnCount()); + assertEquals(2, schema.getColumnCount()); + + assertTrue(schema.getColumn("first").isKey()); + assertFalse(schema.getColumn("second").isKey()); + + KuduScanner scanner = getClient().newScannerBuilder(kuduTable).build(); + List rows = new ArrayList<>(); + scanner.forEach(rows::add); + + assertEquals(1, rows.size()); + assertEquals("f", rows.get(0).getString("first")); + assertEquals("s", rows.get(0).getString("second")); + } + + /** Dummy data class. */ + public static class BookInfo { + public int id, quantity; + public String title, author; + public Double price; + + public BookInfo() {} + + public BookInfo(int id, String title, String author, Double price, int quantity) { + this.id = id; + this.title = title; + this.author = author; + this.price = price; + this.quantity = quantity; + } + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduOutputFormatTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduOutputFormatTest.java new file mode 100644 index 0000000..32c454f --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduOutputFormatTest.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.format; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.writer.AbstractSingleOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.connector.kudu.connector.writer.RowOperationMapper; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.UUID; + +class KuduOutputFormatTest extends KuduTestBase { + + @Test + void testInvalidKuduMaster() { + KuduTableInfo tableInfo = booksTableInfo(UUID.randomUUID().toString(), false); + Assertions.assertThrows( + NullPointerException.class, () -> new KuduOutputFormat<>(null, tableInfo, null)); + } + + @Test + void testInvalidTableInfo() { + String masterAddresses = getMasterAddress(); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses).build(); + Assertions.assertThrows( + NullPointerException.class, () -> new KuduOutputFormat<>(writerConfig, null, null)); + } + + @Test + void testNotTableExist() { + String masterAddresses = getMasterAddress(); + KuduTableInfo tableInfo = booksTableInfo(UUID.randomUUID().toString(), false); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses).build(); + KuduOutputFormat outputFormat = + new KuduOutputFormat<>( + writerConfig, + tableInfo, + new RowOperationMapper( + KuduTestBase.columns, + AbstractSingleOperationMapper.KuduOperation.INSERT)); + Assertions.assertThrows(RuntimeException.class, () -> outputFormat.open(0, 1)); + } + + @Test + void testOutputWithStrongConsistency() throws Exception { + String masterAddresses = getMasterAddress(); + + KuduTableInfo tableInfo = booksTableInfo(UUID.randomUUID().toString(), true); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses).setStrongConsistency().build(); + KuduOutputFormat outputFormat = + new KuduOutputFormat<>( + writerConfig, + tableInfo, + new RowOperationMapper( + KuduTestBase.columns, + AbstractSingleOperationMapper.KuduOperation.INSERT)); + + outputFormat.open(0, 1); + + for (Row kuduRow : booksDataRow()) { + outputFormat.writeRecord(kuduRow); + } + outputFormat.close(); + + List rows = readRows(tableInfo); + Assertions.assertEquals(5, rows.size()); + kuduRowsTest(rows); + + cleanDatabase(tableInfo); + } + + @Test + void testOutputWithEventualConsistency() throws Exception { + String masterAddresses = getMasterAddress(); + + KuduTableInfo tableInfo = booksTableInfo(UUID.randomUUID().toString(), true); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses) + .setEventualConsistency() + .build(); + KuduOutputFormat outputFormat = + new KuduOutputFormat<>( + writerConfig, + tableInfo, + new RowOperationMapper( + KuduTestBase.columns, + AbstractSingleOperationMapper.KuduOperation.INSERT)); + + outputFormat.open(0, 1); + + for (Row kuduRow : booksDataRow()) { + outputFormat.writeRecord(kuduRow); + } + + // sleep to allow eventual consistency to finish + Thread.sleep(1000); + + outputFormat.close(); + + List rows = readRows(tableInfo); + Assertions.assertEquals(5, rows.size()); + kuduRowsTest(rows); + + cleanDatabase(tableInfo); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduRowDataInputFormatTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduRowDataInputFormatTest.java new file mode 100644 index 0000000..8c4f648 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduRowDataInputFormatTest.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.format; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.converter.RowResultRowConverter; +import org.apache.flink.connector.kudu.connector.converter.RowResultRowDataConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduInputSplit; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +class KuduRowDataInputFormatTest extends KuduTestBase { + + @Test + void testInvalidKuduMaster() { + KuduTableInfo tableInfo = booksTableInfo("books", false); + Assertions.assertThrows( + NullPointerException.class, + () -> new KuduRowDataInputFormat(null, new RowResultRowDataConverter(), tableInfo)); + } + + @Test + void testInvalidTableInfo() { + String masterAddresses = getMasterAddress(); + KuduReaderConfig readerConfig = + KuduReaderConfig.Builder.setMasters(masterAddresses).build(); + Assertions.assertThrows( + NullPointerException.class, + () -> + new KuduRowDataInputFormat( + readerConfig, new RowResultRowDataConverter(), null)); + } + + @Test + void testInputFormat() throws Exception { + KuduTableInfo tableInfo = booksTableInfo("books", true); + setUpDatabase(tableInfo); + + List rows = readRowDatas(tableInfo); + Assertions.assertEquals(5, rows.size()); + + cleanDatabase(tableInfo); + } + + @Test + void testInputFormatWithProjection() throws Exception { + KuduTableInfo tableInfo = booksTableInfo("books", true); + setUpDatabase(tableInfo); + + List rows = readRowDatas(tableInfo, "title", "id"); + Assertions.assertEquals(5, rows.size()); + + for (RowData row : rows) { + Assertions.assertEquals(2, row.getArity()); + } + + cleanDatabase(tableInfo); + } + + private List readRowDatas(KuduTableInfo tableInfo, String... fieldProjection) + throws Exception { + String masterAddresses = getMasterAddress(); + KuduReaderConfig readerConfig = + KuduReaderConfig.Builder.setMasters(masterAddresses).build(); + KuduRowDataInputFormat inputFormat = + new KuduRowDataInputFormat( + readerConfig, + new RowResultRowDataConverter(), + tableInfo, + new ArrayList<>(), + fieldProjection == null ? null : Arrays.asList(fieldProjection)); + + KuduInputSplit[] splits = inputFormat.createInputSplits(1); + List rows = new ArrayList<>(); + for (KuduInputSplit split : splits) { + inputFormat.open(split); + while (!inputFormat.reachedEnd()) { + RowData row = inputFormat.nextRecord(new GenericRowData(5)); + if (row != null) { + rows.add(row); + } + } + } + inputFormat.close(); + + return rows; + } + + private List readRows(KuduTableInfo tableInfo, String... fieldProjection) + throws Exception { + String masterAddresses = getMasterAddress(); + KuduReaderConfig readerConfig = + KuduReaderConfig.Builder.setMasters(masterAddresses).build(); + KuduRowInputFormat inputFormat = + new KuduRowInputFormat( + readerConfig, + new RowResultRowConverter(), + tableInfo, + new ArrayList<>(), + fieldProjection == null ? null : Arrays.asList(fieldProjection)); + + KuduInputSplit[] splits = inputFormat.createInputSplits(1); + List rows = new ArrayList<>(); + for (KuduInputSplit split : splits) { + inputFormat.open(split); + while (!inputFormat.reachedEnd()) { + Row row = inputFormat.nextRecord(new Row(5)); + if (row != null) { + rows.add(row); + } + } + } + inputFormat.close(); + + return rows; + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduRowInputFormatTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduRowInputFormatTest.java new file mode 100644 index 0000000..c5a9f46 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/format/KuduRowInputFormatTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.format; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.converter.RowResultRowConverter; +import org.apache.flink.connector.kudu.connector.reader.KuduInputSplit; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +class KuduRowInputFormatTest extends KuduTestBase { + + @Test + void testInvalidKuduMaster() { + KuduTableInfo tableInfo = booksTableInfo("books", false); + Assertions.assertThrows( + NullPointerException.class, + () -> new KuduRowInputFormat(null, new RowResultRowConverter(), tableInfo)); + } + + @Test + void testInvalidTableInfo() { + String masterAddresses = getMasterAddress(); + KuduReaderConfig readerConfig = + KuduReaderConfig.Builder.setMasters(masterAddresses).build(); + Assertions.assertThrows( + NullPointerException.class, + () -> new KuduRowInputFormat(readerConfig, new RowResultRowConverter(), null)); + } + + @Test + void testInputFormat() throws Exception { + KuduTableInfo tableInfo = booksTableInfo("books", true); + setUpDatabase(tableInfo); + + List rows = readRows(tableInfo); + Assertions.assertEquals(5, rows.size()); + + cleanDatabase(tableInfo); + } + + @Test + void testInputFormatWithProjection() throws Exception { + KuduTableInfo tableInfo = booksTableInfo("books", true); + setUpDatabase(tableInfo); + + List rows = readRows(tableInfo, "title", "id"); + Assertions.assertEquals(5, rows.size()); + + for (Row row : rows) { + Assertions.assertEquals(2, row.getArity()); + } + + cleanDatabase(tableInfo); + } + + private List readRows(KuduTableInfo tableInfo, String... fieldProjection) + throws Exception { + String masterAddresses = getMasterAddress(); + KuduReaderConfig readerConfig = + KuduReaderConfig.Builder.setMasters(masterAddresses).build(); + KuduRowInputFormat inputFormat = + new KuduRowInputFormat( + readerConfig, + new RowResultRowConverter(), + tableInfo, + new ArrayList<>(), + fieldProjection == null ? null : Arrays.asList(fieldProjection)); + + KuduInputSplit[] splits = inputFormat.createInputSplits(1); + List rows = new ArrayList<>(); + for (KuduInputSplit split : splits) { + inputFormat.open(split); + while (!inputFormat.reachedEnd()) { + Row row = inputFormat.nextRecord(new Row(5)); + if (row != null) { + rows.add(row); + } + } + } + inputFormat.close(); + + return rows; + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/streaming/KuduSinkTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/streaming/KuduSinkTest.java new file mode 100644 index 0000000..957a1ad --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/streaming/KuduSinkTest.java @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.streaming; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.writer.AbstractSingleOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.connector.kudu.connector.writer.RowOperationMapper; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.types.Row; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Type; +import org.apache.kudu.client.CreateTableOptions; +import org.apache.kudu.shaded.com.google.common.collect.Lists; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.UUID; + +/** Tests for {@link KuduSink}. */ +public class KuduSinkTest extends KuduTestBase { + + private static final Logger LOG = LoggerFactory.getLogger(KuduSinkTest.class); + private static final String[] columns = new String[] {"id", "uuid"}; + private static StreamingRuntimeContext context; + + @BeforeAll + static void start() { + context = Mockito.mock(StreamingRuntimeContext.class); + Mockito.when(context.isCheckpointingEnabled()).thenReturn(true); + } + + @Test + void testInvalidKuduMaster() { + KuduTableInfo tableInfo = booksTableInfo(UUID.randomUUID().toString(), false); + Assertions.assertThrows( + NullPointerException.class, + () -> + new KuduSink<>( + null, + tableInfo, + new RowOperationMapper( + columns, + AbstractSingleOperationMapper.KuduOperation.INSERT))); + } + + @Test + void testInvalidTableInfo() { + String masterAddresses = getMasterAddress(); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses).build(); + Assertions.assertThrows( + NullPointerException.class, + () -> + new KuduSink<>( + writerConfig, + null, + new RowOperationMapper( + columns, + AbstractSingleOperationMapper.KuduOperation.INSERT))); + } + + @Test + void testNotTableExist() { + String masterAddresses = getMasterAddress(); + KuduTableInfo tableInfo = booksTableInfo(UUID.randomUUID().toString(), false); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses).build(); + KuduSink sink = + new KuduSink<>( + writerConfig, + tableInfo, + new RowOperationMapper( + columns, AbstractSingleOperationMapper.KuduOperation.INSERT)); + + sink.setRuntimeContext(context); + Assertions.assertThrows(RuntimeException.class, () -> sink.open(new Configuration())); + } + + @Test + void testOutputWithStrongConsistency() throws Exception { + String masterAddresses = getMasterAddress(); + + KuduTableInfo tableInfo = booksTableInfo(UUID.randomUUID().toString(), true); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses).setStrongConsistency().build(); + KuduSink sink = + new KuduSink<>( + writerConfig, + tableInfo, + new RowOperationMapper( + KuduTestBase.columns, + AbstractSingleOperationMapper.KuduOperation.INSERT)); + + sink.setRuntimeContext(context); + sink.open(new Configuration()); + + for (Row kuduRow : booksDataRow()) { + sink.invoke(kuduRow); + } + sink.close(); + + List rows = readRows(tableInfo); + Assertions.assertEquals(5, rows.size()); + kuduRowsTest(rows); + } + + @Test + void testOutputWithEventualConsistency() throws Exception { + String masterAddresses = getMasterAddress(); + + KuduTableInfo tableInfo = booksTableInfo(UUID.randomUUID().toString(), true); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses) + .setEventualConsistency() + .build(); + KuduSink sink = + new KuduSink<>( + writerConfig, + tableInfo, + new RowOperationMapper( + KuduTestBase.columns, + AbstractSingleOperationMapper.KuduOperation.INSERT)); + + sink.setRuntimeContext(context); + sink.open(new Configuration()); + + for (Row kuduRow : booksDataRow()) { + sink.invoke(kuduRow); + } + + // sleep to allow eventual consistency to finish + Thread.sleep(1000); + + sink.close(); + + List rows = readRows(tableInfo); + Assertions.assertEquals(5, rows.size()); + kuduRowsTest(rows); + } + + @Test + void testSpeed() throws Exception { + String masterAddresses = getMasterAddress(); + + KuduTableInfo tableInfo = + KuduTableInfo.forTable("test_speed") + .createTableIfNotExists( + () -> + Lists.newArrayList( + new ColumnSchema.ColumnSchemaBuilder( + "id", Type.INT32) + .key(true) + .build(), + new ColumnSchema.ColumnSchemaBuilder( + "uuid", Type.STRING) + .build()), + () -> + new CreateTableOptions() + .setNumReplicas(3) + .addHashPartitions(Lists.newArrayList("id"), 6)); + + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(masterAddresses) + .setEventualConsistency() + .build(); + KuduSink sink = + new KuduSink<>( + writerConfig, + tableInfo, + new RowOperationMapper( + columns, AbstractSingleOperationMapper.KuduOperation.INSERT)); + + sink.setRuntimeContext(context); + sink.open(new Configuration()); + + int totalRecords = 100000; + for (int i = 0; i < totalRecords; i++) { + Row kuduRow = new Row(2); + kuduRow.setField(0, i); + kuduRow.setField(1, UUID.randomUUID().toString()); + sink.invoke(kuduRow); + } + + // sleep to allow eventual consistency to finish + Thread.sleep(1000); + + sink.close(); + + List rows = readRows(tableInfo); + Assertions.assertEquals(totalRecords, rows.size()); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduCatalogTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduCatalogTest.java new file mode 100644 index 0000000..54cb625 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduCatalogTest.java @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.writer.AbstractSingleOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.connector.kudu.connector.writer.TupleOperationMapper; +import org.apache.flink.connector.kudu.streaming.KuduSink; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.types.Row; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Schema; +import org.apache.kudu.Type; +import org.apache.kudu.client.CreateTableOptions; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.shaded.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link KuduCatalog}. */ +public class KuduCatalogTest extends KuduTestBase { + + private KuduCatalog catalog; + private StreamTableEnvironment tableEnv; + + @BeforeEach + public void init() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + catalog = new KuduCatalog(getMasterAddress()); + tableEnv = KuduTableTestUtils.createTableEnvWithBlinkPlannerStreamingMode(env); + tableEnv.registerCatalog("kudu", catalog); + tableEnv.useCatalog("kudu"); + } + + @Test + public void testCreateAlterDrop() throws Exception { + tableEnv.executeSql( + "CREATE TABLE TestTable1 (`first` STRING, `second` String) WITH ('kudu.hash-columns' = 'first', 'kudu.primary-key-columns' = 'first')"); + tableEnv.executeSql("INSERT INTO TestTable1 VALUES ('f', 's')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + // Add this once Primary key support has been enabled + // tableEnv.sqlUpdate("CREATE TABLE TestTable2 (`first` STRING, `second` String, PRIMARY + // KEY(`first`)) WITH ('kudu.hash-columns' = 'first')"); + // tableEnv.sqlUpdate("INSERT INTO TestTable2 VALUES ('f', 's')"); + + validateSingleKey("TestTable1"); + // validateSingleKey("TestTable2"); + + tableEnv.executeSql("ALTER TABLE TestTable1 RENAME TO TestTable1R"); + validateSingleKey("TestTable1R"); + + tableEnv.executeSql("DROP TABLE TestTable1R"); + assertFalse(getClient().tableExists("TestTable1R")); + } + + @Test + public void testCreateAndInsertMultiKey() throws Exception { + tableEnv.executeSql( + "CREATE TABLE TestTable3 (`first` STRING, `second` INT, third STRING) WITH ('kudu.hash-columns' = 'first,second', 'kudu.primary-key-columns' = 'first,second')"); + tableEnv.executeSql("INSERT INTO TestTable3 VALUES ('f', 2, 't')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + validateMultiKey("TestTable3"); + } + + @Test + public void testSourceProjection() throws Exception { + tableEnv.executeSql( + "CREATE TABLE TestTable5 (`second` String, `first` STRING, `third` String) WITH ('kudu.hash-columns' = 'second', 'kudu.primary-key-columns' = 'second')"); + tableEnv.executeSql("INSERT INTO TestTable5 VALUES ('s', 'f', 't')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + tableEnv.executeSql( + "CREATE TABLE TestTable6 (`first` STRING, `second` String) WITH ('kudu.hash-columns' = 'first', 'kudu.primary-key-columns' = 'first')"); + tableEnv.executeSql("INSERT INTO TestTable6 (SELECT `first`, `second` FROM TestTable5)") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + validateSingleKey("TestTable6"); + } + + @Test + public void testEmptyProjection() throws Exception { + CollectionSink.output.clear(); + tableEnv.executeSql( + "CREATE TABLE TestTableEP (`first` STRING, `second` STRING) WITH ('kudu.hash-columns' = 'first', 'kudu.primary-key-columns' = 'first')"); + tableEnv.executeSql("INSERT INTO TestTableEP VALUES ('f','s')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + tableEnv.executeSql("INSERT INTO TestTableEP VALUES ('f2','s2')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + Table result = tableEnv.sqlQuery("SELECT COUNT(*) FROM TestTableEP"); + + DataStream> resultDataStream = + tableEnv.toRetractStream(result, Types.ROW(Types.LONG)); + + resultDataStream + .map(t -> Tuple2.of(t.f0, t.f1.getField(0))) + .returns(Types.TUPLE(Types.BOOLEAN, Types.LONG)) + .addSink(new CollectionSink<>()) + .setParallelism(1); + + resultDataStream.getExecutionEnvironment().execute(); + + List> expected = + Lists.newArrayList(Tuple2.of(true, 1L), Tuple2.of(false, 1L), Tuple2.of(true, 2L)); + + assertEquals(new HashSet<>(expected), new HashSet<>(CollectionSink.output)); + CollectionSink.output.clear(); + } + + @Test + public void dataStreamEndToEstTest() throws Exception { + KuduCatalog catalog = new KuduCatalog(getMasterAddress()); + // Creating table through catalog + KuduTableFactory tableFactory = catalog.getKuduTableFactory(); + + KuduTableInfo tableInfo = + KuduTableInfo.forTable("TestTable7") + .createTableIfNotExists( + () -> + Lists.newArrayList( + new ColumnSchema.ColumnSchemaBuilder( + "k", Type.INT32) + .key(true) + .build(), + new ColumnSchema.ColumnSchemaBuilder( + "v", Type.STRING) + .build()), + () -> + new CreateTableOptions() + .setNumReplicas(1) + .addHashPartitions(Lists.newArrayList("k"), 2)); + + catalog.createTable(tableInfo, false); + + ObjectPath path = catalog.getObjectPath("TestTable7"); + CatalogTable table = catalog.getTable(path); + + List> input = + Lists.newArrayList(Tuple2.of(1, "one"), Tuple2.of(2, "two"), Tuple2.of(3, "three")); + + // Writing with simple sink + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + KuduWriterConfig writerConfig = + KuduWriterConfig.Builder.setMasters(getMasterAddress()).build(); + env.fromCollection(input) + .addSink( + new KuduSink<>( + writerConfig, + tableInfo, + new TupleOperationMapper<>( + new String[] {"k", "v"}, + AbstractSingleOperationMapper.KuduOperation.INSERT))); + env.execute(); + // Reading and validating data + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + CollectionSink.output.clear(); + tableFactory + .createTableSource(path, table) + .getDataStream(env) + .map(row -> Tuple2.of((int) row.getField(0), (String) row.getField(1))) + .returns(new TypeHint>() {}) + .addSink(new CollectionSink<>()) + .setParallelism(1); + env.execute(); + + List> expected = + Lists.newArrayList(Tuple2.of(1, "one"), Tuple2.of(2, "two"), Tuple2.of(3, "three")); + assertEquals(new HashSet<>(expected), new HashSet<>(CollectionSink.output)); + CollectionSink.output.clear(); + } + + @Test + public void testTimestamp() throws Exception { + tableEnv.executeSql( + "CREATE TABLE TestTableTsC (`first` STRING, `second` TIMESTAMP(3)) " + + "WITH ('kudu.hash-columns'='first', 'kudu.primary-key-columns'='first')"); + tableEnv.executeSql( + "INSERT INTO TestTableTsC values ('f', TIMESTAMP '2020-01-01 12:12:12.123456')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + KuduTable kuduTable = getClient().openTable("TestTableTsC"); + assertEquals(Type.UNIXTIME_MICROS, kuduTable.getSchema().getColumn("second").getType()); + + KuduScanner scanner = getClient().newScannerBuilder(kuduTable).build(); + List rows = new ArrayList<>(); + scanner.forEach(rows::add); + + assertEquals(1, rows.size()); + assertEquals("f", rows.get(0).getString(0)); + assertEquals(Timestamp.valueOf("2020-01-01 12:12:12.123"), rows.get(0).getTimestamp(1)); + } + + @Test + public void testDatatypes() throws Exception { + tableEnv.executeSql( + "CREATE TABLE TestTable8 (`first` STRING, `second` BOOLEAN, `third` BYTES," + + "`fourth` TINYINT, `fifth` SMALLINT, `sixth` INT, `seventh` BIGINT, `eighth` FLOAT, `ninth` DOUBLE, " + + "`tenth` TIMESTAMP)" + + "WITH ('kudu.hash-columns' = 'first', 'kudu.primary-key-columns' = 'first')"); + + tableEnv.executeSql( + "INSERT INTO TestTable8 values ('f', false, cast('bbbb' as BYTES), cast(12 as TINYINT)," + + "cast(34 as SMALLINT), 56, cast(78 as BIGINT), cast(3.14 as FLOAT), cast(1.2345 as DOUBLE)," + + "TIMESTAMP '2020-04-15 12:34:56.123') ") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + validateManyTypes("TestTable8"); + } + + @Test + public void testMissingPropertiesCatalog() throws Exception { + assertThrows( + TableException.class, + () -> + tableEnv.executeSql( + "CREATE TABLE TestTable9a (`first` STRING, `second` String) " + + "WITH ('kudu.primary-key-columns' = 'second')")); + assertThrows( + TableException.class, + () -> + tableEnv.executeSql( + "CREATE TABLE TestTable9b (`first` STRING, `second` String) " + + "WITH ('kudu.hash-columns' = 'first')")); + assertThrows( + TableException.class, + () -> + tableEnv.executeSql( + "CREATE TABLE TestTable9b (`first` STRING, `second` String) " + + "WITH ('kudu.primary-key-columns' = 'second', 'kudu.hash-columns' = 'first')")); + } + + private void validateManyTypes(String tableName) throws Exception { + KuduTable kuduTable = getClient().openTable(tableName); + Schema schema = kuduTable.getSchema(); + + assertEquals(Type.STRING, schema.getColumn("first").getType()); + assertEquals(Type.BOOL, schema.getColumn("second").getType()); + assertEquals(Type.BINARY, schema.getColumn("third").getType()); + assertEquals(Type.INT8, schema.getColumn("fourth").getType()); + assertEquals(Type.INT16, schema.getColumn("fifth").getType()); + assertEquals(Type.INT32, schema.getColumn("sixth").getType()); + assertEquals(Type.INT64, schema.getColumn("seventh").getType()); + assertEquals(Type.FLOAT, schema.getColumn("eighth").getType()); + assertEquals(Type.DOUBLE, schema.getColumn("ninth").getType()); + assertEquals(Type.UNIXTIME_MICROS, schema.getColumn("tenth").getType()); + + KuduScanner scanner = getClient().newScannerBuilder(kuduTable).build(); + List rows = new ArrayList<>(); + scanner.forEach(rows::add); + + assertEquals(1, rows.size()); + assertEquals("f", rows.get(0).getString(0)); + assertEquals(false, rows.get(0).getBoolean(1)); + assertEquals(ByteBuffer.wrap("bbbb".getBytes()), rows.get(0).getBinary(2)); + assertEquals(12, rows.get(0).getByte(3)); + assertEquals(34, rows.get(0).getShort(4)); + assertEquals(56, rows.get(0).getInt(5)); + assertEquals(78, rows.get(0).getLong(6)); + assertEquals(3.14, rows.get(0).getFloat(7), 0.01); + assertEquals(1.2345, rows.get(0).getDouble(8), 0.0001); + assertEquals(Timestamp.valueOf("2020-04-15 12:34:56.123"), rows.get(0).getTimestamp(9)); + } + + private void validateMultiKey(String tableName) throws Exception { + KuduTable kuduTable = getClient().openTable(tableName); + Schema schema = kuduTable.getSchema(); + + assertEquals(2, schema.getPrimaryKeyColumnCount()); + assertEquals(3, schema.getColumnCount()); + + assertTrue(schema.getColumn("first").isKey()); + assertTrue(schema.getColumn("second").isKey()); + + assertFalse(schema.getColumn("third").isKey()); + + KuduScanner scanner = getClient().newScannerBuilder(kuduTable).build(); + List rows = new ArrayList<>(); + scanner.forEach(rows::add); + + assertEquals(1, rows.size()); + assertEquals("f", rows.get(0).getString("first")); + assertEquals(2, rows.get(0).getInt("second")); + assertEquals("t", rows.get(0).getString("third")); + } + + private static class CollectionSink implements SinkFunction { + + public static List output = Collections.synchronizedList(new ArrayList<>()); + + public void invoke(T value, SinkFunction.Context context) { + output.add(value); + } + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableFactoryTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableFactoryTest.java new file mode 100644 index 0000000..8f27e61 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableFactoryTest.java @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.writer.KuduWriterConfig; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.factories.TableSinkFactory; +import org.apache.flink.table.sinks.TableSink; + +import org.apache.kudu.Type; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.shaded.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +/** Tets for table factory. */ +public class KuduTableFactoryTest extends KuduTestBase { + private StreamTableEnvironment tableEnv; + private String kuduMasters; + + @BeforeEach + public void init() { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment().setParallelism(1); + tableEnv = KuduTableTestUtils.createTableEnvWithBlinkPlannerStreamingMode(env); + kuduMasters = getMasterAddress(); + } + + @Test + public void testMissingMasters() throws Exception { + tableEnv.executeSql( + "CREATE TABLE TestTable11 (`first` STRING, `second` INT) " + + "WITH ('connector.type'='kudu', 'kudu.table'='TestTable11')"); + assertThrows( + TableException.class, + () -> tableEnv.executeSql("INSERT INTO TestTable11 values ('f', 1)")); + } + + @Test + public void testNonExistingTable() throws Exception { + tableEnv.executeSql( + "CREATE TABLE TestTable11 (`first` STRING, `second` INT) " + + "WITH ('connector.type'='kudu', 'kudu.table'='TestTable11', 'kudu.masters'='" + + kuduMasters + + "')"); + JobClient jobClient = + tableEnv.executeSql("INSERT INTO TestTable11 values ('f', 1)").getJobClient().get(); + try { + jobClient.getJobExecutionResult().get(); + fail(); + } catch (ExecutionException ee) { + assertTrue(ee.getCause() instanceof JobExecutionException); + } + } + + @Test + public void testCreateTable() throws Exception { + tableEnv.executeSql( + "CREATE TABLE TestTable11 (`first` STRING, `second` STRING) " + + "WITH ('connector.type'='kudu', 'kudu.table'='TestTable11', 'kudu.masters'='" + + kuduMasters + + "', " + + "'kudu.hash-columns'='first', 'kudu.primary-key-columns'='first')"); + + tableEnv.executeSql("INSERT INTO TestTable11 values ('f', 's')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + validateSingleKey("TestTable11"); + } + + @Test + public void testTimestamp() throws Exception { + // Timestamp should be bridged to sql.Timestamp + // Test it when creating the table... + tableEnv.executeSql( + "CREATE TABLE TestTableTs (`first` STRING, `second` TIMESTAMP(3)) " + + "WITH ('connector.type'='kudu', 'kudu.masters'='" + + kuduMasters + + "', " + + "'kudu.hash-columns'='first', 'kudu.primary-key-columns'='first')"); + tableEnv.executeSql( + "INSERT INTO TestTableTs values ('f', TIMESTAMP '2020-01-01 12:12:12.123456')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + tableEnv.executeSql("INSERT INTO TestTableTs values ('s', TIMESTAMP '2020-02-02 23:23:23')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + KuduTable kuduTable = getClient().openTable("TestTableTs"); + assertEquals(Type.UNIXTIME_MICROS, kuduTable.getSchema().getColumn("second").getType()); + + KuduScanner scanner = getClient().newScannerBuilder(kuduTable).build(); + HashSet results = new HashSet<>(); + scanner.forEach(sc -> results.add(sc.getTimestamp("second"))); + + assertEquals(2, results.size()); + List expected = + Lists.newArrayList( + Timestamp.valueOf("2020-01-01 12:12:12.123"), + Timestamp.valueOf("2020-02-02 23:23:23")); + assertEquals(new HashSet<>(expected), results); + } + + @Test + public void testExistingTable() throws Exception { + // Creating a table + tableEnv.executeSql( + "CREATE TABLE TestTable12 (`first` STRING, `second` STRING) " + + "WITH ('connector.type'='kudu', 'kudu.table'='TestTable12', 'kudu.masters'='" + + kuduMasters + + "', " + + "'kudu.hash-columns'='first', 'kudu.primary-key-columns'='first')"); + + tableEnv.executeSql("INSERT INTO TestTable12 values ('f', 's')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + // Then another one in SQL that refers to the previously created one + tableEnv.executeSql( + "CREATE TABLE TestTable12b (`first` STRING, `second` STRING) " + + "WITH ('connector.type'='kudu', 'kudu.table'='TestTable12', 'kudu.masters'='" + + kuduMasters + + "')"); + tableEnv.executeSql("INSERT INTO TestTable12b values ('f2','s2')") + .getJobClient() + .get() + .getJobExecutionResult() + .get(1, TimeUnit.MINUTES); + + // Validate that both insertions were into the same table + KuduTable kuduTable = getClient().openTable("TestTable12"); + KuduScanner scanner = getClient().newScannerBuilder(kuduTable).build(); + List rows = new ArrayList<>(); + scanner.forEach(rows::add); + + assertEquals(2, rows.size()); + assertEquals("f", rows.get(0).getString("first")); + assertEquals("s", rows.get(0).getString("second")); + assertEquals("f2", rows.get(1).getString("first")); + assertEquals("s2", rows.get(1).getString("second")); + } + + @Test + public void testTableSink() { + final TableSchema schema = + TableSchema.builder() + .field("first", DataTypes.STRING()) + .field("second", DataTypes.STRING()) + .build(); + final Map properties = new HashMap<>(); + properties.put("connector.type", "kudu"); + properties.put("kudu.masters", kuduMasters); + properties.put("kudu.table", "TestTable12"); + properties.put("kudu.ignore-not-found", "true"); + properties.put("kudu.ignore-duplicate", "true"); + properties.put("kudu.flush-interval", "10000"); + properties.put("kudu.max-buffer-size", "10000"); + + KuduWriterConfig.Builder builder = + KuduWriterConfig.Builder.setMasters(kuduMasters) + .setFlushInterval(10000) + .setMaxBufferSize(10000) + .setIgnoreDuplicate(true) + .setIgnoreNotFound(true); + KuduTableInfo kuduTableInfo = KuduTableInfo.forTable("TestTable12"); + KuduTableSink expected = new KuduTableSink(builder, kuduTableInfo, schema); + final TableSink actualSink = + TableFactoryService.find(TableSinkFactory.class, properties) + .createTableSink( + ObjectPath.fromString("default.TestTable12"), + new CatalogTableImpl(schema, properties, null)); + + assertEquals(expected, actualSink); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableSourceITCase.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableSourceITCase.java new file mode 100644 index 0000000..7971bda --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableSourceITCase.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** Integration tests for {@link KuduTableSource}. */ +public class KuduTableSourceITCase extends KuduTestBase { + private TableEnvironment tableEnv; + private KuduCatalog catalog; + + private KuduTableInfo tableInfo = null; + + @BeforeEach + void init() { + tableInfo = booksTableInfo("books", true); + setUpDatabase(tableInfo); + tableEnv = KuduTableTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + catalog = new KuduCatalog(getMasterAddress()); + tableEnv.registerCatalog("kudu", catalog); + tableEnv.useCatalog("kudu"); + } + + @AfterEach + void cleanup() { + if (tableInfo != null) { + cleanDatabase(tableInfo); + tableInfo = null; + } + } + + @Test + void testFullBatchScan() throws Exception { + CloseableIterator it = + tableEnv.executeSql("select * from books order by id").collect(); + List results = new ArrayList<>(); + it.forEachRemaining(results::add); + assertEquals(5, results.size()); + assertEquals( + "+I[1001, Java for dummies, Tan Ah Teck, 11.11, 11]", results.get(0).toString()); + tableEnv.executeSql("DROP TABLE books"); + } + + @Test + void testScanWithProjectionAndFilter() throws Exception { + // (price > 30 and price < 40) + CloseableIterator it = + tableEnv.executeSql( + "SELECT title FROM books WHERE id IN (1003, 1004) and " + + "quantity < 40") + .collect(); + List results = new ArrayList<>(); + it.forEachRemaining(results::add); + assertEquals(1, results.size()); + assertEquals("+I[More Java for more dummies]", results.get(0).toString()); + tableEnv.executeSql("DROP TABLE books"); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableSourceTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableSourceTest.java new file mode 100644 index 0000000..ac04d17 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableSourceTest.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.ScalarFunctionDefinition; +import org.apache.flink.table.types.DataType; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static java.util.Collections.singletonList; +import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AND; +import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.EQUALS; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** Unit Tests for {@link KuduTableSource}. */ +public class KuduTableSourceTest extends KuduTestBase { + private KuduTableSource kuduTableSource; + private KuduCatalog catalog; + + private static final ScalarFunction DUMMY_FUNCTION = new ScalarFunction() { + // dummy + }; + + @BeforeEach + public void init() { + KuduTableInfo tableInfo = booksTableInfo("books", true); + setUpDatabase(tableInfo); + catalog = new KuduCatalog(getMasterAddress()); + ObjectPath op = new ObjectPath("default_database", "books"); + try { + kuduTableSource = + catalog.getKuduTableFactory().createTableSource(op, catalog.getTable(op)); + } catch (TableNotExistException e) { + fail(e.getMessage()); + } + } + + @AfterEach + public void clean() { + KuduTableInfo tableInfo = booksTableInfo("books", true); + cleanDatabase(tableInfo); + } + + @Test + void testGetTableSchema() throws Exception { + TableSchema tableSchema = kuduTableSource.getTableSchema(); + assertNotNull(tableSchema); + assertArrayEquals(getFieldNames(), tableSchema.getFieldNames()); + assertArrayEquals(getFieldDataTypes(), tableSchema.getFieldDataTypes()); + } + + @Test + void testGetProducedDataType() throws Exception { + DataType producedDataType = kuduTableSource.getProducedDataType(); + assertNotNull(producedDataType); + assertEquals( + getReturnDataType(getFieldNames(), getFieldDataTypes()).notNull(), + producedDataType); + } + + @Test + void testProjectFields() throws Exception { + KuduTableSource projectedTableSource = + (KuduTableSource) kuduTableSource.projectFields(new int[] {3, 4, 1}); + // ensure copy is returned + assertTrue(kuduTableSource != projectedTableSource); + // ensure table schema is identical + assertEquals(kuduTableSource.getTableSchema(), projectedTableSource.getTableSchema()); + // ensure IF is configured with selected fields + String[] fieldNames = getFieldNames(); + DataType[] fieldDataTypes = getFieldDataTypes(); + String[] projectedFieldNames = new String[] {fieldNames[3], fieldNames[4], fieldNames[1]}; + DataType[] projectedDataTypes = + new DataType[] {fieldDataTypes[3], fieldDataTypes[4], fieldDataTypes[1]}; + assertEquals( + getReturnDataType(projectedFieldNames, projectedDataTypes), + projectedTableSource.getProducedDataType()); + } + + @Test + void testApplyPredicate() throws Exception { + // expressions for supported predicates + FieldReferenceExpression fieldReferenceExpression = + new FieldReferenceExpression("id", DataTypes.INT(), 0, 0); + ValueLiteralExpression valueLiteralExpression = new ValueLiteralExpression(1); + List args = + new ArrayList<>(Arrays.asList(fieldReferenceExpression, valueLiteralExpression)); + Expression supportedPred = new CallExpression(EQUALS, args, DataTypes.BOOLEAN()); + // unsupported predicate + Expression unsupportedPred = + new CallExpression( + new ScalarFunctionDefinition("dummy", DUMMY_FUNCTION), + singletonList(new ValueLiteralExpression(1)), + DataTypes.INT()); + // invalid predicate + Expression invalidPred = + new CallExpression(AND, Collections.emptyList(), DataTypes.ARRAY(DataTypes.INT())); + + ArrayList preds = + new ArrayList<>(Arrays.asList(supportedPred, unsupportedPred, invalidPred)); + // apply predicates on TableSource + KuduTableSource filteredTableSource = + (KuduTableSource) kuduTableSource.applyPredicate(preds); + // ensure the unable push down expressions are reserved + assertEquals(preds.size(), 2); + assertSame(unsupportedPred, preds.get(0)); + assertSame(invalidPred, preds.get(1)); + // ensure copy is returned + assertNotSame(kuduTableSource, filteredTableSource); + // ensure table schema is identical + assertEquals(kuduTableSource.getTableSchema(), filteredTableSource.getTableSchema()); + // ensure return type is identical + assertEquals( + kuduTableSource.getProducedDataType(), filteredTableSource.getProducedDataType()); + // ensure filter pushdown is correct + assertTrue(filteredTableSource.isFilterPushedDown()); + assertFalse(kuduTableSource.isFilterPushedDown()); + } + + private String[] getFieldNames() { + return new String[] {"id", "title", "author", "price", "quantity"}; + } + + private DataType[] getFieldDataTypes() { + return new DataType[] { + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.DOUBLE(), + DataTypes.INT(), + }; + } + + private DataType getReturnDataType(String[] fieldNames, DataType[] dataTypes) { + DataTypes.Field[] fields = new DataTypes.Field[fieldNames.length]; + for (int i = 0; i < fieldNames.length; i++) { + fields[i] = DataTypes.FIELD(fieldNames[i], dataTypes[i]); + } + return DataTypes.ROW(fields); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableTestUtils.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableTestUtils.java new file mode 100644 index 0000000..88cb81d --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/KuduTableTestUtils.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM; + +/** Table API test utilities. */ +public class KuduTableTestUtils { + + public static StreamTableEnvironment createTableEnvWithBlinkPlannerStreamingMode( + StreamExecutionEnvironment env) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); + StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings); + tableEnv.getConfig() + .getConfiguration() + .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1); + return tableEnv; + } + + public static TableEnvironment createTableEnvWithBlinkPlannerBatchMode() { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + TableEnvironment tableEnv = TableEnvironment.create(settings); + tableEnv.getConfig() + .getConfiguration() + .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1); + return tableEnv; + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicSinkTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicSinkTest.java new file mode 100644 index 0000000..5b417c8 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicSinkTest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.dynamic; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** Unit Tests for {@link KuduDynamicTableSink}. */ +public class KuduDynamicSinkTest extends KuduTestBase { + public static final String INPUT_TABLE = "books"; + public static StreamExecutionEnvironment env; + public static TableEnvironment tEnv; + + @BeforeEach + public void init() { + KuduTableInfo tableInfo = booksTableInfo(INPUT_TABLE, true); + setUpDatabase(tableInfo); + env = StreamExecutionEnvironment.getExecutionEnvironment(); + tEnv = StreamTableEnvironment.create(env); + } + + @AfterEach + public void clean() { + KuduTableInfo tableInfo = booksTableInfo(INPUT_TABLE, true); + cleanDatabase(tableInfo); + } + + @Test + public void testKuduSink() throws Exception { + String createSql = + "CREATE TABLE " + + INPUT_TABLE + + "(" + + "id int," + + "title string," + + "author string," + + "price double," + + "quantity int" + + ") WITH (" + + " 'connector'='kudu'," + + " 'kudu.masters'='" + + 123245 + + "'," + + " 'kudu.table'='" + + INPUT_TABLE + + "','kudu.primary-key-columns'='id" + + "','kudu.max-buffer-size'='1024" + + "','kudu.flush-interval'='1000" + + "','kudu.operation-timeout'='500" + + "','kudu.ignore-not-found'='true" + + "','kudu.ignore-not-found'='true'" + + ")"; + tEnv.executeSql(createSql); + tEnv.executeSql( + "insert into " + INPUT_TABLE + " values(1006,'test title','test author',10.1,10)"); + CloseableIterator collected = + tEnv.executeSql("select * from " + INPUT_TABLE + " where id =1006").collect(); + assertNotNull(collected); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicSourceTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicSourceTest.java new file mode 100644 index 0000000..967d61c --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduDynamicSourceTest.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.dynamic; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** Unit Tests for {@link KuduDynamicTableSource}. */ +public class KuduDynamicSourceTest extends KuduTestBase { + public static final String INPUT_TABLE = "books"; + public static StreamExecutionEnvironment env; + public static TableEnvironment tEnv; + + @BeforeEach + public void init() { + KuduTableInfo tableInfo = booksTableInfo(INPUT_TABLE, true); + setUpDatabase(tableInfo); + env = StreamExecutionEnvironment.getExecutionEnvironment(); + tEnv = StreamTableEnvironment.create(env); + } + + @AfterEach + public void clean() { + KuduTableInfo tableInfo = booksTableInfo(INPUT_TABLE, true); + cleanDatabase(tableInfo); + } + + @Test + public void testKuduSource() throws Exception { + // "id", "title", "author", "price", "quantity" + tEnv.executeSql( + "CREATE TABLE " + + INPUT_TABLE + + "(" + + "id int," + + "title string," + + "author string," + + "price double," + + "quantity int" + + ") WITH (" + + " 'connector'='kudu'," + + " 'kudu.masters'='" + + getMasterAddress() + + "'," + + " 'kudu.table'='" + + INPUT_TABLE + + "'," + + "'kudu.scan.row-size'='10'," + + "'kudu.primary-key-columns'='id'" + + ")"); + + Iterator collected = tEnv.executeSql("SELECT * FROM " + INPUT_TABLE).collect(); + assertNotNull(collected); + } + + @Test + public void testProject() throws Exception { + tEnv.executeSql( + "CREATE TABLE " + + INPUT_TABLE + + "(" + + "id int," + + "title string," + + "author string," + + "price double," + + "quantity int" + + ") WITH (" + + " 'connector'='kudu'," + + " 'kudu.masters'='" + + getMasterAddress() + + "'," + + " 'kudu.table'='" + + INPUT_TABLE + + "'," + + "'kudu.scan.row-size'='10'," + + "'kudu.primary-key-columns'='id'" + + ")"); + + Iterator collected = + tEnv.executeSql("SELECT id,title,author FROM " + INPUT_TABLE).collect(); + assertNotNull(collected); + List result = + CollectionUtil.iteratorToList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + List expected = + Stream.of( + "+I[1001, Java for dummies, Tan Ah Teck]", + "+I[1002, More Java for dummies, Tan Ah Teck]", + "+I[1003, More Java for more dummies, Mohammad Ali]", + "+I[1004, A Cup of Java, Kumar]", + "+I[1005, A Teaspoon of Java, Kevin Jones]") + .sorted() + .collect(Collectors.toList()); + assertEquals(expected, result); + } + + @Test + public void testLimit() throws Exception { + tEnv.executeSql( + "CREATE TABLE " + + INPUT_TABLE + + "(" + + "id int," + + "title string," + + "author string," + + "price double," + + "quantity int" + + ") WITH (" + + " 'connector'='kudu'," + + " 'kudu.masters'='" + + getMasterAddress() + + "'," + + " 'kudu.table'='" + + INPUT_TABLE + + "'," + + "'kudu.scan.row-size'='10'," + + "'kudu.primary-key-columns'='id'" + + ")"); + + Iterator collected = + tEnv.executeSql("SELECT * FROM " + INPUT_TABLE + " LIMIT 1").collect(); + List result = + CollectionUtil.iteratorToList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + assertEquals(1, result.size()); + } + + @Test + public void testLookupJoin() { + tEnv.executeSql( + "CREATE TABLE " + + INPUT_TABLE + + "(" + + "id int," + + "title string," + + "author string," + + "price double," + + "quantity int" + + ") WITH (" + + " 'connector'='kudu'," + + " 'kudu.masters'='" + + getMasterAddress() + + "'," + + " 'kudu.table'='" + + INPUT_TABLE + + "'," + + "'kudu.scan.row-size'='10'," + + "'kudu.primary-key-columns'='id'" + + ")"); + + tEnv.executeSql( + "CREATE TABLE datagen" + + "(" + + "id int," + + "isbn string," + + "proctime as PROCTIME()" + + ") WITH (" + + " 'connector'='datagen'," + + " 'number-of-rows'='5'," + + " 'fields.id.kind'='sequence'," + + " 'fields.isbn.kind'='sequence'," + + " 'fields.id.start'='1001'," + + " 'fields.isbn.start'='1'," + + " 'fields.id.end'='1005'," + + " 'fields.isbn.end'='5'" + + ")"); + + Iterator collected = + tEnv.executeSql( + "SELECT d.id, isbn, title FROM datagen as d" + + " JOIN " + + INPUT_TABLE + + " FOR SYSTEM_TIME AS OF d.proctime AS k" + + " ON d.id=k.id" + + " WHERE k.title='Java for dummies'") + .collect(); + assertNotNull(collected); + List result = + CollectionUtil.iteratorToList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + assertEquals(1, result.size()); + assertEquals("+I[1001, 1, Java for dummies]", result.get(0)); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduRowDataLookupFunctionTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduRowDataLookupFunctionTest.java new file mode 100644 index 0000000..bc728ec --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/table/dynamic/KuduRowDataLookupFunctionTest.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.table.dynamic; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.table.function.lookup.KuduLookupOptions; +import org.apache.flink.connector.kudu.table.function.lookup.KuduRowDataLookupFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Collector; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** Unit Tests for {@link KuduRowDataLookupFunction}. */ +public class KuduRowDataLookupFunctionTest extends KuduTestBase { + public static final String INPUT_TABLE = "books"; + public static KuduTableInfo tableInfo; + + @BeforeEach + public void init() { + tableInfo = booksTableInfo(INPUT_TABLE, true); + setUpDatabase(tableInfo); + } + + @AfterEach + public void clean() { + KuduTableInfo tableInfo = booksTableInfo(INPUT_TABLE, true); + cleanDatabase(tableInfo); + } + + @Test + public void testEval() throws Exception { + KuduLookupOptions lookupOptions = KuduLookupOptions.builder().build(); + + KuduRowDataLookupFunction lookupFunction = + buildRowDataLookupFunction(lookupOptions, new String[] {"id"}); + + ListOutputCollector collector = new ListOutputCollector(); + lookupFunction.setCollector(collector); + + lookupFunction.open(null); + + lookupFunction.eval(1001); + + lookupFunction.eval(1002); + + lookupFunction.eval(1003); + + List result = + new ArrayList<>(collector.getOutputs()) + .stream().map(RowData::toString).sorted().collect(Collectors.toList()); + + assertNotNull(result); + } + + @Test + public void testCacheEval() throws Exception { + KuduLookupOptions lookupOptions = + KuduLookupOptions.builder() + .withCacheMaxSize(1024) + .withMaxRetryTimes(3) + .withCacheExpireMs(10) + .build(); + + KuduRowDataLookupFunction lookupFunction = + buildRowDataLookupFunction(lookupOptions, new String[] {"id"}); + + ListOutputCollector collector = new ListOutputCollector(); + lookupFunction.setCollector(collector); + + lookupFunction.open(null); + + lookupFunction.eval(1001); + + lookupFunction.eval(1002); + + lookupFunction.eval(1003); + + List result = + new ArrayList<>(collector.getOutputs()) + .stream().map(RowData::toString).sorted().collect(Collectors.toList()); + + assertNotNull(result); + } + + private KuduRowDataLookupFunction buildRowDataLookupFunction( + KuduLookupOptions lookupOptions, String[] keyNames) { + KuduReaderConfig config = + KuduReaderConfig.Builder.setMasters(getMasterAddress()).setRowLimit(10).build(); + return new KuduRowDataLookupFunction.Builder() + .kuduReaderConfig(config) + .kuduLookupOptions(lookupOptions) + .keyNames(keyNames) + .projectedFields(getFieldNames()) + .tableInfo(tableInfo) + .build(); + } + + private String[] getFieldNames() { + return new String[] {"id", "title", "author", "price", "quantity"}; + } + + private static final class ListOutputCollector implements Collector { + + private final List output = new ArrayList<>(); + + @Override + public void collect(RowData row) { + this.output.add(row); + } + + @Override + public void close() {} + + public List getOutputs() { + return output; + } + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/AbstractOperationTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/AbstractOperationTest.java new file mode 100644 index 0000000..6b8fa0b --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/AbstractOperationTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.writer; + +import org.apache.flink.connector.kudu.connector.KuduTestBase; + +import org.apache.kudu.Schema; +import org.apache.kudu.client.Delete; +import org.apache.kudu.client.Insert; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.PartialRow; +import org.apache.kudu.client.Update; +import org.apache.kudu.client.Upsert; +import org.junit.jupiter.api.BeforeEach; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.mockito.Mockito.when; + +/** Base class for operation tests. */ +public abstract class AbstractOperationTest { + + public static final Schema TABLE_SCHEMA = + KuduTestBase.booksTableInfo("test_table", true).getSchema(); + @Mock Insert mockInsert; + @Mock Upsert mockUpsert; + @Mock Update mockUpdate; + @Mock Delete mockDelete; + @Mock KuduTable mockTable; + @Mock PartialRow mockPartialRow; + + @BeforeEach + public void setup() { + MockitoAnnotations.initMocks(this); + when(mockInsert.getRow()).thenReturn(mockPartialRow); + when(mockUpsert.getRow()).thenReturn(mockPartialRow); + when(mockUpdate.getRow()).thenReturn(mockPartialRow); + when(mockDelete.getRow()).thenReturn(mockPartialRow); + when(mockTable.newInsert()).thenReturn(mockInsert); + when(mockTable.newUpsert()).thenReturn(mockUpsert); + when(mockTable.newUpdate()).thenReturn(mockUpdate); + when(mockTable.newDelete()).thenReturn(mockDelete); + when(mockTable.getSchema()).thenReturn(TABLE_SCHEMA); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/PojoOperationMapperTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/PojoOperationMapperTest.java new file mode 100644 index 0000000..781f73f --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/PojoOperationMapperTest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.writer; + +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.writer.AbstractSingleOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.PojoOperationMapper; + +import org.apache.kudu.client.Operation; +import org.apache.kudu.client.PartialRow; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +/** Tests for {@link PojoOperationMapper}. */ +public class PojoOperationMapperTest extends AbstractOperationTest { + + @Test + void testPojoMapper() { + + PojoOperationMapper mapper = + new PojoOperationMapper<>( + KuduTestBase.BookInfo.class, + KuduTestBase.columns, + AbstractSingleOperationMapper.KuduOperation.INSERT); + + KuduTestBase.BookInfo bookInfo = KuduTestBase.booksDataPojo().get(0); + + assertEquals(bookInfo.id, mapper.getField(bookInfo, 0)); + assertEquals(bookInfo.title, mapper.getField(bookInfo, 1)); + assertEquals(bookInfo.author, mapper.getField(bookInfo, 2)); + assertEquals(bookInfo.price, mapper.getField(bookInfo, 3)); + assertEquals(bookInfo.quantity, mapper.getField(bookInfo, 4)); + + List operations = mapper.createOperations(bookInfo, mockTable); + assertEquals(1, operations.size()); + + PartialRow row = operations.get(0).getRow(); + Mockito.verify(row, Mockito.times(1)).addObject("id", bookInfo.id); + Mockito.verify(row, Mockito.times(1)).addObject("quantity", bookInfo.quantity); + + Mockito.verify(row, Mockito.times(1)).addObject("title", bookInfo.title); + Mockito.verify(row, Mockito.times(1)).addObject("author", bookInfo.author); + + Mockito.verify(row, Mockito.times(1)).addObject("price", bookInfo.price); + } + + @Test + public void testFieldInheritance() { + PojoOperationMapper mapper = + new PojoOperationMapper<>( + Second.class, + new String[] {"s1", "i1", "i2"}, + AbstractSingleOperationMapper.KuduOperation.INSERT); + Second s = new Second(); + assertEquals("s1", mapper.getField(s, 0)); + assertEquals(1, mapper.getField(s, 1)); + assertEquals(2, mapper.getField(s, 2)); + } + + private static class First { + private int i1 = 1; + public int i2 = 2; + private String s1 = "ignore"; + } + + private static class Second extends First { + private String s1 = "s1"; + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/RowDataUpsertOperationMapperTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/RowDataUpsertOperationMapperTest.java new file mode 100644 index 0000000..8e6f441 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/RowDataUpsertOperationMapperTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.writer; + +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.writer.RowDataUpsertOperationMapper; +import org.apache.flink.table.data.RowData; + +import org.apache.kudu.client.Operation; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.verify; + +/** Unit Tests for {@link RowDataUpsertOperationMapper}. */ +public class RowDataUpsertOperationMapperTest extends AbstractOperationTest { + + @Test + void testGetField() { + RowDataUpsertOperationMapper mapper = + new RowDataUpsertOperationMapper(KuduTestBase.booksTableSchema()); + RowData inputRow = KuduTestBase.booksRowData().get(0); + + Assertions.assertEquals(inputRow.getInt(0), mapper.getField(inputRow, 0)); + } + + @Test + void testCorrectOperationUpsert() { + RowDataUpsertOperationMapper mapper = + new RowDataUpsertOperationMapper(KuduTestBase.booksTableSchema()); + RowData inputRow = KuduTestBase.booksRowData().get(0); + + List operations = mapper.createOperations(inputRow, mockTable); + + assertEquals(1, operations.size()); + verify(mockTable).newUpsert(); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/RowOperationMapperTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/RowOperationMapperTest.java new file mode 100644 index 0000000..0a5f196 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/RowOperationMapperTest.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.writer; + +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.writer.AbstractSingleOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.RowOperationMapper; +import org.apache.flink.types.Row; + +import org.apache.kudu.client.Operation; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.verify; + +/** Tests for {@link RowOperationMapper}. */ +public class RowOperationMapperTest extends AbstractOperationTest { + + @Test + void testGetField() { + RowOperationMapper mapper = + new RowOperationMapper( + KuduTestBase.columns, AbstractSingleOperationMapper.KuduOperation.INSERT); + Row inputRow = KuduTestBase.booksDataRow().get(0); + + for (int i = 0; i < inputRow.getArity(); i++) { + Assertions.assertEquals(inputRow.getField(i), mapper.getField(inputRow, i)); + } + } + + @Test + void testCorrectOperationInsert() { + RowOperationMapper mapper = + new RowOperationMapper( + KuduTestBase.columns, AbstractSingleOperationMapper.KuduOperation.INSERT); + Row inputRow = KuduTestBase.booksDataRow().get(0); + + List operations = mapper.createOperations(inputRow, mockTable); + + assertEquals(1, operations.size()); + verify(mockTable).newInsert(); + } + + @Test + void testCorrectOperationUpsert() { + RowOperationMapper mapper = + new RowOperationMapper( + KuduTestBase.columns, AbstractSingleOperationMapper.KuduOperation.UPSERT); + Row inputRow = KuduTestBase.booksDataRow().get(0); + + List operations = mapper.createOperations(inputRow, mockTable); + + assertEquals(1, operations.size()); + verify(mockTable).newUpsert(); + } +} diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/TupleOperationMapperTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/TupleOperationMapperTest.java new file mode 100644 index 0000000..cd7c721 --- /dev/null +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connector/kudu/writer/TupleOperationMapperTest.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kudu.writer; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.connector.kudu.connector.KuduTestBase; +import org.apache.flink.connector.kudu.connector.writer.AbstractSingleOperationMapper; +import org.apache.flink.connector.kudu.connector.writer.TupleOperationMapper; + +import org.apache.kudu.client.Operation; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.verify; + +/** Tests for {@link TupleOperationMapper}. */ +public class TupleOperationMapperTest extends AbstractOperationTest { + @Test + void testGetField() { + TupleOperationMapper> mapper = + new TupleOperationMapper<>( + KuduTestBase.columns, AbstractSingleOperationMapper.KuduOperation.INSERT); + Tuple5 inputTuple = + KuduTestBase.booksDataTuple().get(0); + + for (int i = 0; i < inputTuple.getArity(); i++) { + Assertions.assertEquals(inputTuple.getField(i), mapper.getField(inputTuple, i)); + } + } + + @Test + void testCorrectOperationInsert() { + TupleOperationMapper> mapper = + new TupleOperationMapper<>( + KuduTestBase.columns, AbstractSingleOperationMapper.KuduOperation.INSERT); + Tuple5 inputTuple = + KuduTestBase.booksDataTuple().get(0); + + List operations = mapper.createOperations(inputTuple, mockTable); + + assertEquals(1, operations.size()); + verify(mockTable).newInsert(); + } + + @Test + void testCorrectOperationUpsert() { + TupleOperationMapper> mapper = + new TupleOperationMapper<>( + KuduTestBase.columns, AbstractSingleOperationMapper.KuduOperation.UPSERT); + Tuple5 inputTuple = + KuduTestBase.booksDataTuple().get(0); + + List operations = mapper.createOperations(inputTuple, mockTable); + + assertEquals(1, operations.size()); + verify(mockTable).newUpsert(); + } +} diff --git a/flink-connector-kudu/src/test/resources/log4j.properties b/flink-connector-kudu/src/test/resources/log4j.properties new file mode 100644 index 0000000..15efe08 --- /dev/null +++ b/flink-connector-kudu/src/test/resources/log4j.properties @@ -0,0 +1,27 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +# This file ensures that tests executed from the IDE show log output + +log4j.rootLogger=WARN, console + +# Log all infos in the given file +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/flink-connector-kudu/src/test/resources/log4j2-test.properties b/flink-connector-kudu/src/test/resources/log4j2-test.properties new file mode 100644 index 0000000..e463a0e --- /dev/null +++ b/flink-connector-kudu/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = INFO +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000..d1548f7 --- /dev/null +++ b/pom.xml @@ -0,0 +1,353 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connector-parent + 1.1.0 + + + flink-connector-kudu-parent + 2.0-SNAPSHOT + + Flink : Connectors : Kudu Parent + pom + https://flink.apache.org + 2024 + + + + The Apache Software License, Version 2.0 + https://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + https://github.com/apache/flink-connector-kudu + git@github.com:apache/flink-connector-kudu.git + scm:git:https://gitbox.apache.org/repos/asf/flink-connector-kudu.git + + + + 1.17.2 + 1.13.0 + 2.12 + + 3.25.3 + 5.10.2 + 1.10.19 + 1.17.6 + + 2.23.1 + 1.7.36 + + + flink-connector-kudu-parent + + --add-opens=java.base/java.lang=ALL-UNNAMED + --add-opens=java.base/java.util=ALL-UNNAMED + + + + + flink-connector-kudu + + + + + + + + org.assertj + assertj-core + test + + + + + org.apache.logging.log4j + log4j-slf4j-impl + test + + + + org.apache.logging.log4j + log4j-api + test + + + + org.apache.logging.log4j + log4j-core + test + + + + + org.apache.logging.log4j + log4j-1.2-api + test + + + + + + + + org.apache.flink + flink-clients + ${flink.version} + + + + org.apache.flink + flink-streaming-java + ${flink.version} + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + + + + org.apache.flink + flink-table-common + ${flink.version} + + + + org.apache.flink + flink-table-planner-loader + ${flink.version} + + + + org.apache.flink + flink-table-runtime + ${flink.version} + + + + org.apache.flink + flink-test-utils + ${flink.version} + + + + + org.apache.flink + flink-architecture-tests-test + ${flink.version} + + + + org.apache.kudu + kudu-client + ${kudu.version} + + + + org.apache.kudu + kudu-test-utils + ${kudu.version} + + + junit + junit + + + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + + org.apache.logging.log4j + log4j-1.2-api + ${log4j.version} + + + + org.assertj + assertj-core + ${assertj.version} + + + + org.mockito + mockito-all + ${mockito.version} + + + + org.junit.jupiter + junit-jupiter-migrationsupport + ${junit5.version} + + + + org.junit + junit-bom + ${junit5.version} + pom + import + + + + org.testcontainers + testcontainers-bom + ${testcontainers.version} + pom + import + + + + + com.esotericsoftware.kryo + kryo + 2.24.0 + + + + org.apache.commons + commons-lang3 + 3.12.0 + + + + + + + + + org.codehaus.mojo + exec-maven-plugin + false + + + org.apache.flink + flink-ci-tools + ${flink.version} + + + + + + org.apache.rat + apache-rat-plugin + false + + + + com.diffplug.spotless + spotless-maven-plugin + + + + org.apache.maven.plugins + maven-jar-plugin + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + + org.apache.maven.plugins + maven-shade-plugin + + + + org.commonjava.maven.plugins + directory-maven-plugin + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + + + + java21 + + [21,) + + + + + + com.diffplug.spotless + spotless-maven-plugin + + + true + + + + + + + + diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties new file mode 100644 index 0000000..7daf1c3 --- /dev/null +++ b/tools/ci/log4j.properties @@ -0,0 +1,43 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +rootLogger.level = INFO +rootLogger.appenderRef.out.ref = ConsoleAppender + +# ----------------------------------------------------------------------------- +# Console (use 'console') +# ----------------------------------------------------------------------------- + +appender.console.name = ConsoleAppender +appender.console.type = CONSOLE +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n + +# ----------------------------------------------------------------------------- +# File (use 'file') +# ----------------------------------------------------------------------------- +appender.file.name = FileAppender +appender.file.type = FILE +appender.file.fileName = ${sys:log.dir}/mvn-${sys:mvn.forkNumber:-output}.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n +appender.file.createOnDemand = true + +# suppress the irrelevant (wrong) warnings from the netty channel handler +logger.netty.name = org.jboss.netty.channel.DefaultChannelPipeline +logger.netty.level = ERROR diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml new file mode 100644 index 0000000..2048fd1 --- /dev/null +++ b/tools/maven/checkstyle.xml @@ -0,0 +1,562 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml new file mode 100644 index 0000000..4f80ec2 --- /dev/null +++ b/tools/maven/suppressions.xml @@ -0,0 +1,26 @@ + + + + + + +