diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index 7ea29bda1c8b..1acf865149ad 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -130,6 +130,15 @@ + + + + + + + + + diff --git a/docs/release-template.md b/docs/release-template.md index ed543d046a92..e589339531db 100644 --- a/docs/release-template.md +++ b/docs/release-template.md @@ -62,6 +62,8 @@ ## Oracle connector +## Paimon connector + ## Pinot connector ## PostgreSQL connector diff --git a/docs/src/main/sphinx/connector.md b/docs/src/main/sphinx/connector.md index 6805fdca440a..6fe8860919ad 100644 --- a/docs/src/main/sphinx/connector.md +++ b/docs/src/main/sphinx/connector.md @@ -33,6 +33,7 @@ MongoDB MySQL OpenSearch Oracle +Paimon Pinot PostgreSQL Prometheus diff --git a/docs/src/main/sphinx/connector/paimon.md b/docs/src/main/sphinx/connector/paimon.md new file mode 100644 index 000000000000..98a491a7e017 --- /dev/null +++ b/docs/src/main/sphinx/connector/paimon.md @@ -0,0 +1,222 @@ +# Paimon connector + +```{raw} html + +``` + +The Paimon connector enables querying [Paimon](https://paimon.apache.org/docs/master/) tables. + +Apache Paimon is an open table format for huge analytic datasets, and it is +also very friendly to streaming read/write. +The Paimon connector allows querying data stored in files written +in [Paimon format](https://paimon.apache.org/docs/master/). + +## Requirements + +To use the Paimon connector, you need: + +- Network access from the Trino coordinator and workers to the distributed + object storage. + +- Data files stored in the file formats + [Parquet](parquet-format-configuration)(default), + [ORC](orc-format-configuration), or Avro on a filesystem that is supported by trino. + +## General configuration + +To configure the Paimon connector, create a catalog properties file +`etc/catalog/paimon.properties` that references the `paimon` connector. + +You must select and configure one of the [supported file +systems](paimon-file-system-configuration). + +```properties +connector.name=paimon +paimon.catalog.type=filesystem +paimon.warehouse=s3://your-bucket/path/to/warehouse +fs.x.enabled=true +``` + +Replace the `fs.x.enabled` configuration property with the desired file system. + +Additionally, following configuration properties can be set depending on the use-case: + +:::{list-table} Paimon configuration properties +:widths: 30, 58, 12 +:header-rows: 1 + +* - Property name + - Description + - Default +* - `paimon.catalog.type` + - Possible values are: + * `filesystem` + * `hive` + - `filesystem` +* - `paimon.warehouse` + - The root path of paimon tables. + - `No default` +* - `paimon.projection-pushdown-enabled` + - Enable [projection pushdown](/optimizer/pushdown) + - `true` +* - `paimon.metadata-cache.enabled` + - Set to `false` to disable in-memory caching of metadata files on the + coordinator. This cache is not used when `fs.cache.enabled` is set to true. + - `true` +::: + +(paimon-file-system-configuration)= +## File system access configuration + +The connector supports accessing the following file systems: + +* [](/object-storage/file-system-azure) +* [](/object-storage/file-system-gcs) +* [](/object-storage/file-system-s3) +* [](/object-storage/file-system-hdfs) + +You must enable and configure the specific file system access. [Legacy +support](file-system-legacy) is not recommended and will be removed. + + +## Paimon to Trino type mapping + +The connector maps Paimon types to the corresponding Trino types according to +the following table: + +:::{list-table} Paimon type to Trino type mapping +:widths: 40, 60 +:header-rows: 1 + +* - Paimon type + - Trino type +* - `BOOLEAN` + - `BOOLEAN` +* - `TINYINT` + - `TINYINT` +* - `SMALLINT` + - `SMALLINT` +* - `INT` + - `INTEGER` +* - `BIGINT` + - `BIGINT` +* - `FLOAT` + - `REAL` +* - `DOUBLE` + - `DOUBLE` +* - `DECIMAL(p,s)` + - `DECIMAL(p,s)` +* - `DATE` + - `DATE` +* - `TIME(n)` + - `TIME(n)` +* - `TIMESTAMP(n)` + - `TIMESTAMP(n)` +* - `TIMESTAMP_WITH_LOCAL_TIME_ZONE(n)` + - `TIMESTAMP(n) WITH TIME ZONE` +* - `CHAR` + - `CHAR` +* - `VARCHAR` + - `VARCHAR` +* - `STRING` + - `VARCHAR` +* - `BINARY` + - `VARBINARY` +* - `VARBINARY` + - `VARBINARY` +::: + +No other types are supported. (MAP, LIST, STRUCT types are not supported yet.) + +## SQL support + +The connector provides read access to data in the Paimon table. +The {ref}`globally available ` +and {ref}`read operation ` statements are supported. + +### Basic usage examples + +In the following example queries, `nation` is the Paimon append table loaded by tpch. +The target file system is s3, which mocked by minio in the local environment. + +My paimon.properties in etc/catalog shows as below: +```properties +connector.name=paimon +paimon.catalog.type=filesystem +paimon.warehouse=s3://test-paimon-connector-ryo1qhtrl9/ +fs.native-s3.enabled=true +s3.aws-access-key=accesskey +s3.aws-secret-key=secretkey +s3.region=us-east-1 +s3.endpoint=http://localhost:32768 +s3.path-style-access=true +``` + + +```sql +USE use paimon.tests; + +SELECT * FROM nation LIMIT 1; +``` + +```text + nationkey | name | regionkey | comment +-----------+---------+-----------+----------------------------------------------------- + 0 | ALGERIA | 0 | haggle. carefully final deposits detect slyly agai +(1 row) +``` + +```sql +SELECT nationkey, regionkey +FROM nation +WHERE regionkey > 3; +``` + +```text + nationkey | regionkey +-----------+----------- + 4 | 4 + 10 | 4 + 11 | 4 + 13 | 4 + 20 | 4 +(5 rows) +``` + +```sql +SELECT regionkey, count(nationkey) +FROM nation +GROUP BY regionkey; +``` + +```text + regionkey | _col1 +-----------+------- + 2 | 5 + 0 | 5 + 4 | 5 + 3 | 5 + 1 | 5 +(5 rows) +``` + +(paimon-metadata-tables)= +### Metadata tables + +The connector exposes a metadata table for each Paimon table. +The metadata table contains information about the internal structure +of the Paimon table. You can query each metadata table by appending the +metadata table name to the table name: + +``` +SELECT * FROM "nation$files" +``` + +```text +partition | bucket | file_path | file_format | schema_id | level | record_count | file_size_in_bytes | min_> +-----------+--------+--------------------------------------------------------------------------------------------------------------------+-------------+-----------+-------+--------------+--------------------+-----> +[] | 0 | s3://test-paimon-connector-ryo1qhtrl9/tests.db/nation/bucket-0/data-101a3045-cd04-40cd-8664-8355c7d642e9-0.parquet | parquet | 0 | 0 | 25 | 2283 | NULL> +(1 row) +``` + +For all available metadata tables, see the [Paimon documentation](https://paimon.apache.org/docs/master/concepts/system-tables/). diff --git a/docs/src/main/sphinx/sql/drop-catalog.md b/docs/src/main/sphinx/sql/drop-catalog.md index 3ec312172371..d96a5847ac05 100644 --- a/docs/src/main/sphinx/sql/drop-catalog.md +++ b/docs/src/main/sphinx/sql/drop-catalog.md @@ -15,7 +15,7 @@ queries that use it, but makes it unavailable to any new queries. Some connectors are known not to release all resources when dropping a catalog that uses such connector. This includes all connectors that can read data from HDFS, S3, GCS, or Azure, which are [](/connector/hive), -[](/connector/iceberg), [](/connector/delta-lake), and +[](/connector/iceberg), [](/connector/delta-lake), [](/connector/paimon), and [](/connector/hudi). ::: diff --git a/docs/src/main/sphinx/static/img/paimon.png b/docs/src/main/sphinx/static/img/paimon.png new file mode 100644 index 000000000000..e1f73e3f078e Binary files /dev/null and b/docs/src/main/sphinx/static/img/paimon.png differ diff --git a/plugin/trino-paimon/pom.xml b/plugin/trino-paimon/pom.xml new file mode 100644 index 000000000000..0831a4559632 --- /dev/null +++ b/plugin/trino-paimon/pom.xml @@ -0,0 +1,378 @@ + + + 4.0.0 + + + io.trino + trino-root + 481-SNAPSHOT + ../../pom.xml + + + trino-paimon + trino-plugin + ${project.artifactId} + Trino - Paimon connector + + + + com.google.cloud.bigdataoss + gcs-connector + shaded + + + + com.google.code.findbugs + jsr305 + + + + com.google.errorprone + error_prone_annotations + + + + com.google.guava + guava + + + + com.google.inject + guice + classes + + + + io.airlift + bootstrap + + + + io.airlift + configuration + + + + io.airlift + json + + + + io.airlift + units + + + + io.trino + trino-filesystem + + + + io.trino + trino-filesystem-manager + + + + io.trino + trino-hive + + + com.github.luben + zstd-jni + + + + + + io.trino + trino-memory-context + + + + io.trino + trino-metastore + + + + io.trino + trino-orc + + + + io.trino + trino-parquet + + + + io.trino + trino-plugin-toolkit + + + + jakarta.validation + jakarta.validation-api + + + + joda-time + joda-time + + + + org.apache.paimon + paimon-bundle + + + org.lz4 + lz4-java + + + + + + org.apache.parquet + parquet-column + + + + org.slf4j + slf4j-api + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + io.airlift + slice + provided + + + + io.opentelemetry + opentelemetry-api + provided + + + + io.opentelemetry + opentelemetry-api-incubator + provided + + + + io.opentelemetry + opentelemetry-common + provided + + + + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi + provided + + + org.openjdk.jol + jol-core + + + + + + com.google.http-client + google-http-client + runtime + + + + io.airlift + log + runtime + + + + io.trino.hadoop + hadoop-apache + runtime + + + + com.github.docker-java + docker-java-api + test + + + + io.airlift + junit-extensions + test + + + + io.airlift + testing + test + + + + io.trino + trino-client + test + + + + io.trino + trino-hdfs + test + + + + io.trino + trino-hive + test-jar + test + + + + io.trino + trino-main + test + + + com.github.luben + zstd-jni + + + + + + io.trino + trino-main + test-jar + test + + + + io.trino + trino-testing + test + + + + io.trino + trino-testing-containers + test + + + + io.trino + trino-testing-services + test + + + + io.trino + trino-tpch + test + + + + io.trino.tpch + tpch + test + + + + org.assertj + assertj-core + test + + + + org.junit.jupiter + junit-jupiter + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.rnorth.duct-tape + duct-tape + test + + + + org.testcontainers + testcontainers + test + + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + + + + com.amazonaws:*:* + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + org.apache.parquet:parquet-common + + + + + + + + org.basepom.maven + duplicate-finder-maven-plugin + + + log4j.properties + parquet.thrift + org/publicsuffix/list/effective_tld_names.dat + google/protobuf/api.proto + google/protobuf/any.proto + google/protobuf/descriptor.proto + google/protobuf/duration.proto + google/protobuf/empty.proto + google/protobuf/field_mask.proto + google/protobuf/source_context.proto + google/protobuf/struct.proto + google/protobuf/timestamp.proto + google/protobuf/type.proto + google/protobuf/wrappers.proto + + + + + + diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/CatalogType.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/CatalogType.java new file mode 100644 index 000000000000..0df046f0a130 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/CatalogType.java @@ -0,0 +1,20 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +public enum CatalogType +{ + FILESYSTEM, + HIVE +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/DirectPaimonPageSource.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/DirectPaimonPageSource.java new file mode 100644 index 000000000000..f37d4c78a009 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/DirectPaimonPageSource.java @@ -0,0 +1,118 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.SourcePage; +import io.trino.spi.metrics.Metrics; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class DirectPaimonPageSource + implements ConnectorPageSource +{ + private final Deque pageSourceQueue; + private ConnectorPageSource current; + private long completedBytes; + private long readTimeNanos; + + public DirectPaimonPageSource(List pageSources) + { + this.pageSourceQueue = new ArrayDeque<>(requireNonNull(pageSources, "pageSources is null")); + this.current = pageSourceQueue.poll(); + } + + @Override + public long getCompletedBytes() + { + return completedBytes + (current == null ? 0 : current.getCompletedBytes()); + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos + (current == null ? 0 : current.getReadTimeNanos()); + } + + @Override + public boolean isFinished() + { + return current == null || (current.isFinished() && pageSourceQueue.isEmpty()); + } + + @Override + public SourcePage getNextSourcePage() + { + while (current != null) { + SourcePage dataPage = current.getNextSourcePage(); + if (dataPage != null) { + return dataPage; + } + advance(); + } + return null; + } + + private void advance() + { + if (current == null) { + throw new IllegalStateException("Current page source is null"); + } + try { + completedBytes += current.getCompletedBytes(); + readTimeNanos += current.getReadTimeNanos(); + current.close(); + } + catch (IOException e) { + current = null; + close(); + throw new UncheckedIOException("Error closing page source", e); + } + current = pageSourceQueue.poll(); + } + + @Override + public void close() + { + try { + if (current != null) { + current.close(); + } + for (ConnectorPageSource source : pageSourceQueue) { + source.close(); + } + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public long getMemoryUsage() + { + return current == null ? 0 : current.getMemoryUsage(); + } + + @Override + public Metrics getMetrics() + { + return current == null ? Metrics.EMPTY : current.getMetrics(); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/EncodingUtils.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/EncodingUtils.java new file mode 100644 index 000000000000..d17d281174ac --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/EncodingUtils.java @@ -0,0 +1,50 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import java.util.Base64; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.paimon.utils.InstantiationUtil.deserializeObject; +import static org.apache.paimon.utils.InstantiationUtil.serializeObject; + +public final class EncodingUtils +{ + private static final Base64.Encoder BASE64_ENCODER = Base64.getUrlEncoder().withoutPadding(); + private static final Base64.Decoder BASE64_DECODER = Base64.getUrlDecoder(); + + private EncodingUtils() {} + + public static String encodeObjectToString(Object object) + { + try { + byte[] bytes = serializeObject(object); + return new String(BASE64_ENCODER.encode(bytes), UTF_8); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static Object decodeStringToObject(String encodedStr) + { + final byte[] bytes = BASE64_DECODER.decode(encodedStr.getBytes(UTF_8)); + try { + return deserializeObject(bytes, EncodingUtils.class.getClassLoader()); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonColumnHandle.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonColumnHandle.java new file mode 100644 index 000000000000..8d6ec8d5ab57 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonColumnHandle.java @@ -0,0 +1,51 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.type.Type; +import org.apache.paimon.types.DataType; +import org.apache.paimon.utils.JsonSerdeUtil; + +import static io.trino.plugin.paimon.PaimonTypeUtils.toTrinoType; +import static java.util.Objects.requireNonNull; + +public record PaimonColumnHandle(String columnName, String typeString, Type trinoType, int columnId) + implements ColumnHandle +{ + public static final String TRINO_ROW_ID_NAME = "$row_id"; + + public PaimonColumnHandle + { + requireNonNull(columnName, "columnName is null"); + requireNonNull(typeString, "typeString is null"); + requireNonNull(trinoType, "trinoType is null"); + } + + public static PaimonColumnHandle of(String columnName, DataType columnType, int columnId) + { + return new PaimonColumnHandle( + columnName, + JsonSerdeUtil.toJson(columnType), + toTrinoType(columnType), + columnId); + } + + @JsonProperty + public boolean isRowId() + { + return TRINO_ROW_ID_NAME.equals(columnName); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConfig.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConfig.java new file mode 100644 index 000000000000..c2a41e58f9ac --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConfig.java @@ -0,0 +1,87 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import jakarta.validation.constraints.NotNull; +import org.apache.paimon.options.Options; + +public class PaimonConfig +{ + private String warehouse; + private CatalogType catalogType = CatalogType.FILESYSTEM; + private boolean metadataCacheEnabled = true; + private boolean projectionPushdownEnabled = true; + + @NotNull + public CatalogType getCatalogType() + { + return catalogType; + } + + @Config("paimon.catalog.type") + @ConfigDescription("Catalog type with paimon, default is FILESYSTEM") + public PaimonConfig setCatalogType(CatalogType catalogType) + { + this.catalogType = catalogType; + return this; + } + + public String getWarehouse() + { + return warehouse; + } + + @Config("paimon.warehouse") + @ConfigDescription("Warehouse path for paimon, this is the root path for all the tables") + public PaimonConfig setWarehouse(String warehouse) + { + this.warehouse = warehouse; + return this; + } + + public boolean isMetadataCacheEnabled() + { + return metadataCacheEnabled; + } + + @Config("paimon.metadata-cache.enabled") + @ConfigDescription("Enables in-memory caching of metadata files on coordinator if fs.cache.enabled is not set to true") + public PaimonConfig setMetadataCacheEnabled(boolean metadataCacheEnabled) + { + this.metadataCacheEnabled = metadataCacheEnabled; + return this; + } + + public boolean isProjectionPushdownEnabled() + { + return projectionPushdownEnabled; + } + + @Config("paimon.projection-pushdown-enabled") + @ConfigDescription("Read only required fields from a row type") + public PaimonConfig setProjectionPushdownEnabled(boolean projectionPushdownEnabled) + { + this.projectionPushdownEnabled = projectionPushdownEnabled; + return this; + } + + public Options toOptions() + { + Options options = new Options(); + options.set("warehouse", warehouse); + return options; + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConnector.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConnector.java new file mode 100644 index 000000000000..80a8a5b3a83e --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConnector.java @@ -0,0 +1,141 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import com.google.inject.Injector; +import io.airlift.bootstrap.LifeCycleManager; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata; +import io.trino.plugin.hive.HiveTransactionHandle; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorCapabilities; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.session.PropertyMetadata; +import io.trino.spi.transaction.IsolationLevel; + +import java.util.List; +import java.util.Set; + +import static com.google.common.collect.Sets.immutableEnumSet; +import static io.trino.spi.connector.ConnectorCapabilities.NOT_NULL_COLUMN_CONSTRAINT; +import static io.trino.spi.transaction.IsolationLevel.SERIALIZABLE; +import static io.trino.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +public class PaimonConnector + implements Connector +{ + private final Injector injector; + private final LifeCycleManager lifeCycleManager; + private final PaimonTransactionManager transactionManager; + private final ConnectorSplitManager splitManager; + private final ConnectorPageSourceProvider pageSourceProvider; + private final List> tableProperties; + private final List> sessionProperties; + + @Inject + public PaimonConnector( + Injector injector, + LifeCycleManager lifeCycleManager, + PaimonTransactionManager transactionManager, + ConnectorSplitManager splitManager, + ConnectorPageSourceProvider pageSourceProvider, + PaimonTableOptions paimonTableOptions, + PaimonSessionProperties paimonSessionProperties) + { + this.injector = requireNonNull(injector, "injector is null"); + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); + this.tableProperties = paimonTableOptions.getTableProperties(); + this.sessionProperties = paimonSessionProperties.getSessionProperties(); + } + + @Override + public ConnectorTransactionHandle beginTransaction( + IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) + { + checkConnectorSupports(SERIALIZABLE, isolationLevel); + ConnectorTransactionHandle transaction = new HiveTransactionHandle(autoCommit); + transactionManager.begin(transaction); + return transaction; + } + + @Override + public void commit(ConnectorTransactionHandle transaction) + { + transactionManager.commit(transaction); + } + + @Override + public void rollback(ConnectorTransactionHandle transactionHandle) + { + transactionManager.rollback(transactionHandle); + } + + @Override + public ConnectorMetadata getMetadata(ConnectorSession session, ConnectorTransactionHandle transactionHandle) + { + ConnectorMetadata metadata = transactionManager.get(transactionHandle, session.getIdentity()); + return new ClassLoaderSafeConnectorMetadata(metadata, getClass().getClassLoader()); + } + + @Override + public Set getCapabilities() + { + return immutableEnumSet(NOT_NULL_COLUMN_CONSTRAINT); + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return pageSourceProvider; + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } + + @Override + public List> getTableProperties() + { + return tableProperties; + } + + @Override + public void shutdown() + { + lifeCycleManager.stop(); + } + + @VisibleForTesting + public Injector getInjector() + { + return injector; + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConnectorFactory.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConnectorFactory.java new file mode 100644 index 000000000000..1b78b5f15e02 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonConnectorFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.airlift.json.JsonModule; +import io.trino.filesystem.manager.FileSystemModule; +import io.trino.plugin.base.ConnectorContextModule; +import io.trino.plugin.paimon.catalog.PaimonCatalogModule; +import io.trino.spi.classloader.ThreadContextClassLoader; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorFactory; + +import java.util.Map; +import java.util.Optional; + +import static com.google.inject.util.Modules.EMPTY_MODULE; +import static java.util.Objects.requireNonNull; + +public class PaimonConnectorFactory + implements ConnectorFactory +{ + @Override + public String getName() + { + return "paimon"; + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + return createConnector(catalogName, config, context, Optional.empty()); + } + + public static Connector createConnector( + String catalogName, + Map config, + ConnectorContext context, + Optional module) + { + ClassLoader classLoader = PaimonConnectorFactory.class.getClassLoader(); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + Bootstrap app = new Bootstrap( + new JsonModule(), + new PaimonCatalogModule(), + new PaimonModule(), + // bind the trino file system module + new PaimonFileSystemModule(catalogName, context), + new ConnectorContextModule(catalogName, context), + binder -> { + binder.bind(ClassLoader.class).toInstance(PaimonConnectorFactory.class.getClassLoader()); + }, + module.orElse(EMPTY_MODULE)); + + Injector injector = app.doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(PaimonConnector.class); + } + } + + private static class PaimonFileSystemModule + extends AbstractConfigurationAwareModule + { + private final String catalogName; + private final ConnectorContext context; + + private PaimonFileSystemModule(String catalogName, ConnectorContext context) + { + this.catalogName = requireNonNull(catalogName, "catalogName is null"); + this.context = requireNonNull(context, "context is null"); + } + + @Override + protected void setup(Binder binder) + { + boolean metadataCacheEnabled = buildConfigObject(PaimonConfig.class).isMetadataCacheEnabled(); + install(new FileSystemModule(catalogName, context, metadataCacheEnabled)); + } + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonErrorCode.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonErrorCode.java new file mode 100644 index 000000000000..c50202af13f6 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonErrorCode.java @@ -0,0 +1,40 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.spi.ErrorCode; +import io.trino.spi.ErrorCodeSupplier; +import io.trino.spi.ErrorType; + +import static io.trino.spi.ErrorType.EXTERNAL; + +public enum PaimonErrorCode + implements ErrorCodeSupplier +{ + PAIMON_CANNOT_OPEN_SPLIT(0, EXTERNAL), + PAIMON_METADATA_FETCH_FAILED(1, EXTERNAL); + + private final ErrorCode errorCode; + + PaimonErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0515_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonFilterConverter.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonFilterConverter.java new file mode 100644 index 000000000000..6083c18138c6 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonFilterConverter.java @@ -0,0 +1,326 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.airlift.slice.Slice; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.BooleanType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DateType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.DoubleType; +import io.trino.spi.type.Int128; +import io.trino.spi.type.IntegerType; +import io.trino.spi.type.LongTimestampWithTimeZone; +import io.trino.spi.type.MapType; +import io.trino.spi.type.RealType; +import io.trino.spi.type.SmallintType; +import io.trino.spi.type.TinyintType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarbinaryType; +import io.trino.spi.type.VarcharType; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.fileindex.FileIndexOptions; +import org.apache.paimon.predicate.CompoundPredicate; +import org.apache.paimon.predicate.In; +import org.apache.paimon.predicate.LeafPredicate; +import org.apache.paimon.predicate.Or; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.types.RowType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static io.trino.plugin.paimon.PaimonTypeUtils.fieldNames; +import static io.trino.spi.type.TimeType.TIME_MILLIS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_MILLISECOND; +import static java.lang.Float.intBitsToFloat; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; +import static org.apache.paimon.data.Decimal.fromBigDecimal; +import static org.apache.paimon.predicate.PredicateBuilder.and; + +public class PaimonFilterConverter +{ + private static final Logger LOG = LoggerFactory.getLogger(PaimonFilterConverter.class); + + private final RowType rowType; + private final PredicateBuilder builder; + + public PaimonFilterConverter(RowType rowType) + { + this.rowType = requireNonNull(rowType, "rowType is null"); + this.builder = new PredicateBuilder(rowType); + } + + public Optional convert(TupleDomain tupleDomain) + { + return convert(tupleDomain, new LinkedHashMap<>(), new LinkedHashMap<>()); + } + + public Optional convert( + TupleDomain tupleDomain, + Map acceptedDomains, + Map unsupportedDomains) + { + if (tupleDomain.isAll()) { + // TODO alwaysTrue + return Optional.empty(); + } + + if (tupleDomain.getDomains().isEmpty()) { + // TODO alwaysFalse + return Optional.empty(); + } + + Map domainMap = tupleDomain.getDomains().get(); + List conjuncts = new ArrayList<>(); + List fieldNames = fieldNames(rowType); + for (Map.Entry entry : domainMap.entrySet()) { + PaimonColumnHandle columnHandle = entry.getKey(); + Domain domain = entry.getValue(); + String field = columnHandle.columnName(); + Optional nestedColumn = FileIndexOptions.topLevelIndexOfNested(field); + if (nestedColumn.isPresent()) { + int position = nestedColumn.get(); + field = field.substring(0, position); + } + int index = fieldNames.indexOf(field); + if (index != -1) { + try { + toPredicate( + index, + columnHandle.columnName(), + columnHandle.trinoType(), + domain).ifPresent(conjuncts::add); + acceptedDomains.put(columnHandle, domain); + continue; + } + catch (UnsupportedOperationException exception) { + LOG.warn("Unsupported predicate, maybe the type of column is not supported yet", exception); + } + } + unsupportedDomains.put(columnHandle, domain); + } + + if (conjuncts.isEmpty()) { + return Optional.empty(); + } + return Optional.of(and(conjuncts)); + } + + private Optional toPredicate(int columnIndex, String field, Type type, Domain domain) + { + if (domain.isAll()) { + // TODO alwaysTrue + throw new UnsupportedOperationException(); + } + if (domain.getValues().isNone()) { + if (domain.isNullAllowed()) { + return Optional.of(builder.isNull(columnIndex)); + } + // TODO alwaysFalse + throw new UnsupportedOperationException(); + } + + if (domain.getValues().isAll()) { + if (domain.isNullAllowed()) { + // TODO alwaysTrue + throw new UnsupportedOperationException(); + } + return Optional.of(builder.isNotNull(columnIndex)); + } + + // TODO support structural types + switch (type) { + case ArrayType _ -> throw new UnsupportedOperationException(); + case io.trino.spi.type.RowType _ -> throw new UnsupportedOperationException(); + case MapType maptype -> { + List orderedRanges = domain.getValues().getRanges().getOrderedRanges(); + List values = new ArrayList<>(); + Predicate predicate = null; + for (Range range : orderedRanges) { + if (range.isSingleValue()) { + values.add( + getLiteralValue(maptype.getValueType(), range.getLowBoundedValue())); + } + } + if (!values.isEmpty()) { + predicate = + new LeafPredicate( + In.INSTANCE, + PaimonTypeUtils.toPaimonType(type), + columnIndex, + field, + values); + } + return Optional.ofNullable(predicate); + } + default -> {} + } + + if (type.isOrderable()) { + List orderedRanges = domain.getValues().getRanges().getOrderedRanges(); + List values = new ArrayList<>(); + List predicates = new ArrayList<>(); + for (Range range : orderedRanges) { + if (range.isSingleValue()) { + values.add(getLiteralValue(type, range.getLowBoundedValue())); + } + else { + predicates.add(toPredicate(columnIndex, range)); + } + } + + if (!values.isEmpty()) { + predicates.add(builder.in(columnIndex, values)); + } + + if (domain.isNullAllowed()) { + predicates.add(builder.isNull(columnIndex)); + } + Predicate predicate = predicates.size() > 1 + ? new CompoundPredicate(Or.INSTANCE, predicates) + : predicates.getFirst(); + return Optional.of(predicate); + } + + throw new UnsupportedOperationException(); + } + + private Predicate toPredicate(int columnIndex, Range range) + { + Type type = range.getType(); + + if (range.isSingleValue()) { + Object value = getLiteralValue(type, range.getSingleValue()); + return builder.equal(columnIndex, value); + } + + List conjuncts = new ArrayList<>(2); + if (!range.isLowUnbounded()) { + Object low = getLiteralValue(type, range.getLowBoundedValue()); + Predicate lowBound; + if (range.isLowInclusive()) { + lowBound = builder.greaterOrEqual(columnIndex, low); + } + else { + lowBound = builder.greaterThan(columnIndex, low); + } + conjuncts.add(lowBound); + } + + if (!range.isHighUnbounded()) { + Object high = getLiteralValue(type, range.getHighBoundedValue()); + Predicate highBound; + if (range.isHighInclusive()) { + highBound = builder.lessOrEqual(columnIndex, high); + } + else { + highBound = builder.lessThan(columnIndex, high); + } + conjuncts.add(highBound); + } + + return and(conjuncts); + } + + private static Object getLiteralValue(Type type, Object trinoNativeValue) + { + requireNonNull(trinoNativeValue, "trinoNativeValue is null"); + + if (type instanceof BooleanType) { + return trinoNativeValue; + } + + if (type instanceof TinyintType) { + return ((Long) trinoNativeValue).byteValue(); + } + + if (type instanceof SmallintType) { + return ((Long) trinoNativeValue).shortValue(); + } + + if (type instanceof IntegerType) { + return toIntExact((long) trinoNativeValue); + } + + if (type instanceof BigintType) { + return trinoNativeValue; + } + + if (type instanceof RealType) { + return intBitsToFloat(toIntExact((long) trinoNativeValue)); + } + + if (type instanceof DoubleType) { + return trinoNativeValue; + } + + if (type instanceof DateType) { + return toIntExact(((Long) trinoNativeValue)); + } + + if (type.equals(TIME_MILLIS)) { + return (int) ((long) trinoNativeValue / PICOSECONDS_PER_MILLISECOND); + } + + if (type.equals(TIMESTAMP_MILLIS)) { + return Timestamp.fromEpochMillis((long) trinoNativeValue / 1000); + } + + if (type.equals(TIMESTAMP_TZ_MILLIS)) { + if (trinoNativeValue instanceof Long) { + return trinoNativeValue; + } + return Timestamp.fromEpochMillis(((LongTimestampWithTimeZone) trinoNativeValue).getEpochMillis()); + } + + if (type instanceof VarcharType || type instanceof CharType) { + return BinaryString.fromBytes(((Slice) trinoNativeValue).getBytes()); + } + + if (type instanceof VarbinaryType) { + return ((Slice) trinoNativeValue).getBytes(); + } + + if (type instanceof DecimalType decimalType) { + BigDecimal bigDecimal; + if (trinoNativeValue instanceof Long) { + bigDecimal = BigDecimal.valueOf((long) trinoNativeValue).movePointLeft(decimalType.getScale()); + } + else { + bigDecimal = new BigDecimal(((Int128) trinoNativeValue).toBigInteger(), decimalType.getScale()); + } + return fromBigDecimal( + bigDecimal, decimalType.getPrecision(), decimalType.getScale()); + } + + throw new UnsupportedOperationException("Unsupported type: " + type); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMergePageSourceWrapper.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMergePageSourceWrapper.java new file mode 100644 index 000000000000..a703ada13932 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMergePageSourceWrapper.java @@ -0,0 +1,94 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.Page; +import io.trino.spi.block.Block; +import io.trino.spi.block.RowBlock; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.SourcePage; + +import java.io.IOException; +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public class PaimonMergePageSourceWrapper + implements ConnectorPageSource +{ + private final ConnectorPageSource pageSource; + private final Map fieldToIndex; + + public PaimonMergePageSourceWrapper(ConnectorPageSource pageSource, Map fieldToIndex) + { + this.pageSource = requireNonNull(pageSource, "pageSource is null"); + this.fieldToIndex = ImmutableMap.copyOf(fieldToIndex); + } + + @Override + public long getCompletedBytes() + { + return pageSource.getCompletedBytes(); + } + + @Override + public long getReadTimeNanos() + { + return pageSource.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return pageSource.isFinished(); + } + + @Override + public SourcePage getNextSourcePage() + { + SourcePage nextPage = pageSource.getNextSourcePage(); + if (nextPage == null) { + return null; + } + int rowCount = nextPage.getPositionCount(); + + Block[] newBlocks = new Block[nextPage.getChannelCount() + 1]; + Block[] rowIdBlocks = new Block[fieldToIndex.size()]; + for (int i = 0, idx = 0; i < nextPage.getChannelCount(); i++) { + Block block = nextPage.getBlock(i); + newBlocks[i] = block; + if (fieldToIndex.containsValue(i)) { + rowIdBlocks[idx] = block; + idx++; + } + } + newBlocks[nextPage.getChannelCount()] = RowBlock.fromFieldBlocks(rowCount, rowIdBlocks); + + return SourcePage.create(new Page(rowCount, newBlocks)); + } + + @Override + public long getMemoryUsage() + { + return pageSource.getMemoryUsage(); + } + + @Override + public void close() + throws IOException + { + pageSource.close(); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMetadata.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMetadata.java new file mode 100644 index 000000000000..ce1e9ff8b620 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMetadata.java @@ -0,0 +1,211 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.plugin.paimon.catalog.TrinoCatalog; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.ConnectorTableVersion; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SchemaTablePrefix; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; +import org.apache.paimon.table.Table; +import org.apache.paimon.types.DataField; +import org.apache.paimon.utils.Pair; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static io.trino.plugin.paimon.PaimonErrorCode.PAIMON_METADATA_FETCH_FAILED; +import static io.trino.plugin.paimon.PaimonTypeUtils.fieldNames; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toList; +import static java.util.stream.Collectors.toMap; + +public record PaimonMetadata(TrinoCatalog catalog) + implements ConnectorMetadata +{ + public PaimonMetadata(TrinoCatalog catalog) + { + this.catalog = requireNonNull(catalog, "catalog is null"); + } + + @Override + public boolean schemaExists(ConnectorSession session, String schemaName) + { + return catalog.databaseExists(session, schemaName); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return catalog.listDatabases(session); + } + + @Override + public ConnectorTableHandle getTableHandle( + ConnectorSession session, + SchemaTableName tableName, + Optional startVersion, + Optional endVersion) + { + if (startVersion.isPresent() || endVersion.isPresent()) { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support versioned tables"); + } + + try { + catalog.loadTable(session, tableName); + return new PaimonTableHandle(tableName.getSchemaName(), tableName.getTableName(), TupleDomain.all()); + } + catch (TableNotFoundException e) { + return null; + } + } + + @Override + public ConnectorTableMetadata getTableMetadata( + ConnectorSession session, ConnectorTableHandle tableHandle) + { + PaimonTableHandle paimonTableHandle = (PaimonTableHandle) tableHandle; + try { + Table table = table(session, paimonTableHandle.getSchemaName(), paimonTableHandle.getTableName()); + return tableMetadata(table, paimonTableHandle.getSchemaName(), paimonTableHandle.getTableName()); + } + catch (Exception e) { + throw new TrinoException(PAIMON_METADATA_FETCH_FAILED, "Failed to get table metadata!", e); + } + } + + public ConnectorTableMetadata tableMetadata(Table table, String schemaName, String tableName) + { + return new ConnectorTableMetadata( + SchemaTableName.schemaTableName(schemaName, tableName), + columnMetadatas(table), + Collections.emptyMap(), + table.comment()); + } + + public Table table(ConnectorSession session, String schemaName, String tableName) + { + return catalog.loadTable(session, SchemaTableName.schemaTableName(schemaName, tableName)); + } + + public List columnMetadatas(Table table) + { + return table.rowType().getFields().stream() + .map(column -> ColumnMetadata.builder() + .setName(column.name()) + .setType(PaimonTypeUtils.toTrinoType(column.type())) + .setNullable(column.type().isNullable()) + .setComment(Optional.ofNullable(column.description())) + .build()) + .collect(toList()); + } + + public PaimonColumnHandle columnHandle(Table paimonTable, String field) + { + List lowerCaseFieldNames = fieldNames(paimonTable.rowType()); + List originFieldNames = paimonTable.rowType().getFieldNames(); + int index = lowerCaseFieldNames.indexOf(field); + if (index == -1) { + throw new RuntimeException(format("Cannot find field %s in schema %s", field, lowerCaseFieldNames)); + } + DataField dataField = paimonTable.rowType().getFields().get(index); + return PaimonColumnHandle.of(originFieldNames.get(index), dataField.type(), dataField.id()); + } + + @Override + public List listTables(ConnectorSession session, Optional schemaName) + { + List tables = new ArrayList<>(); + schemaName.map(Collections::singletonList) + .orElseGet(() -> catalog.listDatabases(session)) + .forEach(schema -> tables.addAll(listTables(session, schema))); + return tables; + } + + private List listTables(ConnectorSession session, String schema) + { + return catalog.listTables(session, Optional.of(schema)).stream() + .map(table -> new SchemaTableName(schema, table)) + .collect(toList()); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + PaimonTableHandle handle = (PaimonTableHandle) tableHandle; + Table table = table(session, handle.getSchemaName(), handle.getTableName()); + Map columnHandles = new HashMap<>(); + for (ColumnMetadata column : columnMetadatas(table)) { + columnHandles.put(column.getName(), columnHandle(table, column.getName())); + } + return columnHandles; + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + PaimonColumnHandle column = (PaimonColumnHandle) columnHandle; + return new ColumnMetadata(column.columnName(), column.trinoType()); + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + requireNonNull(prefix, "prefix is null"); + + List tableNames; + if (prefix.getTable().isPresent()) { + tableNames = Collections.singletonList(prefix.toSchemaTableName()); + } + else { + tableNames = listTables(session, prefix.getSchema()); + } + + return tableNames.stream() + .map(table -> { + List columnMetadata; + try { + getTableHandle(session, table, Optional.empty(), Optional.empty()); + Table paimonTable = table(session, table.getSchemaName(), table.getTableName()); + columnMetadata = columnMetadatas(paimonTable); + } + catch (RuntimeException e) { + // Error when getting column metadata, return null + return Pair.of(table, (List) null); + } + + return Pair.of(table, columnMetadata); + }).filter(p -> p.getRight() != null) + .collect(toMap(Pair::getLeft, Pair::getRight)); + } + + public void rollback() + { + // do nothing + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMetadataFactory.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMetadataFactory.java new file mode 100644 index 000000000000..796a70b74012 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonMetadataFactory.java @@ -0,0 +1,36 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.inject.Inject; +import io.trino.plugin.paimon.catalog.TrinoCatalogFactory; +import io.trino.spi.security.ConnectorIdentity; + +import static java.util.Objects.requireNonNull; + +public class PaimonMetadataFactory +{ + private final TrinoCatalogFactory catalogFactory; + + @Inject + public PaimonMetadataFactory(TrinoCatalogFactory catalogFactory) + { + this.catalogFactory = requireNonNull(catalogFactory, "catalogFactory is null"); + } + + public PaimonMetadata create(ConnectorIdentity identity) + { + return new PaimonMetadata(catalogFactory.create(identity)); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonModule.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonModule.java new file mode 100644 index 000000000000..c5f279649baf --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonModule.java @@ -0,0 +1,74 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.trino.plugin.base.classloader.ForClassLoaderSafe; +import io.trino.plugin.base.metrics.FileFormatDataSourceStats; +import io.trino.plugin.hive.HideDeltaLakeTables; +import io.trino.plugin.hive.orc.OrcReaderConfig; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSplitManager; + +import static com.google.inject.Scopes.SINGLETON; +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class PaimonModule + implements Module +{ + @Override + public void configure(Binder binder) + { + configBinder(binder).bindConfig(PaimonConfig.class); + configBinder(binder).bindConfig(OrcReaderConfig.class); + configBinder(binder).bindConfig(ParquetReaderConfig.class); + + binder.bind(PaimonMetadataFactory.class).in(SINGLETON); + binder.bind(PaimonSplitManager.class).in(SINGLETON); + binder.bind(PaimonPageSourceProvider.class).in(SINGLETON); + binder.bind(PaimonSessionProperties.class).in(SINGLETON); + binder.bind(PaimonTableOptions.class).in(SINGLETON); + binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); + binder.bind(PaimonTransactionManager.class).in(SINGLETON); +// binder.bind(PaimonTrinoCatalogFactory.class).in(SINGLETON); + binder.bind(ConnectorSplitManager.class) + .annotatedWith(ForClassLoaderSafe.class) + .to(PaimonSplitManager.class) + .in(Scopes.SINGLETON); + binder.bind(ConnectorSplitManager.class) + .to(ClassLoaderSafeConnectorSplitManager.class) + .in(Scopes.SINGLETON); + binder.bind(ConnectorPageSourceProvider.class) + .annotatedWith(ForClassLoaderSafe.class) + .to(PaimonPageSourceProvider.class) + .in(Scopes.SINGLETON); + binder.bind(ConnectorPageSourceProvider.class) + .to(ClassLoaderSafeConnectorPageSourceProvider.class) + .in(Scopes.SINGLETON); + + binder.bind(PaimonConnector.class).in(Scopes.SINGLETON); + + binder.bind(boolean.class).annotatedWith(HideDeltaLakeTables.class).toInstance(false); + newOptionalBinder(binder, Key.get(HiveMetastoreFactory.class, RawHiveMetastoreFactory.class)); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonOrcDataSource.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonOrcDataSource.java new file mode 100644 index 000000000000..0c0b0e3ca410 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonOrcDataSource.java @@ -0,0 +1,67 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.airlift.slice.Slice; +import io.trino.filesystem.TrinoInput; +import io.trino.filesystem.TrinoInputFile; +import io.trino.orc.AbstractOrcDataSource; +import io.trino.orc.OrcDataSourceId; +import io.trino.orc.OrcReaderOptions; +import io.trino.plugin.base.metrics.FileFormatDataSourceStats; + +import java.io.IOException; + +import static java.util.Objects.requireNonNull; + +public class PaimonOrcDataSource + extends AbstractOrcDataSource +{ + private final FileFormatDataSourceStats stats; + private final TrinoInput input; + + public PaimonOrcDataSource(TrinoInputFile file, OrcReaderOptions options, FileFormatDataSourceStats stats) + throws IOException + { + super(new OrcDataSourceId(file.location().toString()), file.length(), options); + this.stats = requireNonNull(stats, "stats is null"); + this.input = file.newInput(); + } + + @Override + public void close() + throws IOException + { + input.close(); + } + + @Override + protected Slice readTailInternal(int length) + throws IOException + { + long readStart = System.nanoTime(); + Slice tail = input.readTail(length); + stats.readDataBytesPerSecond(tail.length(), System.nanoTime() - readStart); + return input.readTail(length); + } + + @Override + protected void readInternal(long position, byte[] buffer, int bufferOffset, int bufferLength) + throws IOException + { + long readStart = System.nanoTime(); + input.readFully(position, buffer, bufferOffset, bufferLength); + stats.readDataBytesPerSecond(bufferLength, System.nanoTime() - readStart); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSource.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSource.java new file mode 100644 index 000000000000..909d631f7d75 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSource.java @@ -0,0 +1,385 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.airlift.slice.Slice; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.TrinoException; +import io.trino.spi.block.ArrayBlockBuilder; +import io.trino.spi.block.ArrayValueBuilder; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.MapBlockBuilder; +import io.trino.spi.block.MapValueBuilder; +import io.trino.spi.block.RowBlockBuilder; +import io.trino.spi.block.RowValueBuilder; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.SourcePage; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Decimals; +import io.trino.spi.type.LongTimestampWithTimeZone; +import io.trino.spi.type.MapType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarbinaryType; +import io.trino.spi.type.VarcharType; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeChecks; +import org.apache.paimon.utils.CloseableIterator; +import org.apache.paimon.utils.InternalRowUtils; +import org.apache.paimon.utils.JsonSerdeUtil; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.wrappedBuffer; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.Decimals.encodeShortScaledValue; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.LongTimestampWithTimeZone.fromEpochMillisAndFraction; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TimeType.TIME_MICROS; +import static io.trino.spi.type.TimeZoneKey.UTC_KEY; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MICROS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_SECONDS; +import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS; +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.TinyintType.TINYINT; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class PaimonPageSource + implements ConnectorPageSource +{ + private static final int ROWS_PER_REQUEST = 4096; + + private final CloseableIterator iterator; + private final PageBuilder pageBuilder; + private final List columnTypes; + private final List logicalTypes; + private final AggregatedMemoryContext memoryUsage; + + private boolean isFinished; + private long readBytes; + private long readTimeNanos; + + public PaimonPageSource( + RecordReader reader, + List projectedColumns, + AggregatedMemoryContext memoryUsage) + { + this.iterator = reader.toCloseableIterator(); + this.columnTypes = new ArrayList<>(); + this.logicalTypes = new ArrayList<>(); + for (ColumnHandle handle : projectedColumns) { + PaimonColumnHandle columnHandle = (PaimonColumnHandle) handle; + columnTypes.add(columnHandle.trinoType()); + logicalTypes.add(JsonSerdeUtil.fromJson(columnHandle.typeString(), DataType.class)); + } + + this.memoryUsage = requireNonNull(memoryUsage, "memoryUsage is null"); + this.pageBuilder = new PageBuilder(columnTypes); + } + + private static void writeSlice(BlockBuilder output, Type type, Object value) + { + if (type instanceof VarcharType || type instanceof CharType) { + type.writeSlice(output, wrappedBuffer(((BinaryString) value).toBytes())); + } + else if (type instanceof VarbinaryType) { + type.writeSlice(output, wrappedBuffer((byte[]) value)); + } + else { + throw new TrinoException( + GENERIC_INTERNAL_ERROR, "Unhandled type for Slice: " + type.getTypeSignature()); + } + } + + private static void writeObject(BlockBuilder output, Type type, Object value) + { + if (type instanceof DecimalType decimalType) { + BigDecimal decimal = ((Decimal) value).toBigDecimal(); + type.writeObject(output, Decimals.encodeScaledValue(decimal, decimalType.getScale())); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, "Unhandled type for Object: " + type.getTypeSignature()); + } + } + + @Override + public long getCompletedBytes() + { + return readBytes; + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos; + } + + @Override + public boolean isFinished() + { + return isFinished; + } + + @Override + public SourcePage getNextSourcePage() + { + try { + return nextPage(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public long getMemoryUsage() + { + return memoryUsage.getBytes(); + } + + @Nullable + private SourcePage nextPage() + throws IOException + { + long start = System.nanoTime(); + int count = 0; + while (count < ROWS_PER_REQUEST && !pageBuilder.isFull()) { + if (!iterator.hasNext()) { + isFinished = true; + break; + } + + InternalRow row = iterator.next(); + pageBuilder.declarePosition(); + count++; + for (int i = 0; i < columnTypes.size(); i++) { + BlockBuilder output = pageBuilder.getBlockBuilder(i); + appendTo( + columnTypes.get(i), + logicalTypes.get(i), + InternalRowUtils.get(row, i, logicalTypes.get(i)), + output); + } + } + + if (count == 0) { + return null; + } + Page page = pageBuilder.build(); + readBytes += page.getSizeInBytes(); + readTimeNanos += System.nanoTime() - start; + pageBuilder.reset(); + return SourcePage.create(page); + } + + @Override + public void close() + throws IOException + { + try { + this.iterator.close(); + } + catch (Exception e) { + throw new IOException(e); + } + } + + protected void appendTo(Type type, DataType logicalType, Object value, BlockBuilder output) + { + if (value == null) { + output.appendNull(); + return; + } + + Class javaType = type.getJavaType(); + if (javaType == boolean.class) { + type.writeBoolean(output, (Boolean) value); + } + else if (javaType == long.class) { + if (type.equals(BIGINT) + || type.equals(INTEGER) + || type.equals(TINYINT) + || type.equals(SMALLINT) + || type.equals(DATE)) { + type.writeLong(output, ((Number) value).longValue()); + } + else if (type.equals(REAL)) { + type.writeLong(output, Float.floatToIntBits((Float) value)); + } + else if (type instanceof DecimalType decimalType) { + BigDecimal decimal = ((Decimal) value).toBigDecimal(); + type.writeLong(output, encodeShortScaledValue(decimal, decimalType.getScale())); + } + else if (type.equals(TIMESTAMP_MILLIS) || type.equals(TIMESTAMP_SECONDS)) { + type.writeLong( + output, + ((Timestamp) value).getMillisecond() * MICROSECONDS_PER_MILLISECOND); + } + else if (type.equals(TIMESTAMP_MICROS)) { + type.writeLong(output, ((Timestamp) value).toMicros()); + } + else if (type.equals(TIMESTAMP_TZ_MILLIS)) { + type.writeLong( + output, + packDateTimeWithZone(((Timestamp) value).getMillisecond(), UTC_KEY)); + } + else if (type.equals(TIME_MICROS)) { + type.writeLong(output, ((int) value) * ((long) MICROSECONDS_PER_MILLISECOND)); + } + else { + throw new TrinoException( + GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s", javaType.getSimpleName(), type)); + } + } + else if (javaType == double.class) { + type.writeDouble(output, ((Number) value).doubleValue()); + } + else if (type instanceof DecimalType) { + writeObject(output, type, value); + } + else if (javaType == Slice.class) { + writeSlice(output, type, value); + } + else if (javaType == LongTimestampWithTimeZone.class) { + checkArgument(type.equals(TIMESTAMP_TZ_MILLIS)); + Timestamp timestamp = (Timestamp) value; + type.writeObject( + output, fromEpochMillisAndFraction(timestamp.getMillisecond(), 0, UTC_KEY)); + } + else if (type instanceof ArrayType + || type instanceof MapType + || type instanceof RowType) { + writeBlock(output, type, logicalType, value); + } + else { + throw new TrinoException( + GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s", javaType.getSimpleName(), type)); + } + } + + protected void writeBlock(BlockBuilder output, Type type, DataType logicalType, Object value) + { + if (type instanceof ArrayType) { + ArrayBlockBuilder arrayBlockBuilder = (ArrayBlockBuilder) output; + try { + arrayBlockBuilder.buildEntry( + (ArrayValueBuilder) + elementBuilder -> { + InternalArray arrayData = (InternalArray) value; + DataType elementType = + DataTypeChecks.getNestedTypes(logicalType).get(0); + for (int i = 0; i < arrayData.size(); i++) { + appendTo( + type.getTypeParameters().get(0), + elementType, + InternalRowUtils.get(arrayData, i, elementType), + elementBuilder); + } + }); + } + catch (Throwable e) { + throw new RuntimeException(e); + } + return; + } + if (type instanceof RowType) { + RowBlockBuilder rowBlockBuilder = (RowBlockBuilder) output; + try { + rowBlockBuilder.buildEntry( + (RowValueBuilder) + fieldBuilders -> { + InternalRow rowData = (InternalRow) value; + for (int index = 0; + index < type.getTypeParameters().size(); + index++) { + Type fieldType = type.getTypeParameters().get(index); + DataType fieldLogicalType = + ((org.apache.paimon.types.RowType) logicalType) + .getTypeAt(index); + appendTo( + fieldType, + fieldLogicalType, + InternalRowUtils.get( + rowData, index, fieldLogicalType), + fieldBuilders.get(index)); + } + }); + } + catch (Throwable e) { + throw new RuntimeException(e); + } + return; + } + if (type instanceof MapType) { + InternalMap mapData = (InternalMap) value; + InternalArray keyArray = mapData.keyArray(); + InternalArray valueArray = mapData.valueArray(); + DataType keyType = ((org.apache.paimon.types.MapType) logicalType).getKeyType(); + DataType valueType = ((org.apache.paimon.types.MapType) logicalType).getValueType(); + MapBlockBuilder mapBlockBuilder = (MapBlockBuilder) output; + try { + mapBlockBuilder.buildEntry( + (MapValueBuilder) + (keyBuilder, valueBuilder) -> { + for (int i = 0; i < keyArray.size(); i++) { + appendTo( + type.getTypeParameters().get(0), + keyType, + InternalRowUtils.get(keyArray, i, keyType), + keyBuilder); + appendTo( + type.getTypeParameters().get(1), + valueType, + InternalRowUtils.get(valueArray, i, valueType), + valueBuilder); + } + }); + } + catch (Throwable e) { + throw new RuntimeException(e); + } + return; + } + throw new TrinoException( + GENERIC_INTERNAL_ERROR, "Unhandled type for Block: " + type.getTypeSignature()); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSourceProvider.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSourceProvider.java new file mode 100644 index 000000000000..9e3830291fa5 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSourceProvider.java @@ -0,0 +1,624 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.cloud.hadoop.repackaged.gcs.com.google.common.collect.ImmutableList; +import com.google.cloud.hadoop.repackaged.gcs.com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoInputFile; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.orc.OrcColumn; +import io.trino.orc.OrcDataSource; +import io.trino.orc.OrcReader; +import io.trino.orc.OrcReaderOptions; +import io.trino.orc.OrcRecordReader; +import io.trino.orc.TupleDomainOrcPredicate; +import io.trino.parquet.Column; +import io.trino.parquet.Field; +import io.trino.parquet.GroupField; +import io.trino.parquet.ParquetDataSource; +import io.trino.parquet.ParquetReaderOptions; +import io.trino.parquet.PrimitiveField; +import io.trino.parquet.metadata.FileMetadata; +import io.trino.parquet.metadata.ParquetMetadata; +import io.trino.parquet.predicate.TupleDomainParquetPredicate; +import io.trino.parquet.reader.MetadataReader; +import io.trino.parquet.reader.ParquetReader; +import io.trino.parquet.reader.RowGroupInfo; +import io.trino.plugin.base.metrics.FileFormatDataSourceStats; +import io.trino.plugin.hive.TransformConnectorPageSource; +import io.trino.plugin.hive.coercions.TypeCoercer; +import io.trino.plugin.hive.orc.OrcPageSource; +import io.trino.plugin.hive.parquet.ParquetPageSource; +import io.trino.plugin.hive.parquet.ParquetTypeTranslator; +import io.trino.plugin.paimon.catalog.TrinoCatalog; +import io.trino.plugin.paimon.catalog.TrinoCatalogFactory; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.MapType; +import io.trino.spi.type.Type; +import org.apache.paimon.deletionvectors.DeletionVector; +import org.apache.paimon.fileindex.FileIndexPredicate; +import org.apache.paimon.fs.Path; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.DeletionFile; +import org.apache.paimon.table.source.IndexFile; +import org.apache.paimon.table.source.RawFile; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.types.RowType; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.ColumnIO; +import org.apache.parquet.io.GroupColumnIO; +import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.io.PrimitiveColumnIO; +import org.apache.parquet.schema.MessageType; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.trino.orc.OrcReader.INITIAL_BATCH_SIZE; +import static io.trino.parquet.ParquetTypeUtils.getArrayElementColumn; +import static io.trino.parquet.ParquetTypeUtils.getColumnIO; +import static io.trino.parquet.ParquetTypeUtils.getDescriptors; +import static io.trino.parquet.ParquetTypeUtils.getMapKeyValueColumn; +import static io.trino.parquet.predicate.PredicateUtils.buildPredicate; +import static io.trino.parquet.predicate.PredicateUtils.getFilteredRowGroups; +import static io.trino.plugin.hive.orc.OrcTypeTranslator.createCoercer; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.createDataSource; +import static io.trino.plugin.paimon.PaimonSessionProperties.getParquetSmallFileThreshold; +import static java.util.Objects.requireNonNull; +import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; +import static org.joda.time.DateTimeZone.UTC; + +public class PaimonPageSourceProvider + implements ConnectorPageSourceProvider +{ + private final TrinoFileSystemFactory fileSystemFactory; + private final FileFormatDataSourceStats fileFormatDataSourceStats; + private final TrinoCatalogFactory paimonTrinoCatalogFactory; + + @Inject + public PaimonPageSourceProvider( + TrinoFileSystemFactory fileSystemFactory, + TrinoCatalogFactory paimonTrinoCatalogFactory, + FileFormatDataSourceStats fileFormatDataSourceStats) + { + this.paimonTrinoCatalogFactory = requireNonNull(paimonTrinoCatalogFactory, "paimonTrinoCatalogFactory is null"); + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.fileFormatDataSourceStats = requireNonNull(fileFormatDataSourceStats, "fileFormatDataSourceStats is null"); + } + + static TupleDomain getParquetTupleDomain( + MessageType fileSchema, + List columns, + TupleDomain predicates) + { + ImmutableMap.Builder predicate = ImmutableMap.builder(); + Map columnDescriptorMap = new HashMap<>(); + + fileSchema + .getPaths() + .forEach( + path -> { + org.apache.parquet.schema.Type type = fileSchema.getType(path[0]); + if (type.isPrimitive()) { + columnDescriptorMap.put( + type.getId().intValue(), + fileSchema.getColumnDescription(path)); + } + }); + + for (PaimonColumnHandle column : columns) { + if (predicates.getDomains().isPresent()) { + Domain domain = predicates.getDomains().get().get(column); + if (domain != null && columnDescriptorMap.containsKey(column.columnId())) { + predicate.put(columnDescriptorMap.get(column.columnId()), domain); + } + } + } + + return TupleDomain.withColumnDomains(predicate.build()); + } + + public static Optional constructField(Type type, ColumnIO columnIO) + { + if (columnIO == null) { + return Optional.empty(); + } + boolean required = columnIO.getType().getRepetition() != OPTIONAL; + int repetitionLevel = columnIO.getRepetitionLevel(); + int definitionLevel = columnIO.getDefinitionLevel(); + if (type instanceof io.trino.spi.type.RowType rowType) { + GroupColumnIO groupColumnIO = (GroupColumnIO) columnIO; + ImmutableList.Builder> fieldsBuilder = ImmutableList.builder(); + List fields = rowType.getFields(); + boolean structHasParameters = false; + for (int i = 0; i < fields.size(); i++) { + io.trino.spi.type.RowType.Field rowField = fields.get(i); + Optional field = + constructField(rowField.getType(), groupColumnIO.getChild(i)); + structHasParameters |= field.isPresent(); + fieldsBuilder.add(field); + } + if (structHasParameters) { + return Optional.of( + new GroupField( + type, + repetitionLevel, + definitionLevel, + required, + fieldsBuilder.build())); + } + return Optional.empty(); + } + if (type instanceof MapType mapType) { + GroupColumnIO groupColumnIO = (GroupColumnIO) columnIO; + GroupColumnIO keyValueColumnIO = getMapKeyValueColumn(groupColumnIO); + if (keyValueColumnIO.getChildrenCount() != 2) { + return Optional.empty(); + } + Optional keyField = + constructField(mapType.getKeyType(), keyValueColumnIO.getChild(0)); + Optional valueField = + constructField(mapType.getValueType(), keyValueColumnIO.getChild(1)); + return Optional.of( + new GroupField( + type, + repetitionLevel, + definitionLevel, + required, + ImmutableList.of(keyField, valueField))); + } + if (type instanceof ArrayType arrayType) { + GroupColumnIO groupColumnIO = (GroupColumnIO) columnIO; + if (groupColumnIO.getChildrenCount() != 1) { + return Optional.empty(); + } + Optional field = + constructField( + arrayType.getElementType(), + getArrayElementColumn(groupColumnIO.getChild(0))); + return Optional.of( + new GroupField( + type, + repetitionLevel, + definitionLevel, + required, + ImmutableList.of(field))); + } + PrimitiveColumnIO primitiveColumnIO = (PrimitiveColumnIO) columnIO; + return Optional.of( + new PrimitiveField( + type, + required, + primitiveColumnIO.getColumnDescriptor(), + primitiveColumnIO.getId())); + } + + @Override + public ConnectorPageSource createPageSource( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit split, + ConnectorTableHandle tableHandle, + List columns, + // TODO: support dynamic filter + DynamicFilter dynamicFilter) + { + TrinoCatalog paimonTrinoCatalog = paimonTrinoCatalogFactory.create(session.getIdentity()); + PaimonTableHandle paimonTableHandle = (PaimonTableHandle) tableHandle; + Table table = paimonTrinoCatalog.loadTable(session, SchemaTableName.schemaTableName(paimonTableHandle.getSchemaName(), paimonTableHandle.getTableName())); + Optional rowId = columns.stream() + .map(PaimonColumnHandle.class::cast) + .filter(PaimonColumnHandle::isRowId) + .findFirst(); + if (rowId.isPresent()) { + List dataColumns = columns.stream() + .map(PaimonColumnHandle.class::cast) + .filter(column -> !column.isRowId()) + .collect(toImmutableList()); + Set rowIdFields = ((io.trino.spi.type.RowType) rowId.get().trinoType()) + .getFields().stream() + .map(io.trino.spi.type.RowType.Field::getName) + .map(Optional::get) + .collect(toImmutableSet()); + + HashMap fieldToIndex = new HashMap<>(); + for (int i = 0; i < dataColumns.size(); i++) { + PaimonColumnHandle columnHandle = (PaimonColumnHandle) dataColumns.get(i); + if (rowIdFields.contains(columnHandle.columnName())) { + fieldToIndex.put(columnHandle.columnName(), i); + } + } + return new PaimonMergePageSourceWrapper(createPageSource( + session, + table, + paimonTableHandle.getPredicate(), + (PaimonSplit) split, + dataColumns), + fieldToIndex); + } + return createPageSource( + session, + table, + paimonTableHandle.getPredicate(), + (PaimonSplit) split, + columns); + } + + private ConnectorPageSource createPageSource( + ConnectorSession session, + Table table, + TupleDomain filter, + PaimonSplit split, + List columns) + { + RowType rowType = table.rowType(); + List fieldNames = rowType.getFieldNames(); + List projectedFields = columns.stream() + .map(PaimonColumnHandle.class::cast) + .map(PaimonColumnHandle::columnName) + .toList(); + + List projectedColumns = columns.stream().map(PaimonColumnHandle.class::cast).toList(); + + TrinoFileSystem fileSystem = fileSystemFactory.create(session); + Optional paimonFilter = new PaimonFilterConverter(rowType).convert(filter); + Optional pageSource = Optional.empty(); + + try { + Split paimonSplit = split.decodeSplit(); + Optional> optionalRawFiles = paimonSplit.convertToRawFiles(); + if (checkRawFile(optionalRawFiles)) { + FileStoreTable fileStoreTable = (FileStoreTable) table; + boolean readIndex = fileStoreTable.coreOptions().fileIndexReadEnabled(); + + Optional> deletionFiles = paimonSplit.deletionFiles(); + Optional> indexFiles = readIndex ? paimonSplit.indexFiles() : Optional.empty(); + + try { + List files = optionalRawFiles.orElseThrow(); + List sources = new ArrayList<>(); + + // if file index exists, do the filter. + for (int i = 0; i < files.size(); i++) { + RawFile rawFile = files.get(i); + if (indexFiles.isPresent()) { + IndexFile indexFile = indexFiles.get().get(i); + if (indexFile != null && paimonFilter.isPresent()) { + try (FileIndexPredicate fileIndexPredicate = + new FileIndexPredicate( + new Path(indexFile.path()), + ((FileStoreTable) table).fileIO(), + rowType)) { + if (!fileIndexPredicate.evaluate(paimonFilter.get()).remain()) { + continue; + } + } + } + } + ConnectorPageSource source = + createDataPageSource( + session, + rawFile.format(), + fileSystem.newInputFile(Location.of(rawFile.path())), + projectedColumns, + filter); + + if (deletionFiles.isPresent()) { + source = + new PaimonPageSourceWrapper( + source, + Optional.ofNullable(deletionFiles.get().get(i)) + .map( + deletionFile -> { + try { + return DeletionVector.read( + fileStoreTable.fileIO(), + deletionFile); + } + catch (IOException e) { + throw new RuntimeException(e); + } + })); + } + sources.add(source); + } + + pageSource = Optional.of(new DirectPaimonPageSource(sources)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + if (pageSource.isEmpty()) { + int[] columnIndex = + projectedFields.stream().mapToInt(fieldNames::indexOf).toArray(); + + // old read way + ReadBuilder read = table.newReadBuilder(); + paimonFilter.ifPresent(read::withFilter); + + if (!fieldNames.equals(projectedFields)) { + read.withProjection(columnIndex); + } + + pageSource = Optional.of(new PaimonPageSource( + read.newRead().executeFilter().createReader(paimonSplit), columns, newSimpleAggregatedMemoryContext())); + } + return pageSource.get(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private boolean checkRawFile(Optional> optionalRawFiles) + { + return optionalRawFiles.isPresent() && canUseTrinoPageSource(optionalRawFiles.get()); + } + + // TODO: support avro + private boolean canUseTrinoPageSource(List rawFiles) + { + for (RawFile rawFile : rawFiles) { + if (!(rawFile.format().equals("orc") || rawFile.format().equals("parquet"))) { + return false; + } + } + return true; + } + + private ConnectorPageSource createDataPageSource( + ConnectorSession session, + String format, + TrinoInputFile inputFile, + List columns, + TupleDomain predicates) + { + return switch (format) { + case "orc" -> createOrcDataPageSource( + inputFile, + // TODO: pass options to orc read from configuration + new OrcReaderOptions() + .withTinyStripeThreshold( + PaimonSessionProperties.getOrcTinyStripeThreshold(session)), + columns, + predicates); + case "parquet" -> createParquetDataPageSource( + // TODO: pass options to parquet read from configuration + inputFile, ParquetReaderOptions.builder() + .withSmallFileThreshold(getParquetSmallFileThreshold(session)).build(), columns, predicates); + case "avro" -> throw new RuntimeException("Unsupport file format: " + format); + default -> throw new RuntimeException("Unsupport file format: " + format); + }; + } + + private ConnectorPageSource createOrcDataPageSource( + TrinoInputFile inputFile, + OrcReaderOptions options, + List columns, + TupleDomain predicates) + { + try { + OrcDataSource orcDataSource = + new PaimonOrcDataSource(inputFile, options, fileFormatDataSourceStats); + OrcReader reader = + OrcReader.createOrcReader(orcDataSource, options) + .orElseThrow(() -> new RuntimeException("ORC file is zero length")); + + List fileColumns = reader.getRootColumn().getNestedColumns(); + Map fieldsMap = new HashMap<>(); + fileColumns.forEach( + column -> + fieldsMap.put( + Integer.parseInt(column.getAttributes().get("paimon.id")), + column)); + TupleDomainOrcPredicate.TupleDomainOrcPredicateBuilder predicateBuilder = + TupleDomainOrcPredicate.builder(); + TransformConnectorPageSource.Builder transforms = TransformConnectorPageSource.builder(); + List fileReadColumns = new ArrayList<>(columns.size()); + List fileReadTypes = new ArrayList<>(columns.size()); + + for (PaimonColumnHandle column : columns) { + OrcColumn orcColumn = fieldsMap.get(column.columnId()); + if (orcColumn == null) { + transforms.constantValue(column.trinoType().createNullBlock()); + } + else { + Optional> coercer = createCoercer(orcColumn.getColumnType(), orcColumn.getNestedColumns(), column.trinoType()); + if (coercer.isPresent()) { + fileReadTypes.add(coercer.get().getFromType()); + transforms.transform(fileReadColumns.size(), coercer.get()); + } + else { + fileReadTypes.add(column.trinoType()); + transforms.column(fileReadColumns.size()); + } + fileReadColumns.add(orcColumn); + if (predicates.getDomains().isPresent()) { + Domain predicate = predicates.getDomains().get().get(column); + if (predicate != null) { + predicateBuilder.addColumn(orcColumn.getColumnId(), predicate); + } + } + } + } + + AggregatedMemoryContext memoryUsage = newSimpleAggregatedMemoryContext(); + OrcRecordReader recordReader = + reader.createRecordReader( + fileReadColumns, + fileReadTypes, + false, + predicateBuilder.build(), + DateTimeZone.UTC, + memoryUsage, + INITIAL_BATCH_SIZE, + RuntimeException::new); + + ConnectorPageSource pageSource = new OrcPageSource( + recordReader, + orcDataSource, + Optional.empty(), + Optional.empty(), + memoryUsage, + fileFormatDataSourceStats, + reader.getCompressionKind()); + + return transforms.build(pageSource); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private ConnectorPageSource createParquetDataPageSource( + TrinoInputFile inputFile, + ParquetReaderOptions options, + List columns, + TupleDomain predicates) + { + try { + AggregatedMemoryContext memoryUsage = newSimpleAggregatedMemoryContext(); + ParquetDataSource dataSource = + createDataSource( + inputFile, + OptionalLong.of(inputFile.length()), + options, + memoryUsage, + fileFormatDataSourceStats); + ParquetMetadata parquetMetadata = + MetadataReader.readFooter(dataSource, Optional.empty()); + FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); + MessageType fileSchema = fileMetaData.getSchema(); + + Map fileSchemaMap = new HashMap<>(); + for (org.apache.parquet.schema.Type field : fileSchema.getFields()) { + fileSchemaMap.put(field.getId().intValue(), field); + } + List projectedTypes = new ArrayList<>(); + for (PaimonColumnHandle column : columns) { + if (fileSchemaMap.containsKey(column.columnId())) { + projectedTypes.add(fileSchemaMap.get(column.columnId())); + } + } + MessageType projectedSchema = new MessageType(fileSchema.getName(), projectedTypes); + Map, ColumnDescriptor> descriptorsByPath = + getDescriptors(fileSchema, projectedSchema); + TupleDomain parquetTupleDomain = + options.isIgnoreStatistics() + ? TupleDomain.all() + : getParquetTupleDomain(fileSchema, columns, predicates); + TupleDomainParquetPredicate parquetPredicate = + buildPredicate(projectedSchema, parquetTupleDomain, descriptorsByPath, UTC); + List rowGroups = + getFilteredRowGroups( + 0, + inputFile.length(), + dataSource, + parquetMetadata, + ImmutableList.of(parquetTupleDomain), + ImmutableList.of(parquetPredicate), + descriptorsByPath, + UTC, + 1000, + options); + + MessageColumnIO messageColumnIO = getColumnIO(fileSchema, projectedSchema); + TransformConnectorPageSource.Builder transforms = TransformConnectorPageSource.builder(); + + int parquetSourceChannel = 0; + List returnColumns = new ArrayList<>(); + for (PaimonColumnHandle columnHandle : columns) { + if (fileSchemaMap.containsKey(columnHandle.columnId())) { + org.apache.parquet.schema.Type type = + fileSchemaMap.get(columnHandle.columnId()); + ColumnIO columnIO = messageColumnIO.getChild(type.getName()); + Type trinoType = columnHandle.trinoType(); + Optional> coercer = Optional.empty(); + if (type.isPrimitive()) { + coercer = ParquetTypeTranslator.createCoercer(type.asPrimitiveType().getPrimitiveTypeName(), type.getLogicalTypeAnnotation(), trinoType); + } + if (coercer.isPresent()) { + returnColumns.add( + new Column( + columnHandle.columnName(), + constructField(coercer.get().getFromType(), columnIO) + .orElseThrow())); + transforms.transform(parquetSourceChannel++, coercer.get()); + } + else { + returnColumns.add( + new Column( + columnHandle.columnName(), + constructField(columnHandle.trinoType(), columnIO) + .orElseThrow())); + transforms.column(parquetSourceChannel++); + } + } + else { + transforms.constantValue(columnHandle.trinoType().createNullBlock()); + } + } + + ParquetReader parquetReader = + new ParquetReader( + Optional.ofNullable(fileMetaData.getCreatedBy()), + returnColumns, + false, + rowGroups, + dataSource, + UTC, + memoryUsage, + options, + RuntimeException::new, + Optional.empty(), + Optional.empty(), + Optional.empty()); + + ConnectorPageSource pageSource = new ParquetPageSource(parquetReader); + return transforms.build(pageSource); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSourceWrapper.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSourceWrapper.java new file mode 100644 index 000000000000..18d1b0d26fc2 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPageSourceWrapper.java @@ -0,0 +1,120 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.SourcePage; +import io.trino.spi.metrics.Metrics; +import org.apache.paimon.deletionvectors.DeletionVector; + +import java.io.IOException; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.CompletableFuture; + +import static java.util.Objects.requireNonNull; + +public class PaimonPageSourceWrapper + implements ConnectorPageSource +{ + private final ConnectorPageSource source; + private final Optional deletionVector; + + public PaimonPageSourceWrapper(ConnectorPageSource source, Optional deletionVector) + { + this.source = requireNonNull(source, "source is null"); + this.deletionVector = requireNonNull(deletionVector, "deletionVector is null"); + } + + @Override + public long getCompletedBytes() + { + return source.getCompletedBytes(); + } + + @Override + public OptionalLong getCompletedPositions() + { + return source.getCompletedPositions(); + } + + @Override + public long getReadTimeNanos() + { + return source.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return source.isFinished(); + } + + @Override + public SourcePage getNextSourcePage() + { + int startPosition = (int) source.getCompletedPositions().orElseThrow(); + SourcePage next = source.getNextSourcePage(); + if (next == null) { + return next; + } + + int pageCount = next.getPositionCount(); + + return deletionVector + .map(deletionVector -> convertToRetained(next, deletionVector, startPosition, pageCount)) + .orElse(next); + } + + private static SourcePage convertToRetained(SourcePage page, DeletionVector deletionVector, int startPosition, int pageCount) + { + int[] retained = new int[pageCount]; + int retainedLength = 0; + for (int pagePosition = 0; pagePosition < pageCount; pagePosition++) { + if (!deletionVector.isDeleted(startPosition + pagePosition)) { + retained[retainedLength++] = pagePosition; + } + } + if (retainedLength == pageCount) { + return page; + } + + return SourcePage.create(page.getPage().getPositions(retained, 0, retainedLength)); + } + + @Override + public long getMemoryUsage() + { + return source.getMemoryUsage(); + } + + @Override + public void close() + throws IOException + { + source.close(); + } + + @Override + public CompletableFuture isBlocked() + { + return source.isBlocked(); + } + + @Override + public Metrics getMetrics() + { + return source.getMetrics(); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPlugin.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPlugin.java new file mode 100644 index 000000000000..40dbce934f55 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonPlugin.java @@ -0,0 +1,28 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; + +public class PaimonPlugin + implements Plugin +{ + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new PaimonConnectorFactory()); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonRow.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonRow.java new file mode 100644 index 000000000000..76d687161aed --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonRow.java @@ -0,0 +1,229 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.airlift.slice.Slice; +import io.trino.spi.Page; +import io.trino.spi.TrinoException; +import io.trino.spi.block.ByteArrayBlock; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Int128; +import io.trino.spi.type.SqlTime; +import io.trino.spi.type.SqlTimestamp; +import io.trino.spi.type.SqlTimestampWithTimeZone; +import io.trino.spi.type.TimeType; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TimestampWithTimeZoneType; +import io.trino.spi.type.Type; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.data.variant.Variant; +import org.apache.paimon.types.DataTypeRoot; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.types.RowType; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.math.BigInteger; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.Decimals.MAX_SHORT_PRECISION; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TypeUtils.readNativeValue; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static java.lang.Float.intBitsToFloat; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public class PaimonRow + implements InternalRow, Serializable +{ + private final RowType rowType; + private final RowKind rowKind; + private final Page singlePage; + private final Type[] cacheTypes; + + public PaimonRow(RowType rowType, Page singlePage, RowKind rowKind) + { + verify(singlePage.getPositionCount() == 1, "singlePage must have only one row"); + this.rowType = requireNonNull(rowType, "rowType is null"); + this.singlePage = requireNonNull(singlePage, "singlePage is null"); + this.rowKind = requireNonNull(rowKind, "rowKind is null"); + cacheTypes = new Type[singlePage.getChannelCount()]; + } + + @Override + public int getFieldCount() + { + return singlePage.getChannelCount(); + } + + @Override + public RowKind getRowKind() + { + return rowKind; + } + + @Override + public void setRowKind(RowKind rowKind) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isNullAt(int i) + { + return singlePage.getBlock(i).isNull(0); + } + + @Override + public boolean getBoolean(int i) + { + return (boolean) readNativeValue(BOOLEAN, singlePage.getBlock(i), 0); + } + + @Override + public byte getByte(int i) + { + return ((ByteArrayBlock) singlePage.getBlock(i)).getByte(0); + } + + @Override + public short getShort(int i) + { + long value = (long) readNativeValue(SMALLINT, singlePage.getBlock(i), 0); + checkArgument(value >= Short.MIN_VALUE && value <= Short.MAX_VALUE, "Value out of range for short: %s", value); + return (short) value; + } + + @Override + public int getInt(int i) + { + if (rowType.getTypeAt(i).getTypeRoot() == DataTypeRoot.TIME_WITHOUT_TIME_ZONE) { + if (cacheTypes[i] == null) { + cacheTypes[i] = TimeType.createTimeType(((org.apache.paimon.types.TimeType) rowType.getTypeAt(i)).getPrecision()); + } + SqlTime sqlTime = (SqlTime) cacheTypes[i].getObjectValue(singlePage.getBlock(i), 0); + return (int) (sqlTime.getPicos() / 1_000_000); + } + + long value = (long) readNativeValue(INTEGER, singlePage.getBlock(i), 0); + checkArgument(value >= Integer.MIN_VALUE && value <= Integer.MAX_VALUE, "Value out of range for int: %s", value); + return (int) value; + } + + @Override + public long getLong(int i) + { + return (long) readNativeValue(BIGINT, singlePage.getBlock(i), 0); + } + + @Override + public float getFloat(int i) + { + return intBitsToFloat( + toIntExact((long) readNativeValue(REAL, singlePage.getBlock(i), 0))); + } + + @Override + public double getDouble(int i) + { + return (double) readNativeValue(DOUBLE, singlePage.getBlock(i), 0); + } + + @Override + public BinaryString getString(int i) + { + return BinaryString.fromBytes(getBinary(i)); + } + + @Override + public Decimal getDecimal(int i, int decimalPrecision, int decimalScale) + { + Object value = + readNativeValue( + DecimalType.createDecimalType(decimalPrecision, decimalScale), + singlePage.getBlock(i), + 0); + if (decimalPrecision <= MAX_SHORT_PRECISION) { + return Decimal.fromUnscaledLong((Long) value, decimalPrecision, decimalScale); + } + else { + long high = ((Int128) value).getHigh(); + long low = ((Int128) value).getLow(); + BigInteger bigIntegerValue = + BigInteger.valueOf(high).shiftLeft(64).add(BigInteger.valueOf(low)); + BigDecimal bigDecimalValue = new BigDecimal(bigIntegerValue, decimalScale); + return Decimal.fromBigDecimal(bigDecimalValue, decimalPrecision, decimalScale); + } + } + + @Override + public Timestamp getTimestamp(int i, int timestampPrecision) + { + if (rowType.getTypeAt(i).getTypeRoot() == DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE) { + SqlTimestampWithTimeZone sqlTimestampWithTimeZone = (SqlTimestampWithTimeZone) TimestampWithTimeZoneType.createTimestampWithTimeZoneType(timestampPrecision).getObjectValue(singlePage.getBlock(i), 0); + return Timestamp.fromEpochMillis(sqlTimestampWithTimeZone.getEpochMillis(), sqlTimestampWithTimeZone.getPicosOfMilli() * 1000); + } + else { + SqlTimestamp sqlTimestamp = (SqlTimestamp) TimestampType.createTimestampType(timestampPrecision).getObjectValue(singlePage.getBlock(i), 0); + return Timestamp.fromLocalDateTime(sqlTimestamp.toLocalDateTime()); + } + } + + @Override + public byte[] getBinary(int i) + { + Slice slice = (Slice) readNativeValue(VARBINARY, singlePage.getBlock(i), 0); + return slice.getBytes(); + } + + @Override + public Variant getVariant(int i) + { + throw new TrinoException(NOT_SUPPORTED, "variant type is not supported."); + } + + @Override + public InternalArray getArray(int i) + { + // todo + throw new TrinoException(NOT_SUPPORTED, "array type is not yet supported."); + } + + @Override + public InternalMap getMap(int i) + { + // todo + throw new TrinoException(NOT_SUPPORTED, "map type is not yet supported."); + } + + @Override + public InternalRow getRow(int i, int i1) + { + // todo + throw new TrinoException(NOT_SUPPORTED, "row type is not yet supported."); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSessionProperties.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSessionProperties.java new file mode 100644 index 000000000000..f3785a6fc652 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSessionProperties.java @@ -0,0 +1,99 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.airlift.units.DataSize; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.orc.OrcReaderConfig; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.session.PropertyMetadata; + +import java.util.List; + +import static io.trino.plugin.base.session.PropertyMetadataUtil.dataSizeProperty; +import static io.trino.plugin.base.session.PropertyMetadataUtil.validateMaxDataSize; +import static io.trino.plugin.hive.parquet.ParquetReaderConfig.PARQUET_READER_MAX_SMALL_FILE_THRESHOLD; +import static io.trino.spi.session.PropertyMetadata.booleanProperty; +import static io.trino.spi.session.PropertyMetadata.doubleProperty; + +public class PaimonSessionProperties + implements SessionPropertiesProvider +{ + public static final String MINIMUM_SPLIT_WEIGHT = "minimum_split_weight"; + private static final String PARQUET_SMALL_FILE_THRESHOLD = "parquet_small_file_threshold"; + private static final String ORC_TINY_STRIPE_THRESHOLD = "orc_tiny_stripe_threshold"; + private static final String PROJECTION_PUSHDOWN_ENABLED = "projection_pushdown_enabled"; + + private final List> sessionProperties; + + @Inject + public PaimonSessionProperties( + OrcReaderConfig orcReaderConfig, + ParquetReaderConfig parquetReaderConfig, + PaimonConfig paimonConfig) + { + sessionProperties = ImmutableList.>builder() + .add(doubleProperty( + MINIMUM_SPLIT_WEIGHT, + "Minimum split weight", + 0.05, + false)) + .add(dataSizeProperty( + PARQUET_SMALL_FILE_THRESHOLD, + "Parquet: Size below which a parquet file will be read entirely", + parquetReaderConfig.getSmallFileThreshold(), + value -> validateMaxDataSize(PARQUET_SMALL_FILE_THRESHOLD, value, DataSize.valueOf(PARQUET_READER_MAX_SMALL_FILE_THRESHOLD)), + false)) + .add(dataSizeProperty( + ORC_TINY_STRIPE_THRESHOLD, + "ORC: Threshold below which an ORC stripe or file will read in its entirety", + orcReaderConfig.getTinyStripeThreshold(), + false)) + .add(booleanProperty( + PROJECTION_PUSHDOWN_ENABLED, + "Read only required fields from a row type", + paimonConfig.isProjectionPushdownEnabled(), + false)) + .build(); + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } + + public static Double getMinimumSplitWeight(ConnectorSession session) + { + return session.getProperty(MINIMUM_SPLIT_WEIGHT, Double.class); + } + + public static DataSize getParquetSmallFileThreshold(ConnectorSession session) + { + return session.getProperty(PARQUET_SMALL_FILE_THRESHOLD, DataSize.class); + } + + public static DataSize getOrcTinyStripeThreshold(ConnectorSession session) + { + return session.getProperty(ORC_TINY_STRIPE_THRESHOLD, DataSize.class); + } + + public static boolean isProjectionPushdownEnabled(ConnectorSession session) + { + return session.getProperty(PROJECTION_PUSHDOWN_ENABLED, Boolean.class); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplit.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplit.java new file mode 100644 index 000000000000..e52a6e0fdb86 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplit.java @@ -0,0 +1,58 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.spi.HostAddress; +import io.trino.spi.SplitWeight; +import io.trino.spi.connector.ConnectorSplit; +import org.apache.paimon.table.source.Split; + +import java.util.Collections; +import java.util.List; + +import static io.trino.plugin.paimon.EncodingUtils.decodeStringToObject; +import static io.trino.plugin.paimon.EncodingUtils.encodeObjectToString; +import static java.util.Objects.requireNonNull; + +public record PaimonSplit(String splitSerialized, Double weight) + implements ConnectorSplit +{ + public PaimonSplit + { + requireNonNull(splitSerialized, "splitSerialized is null"); + requireNonNull(weight, "weight is null"); + } + + public static PaimonSplit fromSplit(Split split, Double weight) + { + return new PaimonSplit(encodeObjectToString(split), weight); + } + + public Split decodeSplit() + { + return (Split) decodeStringToObject(splitSerialized); + } + + @Override + public List getAddresses() + { + return Collections.emptyList(); + } + + @Override + public SplitWeight getSplitWeight() + { + return SplitWeight.fromProportion(weight); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplitManager.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplitManager.java new file mode 100644 index 000000000000..effcb56a1bdd --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplitManager.java @@ -0,0 +1,77 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.inject.Inject; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.DynamicFilter; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.Split; + +import java.util.List; +import java.util.stream.Collectors; + +import static java.util.Objects.requireNonNull; + +public class PaimonSplitManager + implements ConnectorSplitManager +{ + private final PaimonTransactionManager transactionManager; + + @Inject + public PaimonSplitManager(PaimonTransactionManager transactionManager) + { + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle handle, + DynamicFilter dynamicFilter, + Constraint constraint) + { + try { + // TODO dynamicFilter? + // TODO what is constraint? + PaimonMetadata metadata = transactionManager.get(transaction, session.getIdentity()); + PaimonTableHandle tableHandle = (PaimonTableHandle) handle; + Table table = metadata.table(session, tableHandle.getSchemaName(), tableHandle.getTableName()); + ReadBuilder readBuilder = table.newReadBuilder(); + new PaimonFilterConverter(table.rowType()) + .convert(tableHandle.getPredicate()) + .ifPresent(readBuilder::withFilter); + List splits = readBuilder.dropStats().newScan().plan().splits(); + + long maxRowCount = splits.stream().mapToLong(Split::rowCount).max().orElse(0L); + double minimumSplitWeight = PaimonSessionProperties.getMinimumSplitWeight(session); + return new PaimonSplitSource(splits.stream() + .map(split -> PaimonSplit.fromSplit( + split, + Math.min(Math.max((double) split.rowCount() / maxRowCount, minimumSplitWeight), 1.0))) + .collect(Collectors.toList())); + } + catch (IllegalArgumentException e) { + throw new TrinoException(PaimonErrorCode.PAIMON_CANNOT_OPEN_SPLIT, "Fetch splits failed!", e); + } + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplitSource.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplitSource.java new file mode 100644 index 000000000000..06d0dd52085d --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonSplitSource.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorSplitSource; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; + +public class PaimonSplitSource + implements ConnectorSplitSource +{ + private final Queue splits; + + public PaimonSplitSource(List splits) + { + this.splits = new ArrayDeque<>(splits); + } + + protected CompletableFuture innerGetNextBatch(int maxSize) + { + List batch = new ArrayList<>(); + for (int i = 0; i < maxSize; i++) { + PaimonSplit split = splits.poll(); + if (split == null) { + break; + } + batch.add(split); + } + return CompletableFuture.completedFuture(new ConnectorSplitBatch(batch, isFinished())); + } + + @Override + public CompletableFuture getNextBatch(int maxSize) + { + return innerGetNextBatch(maxSize); + } + + @Override + public void close() {} + + @Override + public boolean isFinished() + { + return splits.isEmpty(); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableHandle.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableHandle.java new file mode 100644 index 000000000000..ebbc99f063e2 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableHandle.java @@ -0,0 +1,83 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorOutputTableHandle; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.predicate.TupleDomain; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class PaimonTableHandle + implements ConnectorTableHandle, ConnectorInsertTableHandle, ConnectorOutputTableHandle +{ + private final String schemaName; + private final String tableName; + private final TupleDomain predicate; + + @JsonCreator + public PaimonTableHandle( + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("predicate") TupleDomain predicate) + { + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.predicate = requireNonNull(predicate, "predicate is null"); + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public TupleDomain getPredicate() + { + return predicate; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PaimonTableHandle that = (PaimonTableHandle) o; + return Objects.equals(schemaName, that.schemaName) + && Objects.equals(tableName, that.tableName) + && Objects.equals(predicate, that.predicate); + } + + @Override + public int hashCode() + { + return Objects.hash(schemaName, tableName, predicate); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableOptionUtils.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableOptionUtils.java new file mode 100644 index 000000000000..0f55256d7090 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableOptionUtils.java @@ -0,0 +1,164 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.utils.StringUtils; + +import java.lang.reflect.Field; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public final class PaimonTableOptionUtils +{ + private PaimonTableOptionUtils() {} + + public static void buildOptions(Schema.Builder builder, Map properties) + { + List optionInfos = PaimonTableOptionUtils.getOptionInfos(); + for (OptionInfo optionInfo : optionInfos) { + if (properties.get(optionInfo.trinoOptionKey) != null + && !StringUtils.isNullOrWhitespaceOnly( + String.valueOf(properties.get(optionInfo.trinoOptionKey)))) { + builder.option( + optionInfo.paimonOptionKey, + String.valueOf(properties.get(optionInfo.trinoOptionKey))); + } + } + } + + public static List getOptionInfos() + { + List optionInfos = new ArrayList<>(); + List optionWithMetaInfos = extractConfigOptions(CoreOptions.class); + String className = ""; + for (OptionWithMetaInfo optionWithMetaInfo : optionWithMetaInfos) { + if (shouldSkip(optionWithMetaInfo.field.getName())) { + continue; + } + + Type genericType = optionWithMetaInfo.field.getGenericType(); + if (genericType instanceof ParameterizedType parameterizedType) { + Type[] actualTypeArguments = parameterizedType.getActualTypeArguments(); + for (Type actualTypeArgument : actualTypeArguments) { + if (actualTypeArgument instanceof Class) { + className = ((Class) actualTypeArgument).getSimpleName(); + } + } + } + + optionInfos.add( + new OptionInfo( + convertOptionKey(optionWithMetaInfo.option.key()), + optionWithMetaInfo.option.key(), + buildClass(className), + isEnum(className), + className)); + } + return optionInfos; + } + + private static boolean shouldSkip(String fieldName) + { + return switch (fieldName) { + case "PRIMARY_KEY", "PARTITION", "FILE_COMPRESSION_PER_LEVEL", "STREAMING_COMPACT" -> true; + default -> false; + }; + } + + private static boolean isEnum(String className) + { + return switch (className) { + case "StartupMode", "MergeEngine", "ChangelogProducer", "LogConsistency", "LogChangelogMode", "StreamingReadMode" -> true; + default -> false; + }; + } + + private static Class buildClass(String className) + { + return switch (className) { + case "MergeEngine" -> CoreOptions.MergeEngine.class; + case "ChangelogProducer" -> CoreOptions.ChangelogProducer.class; + case "StartupMode" -> CoreOptions.StartupMode.class; + case "LogConsistency" -> CoreOptions.LogConsistency.class; + case "LogChangelogMode" -> CoreOptions.LogChangelogMode.class; + case "StreamingReadMode" -> CoreOptions.StreamingReadMode.class; + default -> null; + }; + } + + private static String convertOptionKey(String key) + { + String regex = "[.\\-]"; + Pattern pattern = Pattern.compile(regex); + Matcher matcher = pattern.matcher(key); + return matcher.replaceAll("_"); + } + + private static List extractConfigOptions(Class clazz) + { + try { + List configOptions = new ArrayList<>(8); + Field[] fields = clazz.getFields(); + for (Field field : fields) { + if (isConfigOption(field)) { + configOptions.add( + new OptionWithMetaInfo((ConfigOption) field.get(null), field)); + } + } + return configOptions; + } + catch (Exception e) { + throw new RuntimeException( + "Failed to extract config options from class " + clazz + '.', e); + } + } + + private static boolean isConfigOption(Field field) + { + return field.getType().equals(ConfigOption.class); + } + + private record OptionWithMetaInfo(ConfigOption option, Field field) {} + + static class OptionInfo + { + String trinoOptionKey; + String paimonOptionKey; + Class clazz; + boolean isEnum; + String type; + + public OptionInfo( + String trinoOptionKey, + String paimonOptionKey, + Class clazz, + boolean isEnum, + String type) + { + this.trinoOptionKey = trinoOptionKey; + this.paimonOptionKey = paimonOptionKey; + this.clazz = clazz; + this.isEnum = isEnum; + this.type = type; + } + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableOptions.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableOptions.java new file mode 100644 index 000000000000..c3cef7b692b8 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTableOptions.java @@ -0,0 +1,91 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.session.PropertyMetadata; +import io.trino.spi.type.ArrayType; + +import java.util.List; +import java.util.Map; + +import static io.trino.spi.session.PropertyMetadata.enumProperty; +import static io.trino.spi.session.PropertyMetadata.stringProperty; +import static io.trino.spi.type.VarcharType.VARCHAR; + +public class PaimonTableOptions +{ + public static final String PRIMARY_KEY_IDENTIFIER = "primary_key"; + public static final String PARTITIONED_BY_PROPERTY = "partitioned_by"; + + private final List> tableProperties; + + public PaimonTableOptions() + { + ImmutableList.Builder> builder = ImmutableList.builder(); + List optionInfos = PaimonTableOptionUtils.getOptionInfos(); + optionInfos.forEach( + item -> { + if (item.isEnum) { + builder.add(enumProperty(item.trinoOptionKey, "option", item.clazz, null, false)); + } + else { + builder.add(stringProperty(item.trinoOptionKey, "option", null, false)); + } + }); + + builder.add( + new PropertyMetadata<>( + PRIMARY_KEY_IDENTIFIER, + "Primary keys for the table.", + new ArrayType(VARCHAR), + List.class, + ImmutableList.of(), + false, + value -> (List) value, + value -> value)); + + builder.add( + new PropertyMetadata<>( + PARTITIONED_BY_PROPERTY, + "Partition keys for the table.", + new ArrayType(VARCHAR), + List.class, + ImmutableList.of(), + false, + value -> (List) value, + value -> value)); + + tableProperties = builder.build(); + } + + @SuppressWarnings("unchecked") + public static List getPrimaryKeys(Map tableProperties) + { + List primaryKeys = (List) tableProperties.get(PRIMARY_KEY_IDENTIFIER); + return primaryKeys == null ? ImmutableList.of() : ImmutableList.copyOf(primaryKeys); + } + + @SuppressWarnings("unchecked") + public static List getPartitionedKeys(Map tableProperties) + { + List partitionedKeys = (List) tableProperties.get(PARTITIONED_BY_PROPERTY); + return partitionedKeys == null ? ImmutableList.of() : ImmutableList.copyOf(partitionedKeys); + } + + public List> getTableProperties() + { + return tableProperties; + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTransactionManager.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTransactionManager.java new file mode 100644 index 000000000000..d96badb03e32 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTransactionManager.java @@ -0,0 +1,92 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.errorprone.annotations.concurrent.GuardedBy; +import com.google.inject.Inject; +import io.trino.spi.classloader.ThreadContextClassLoader; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.security.ConnectorIdentity; + +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +public class PaimonTransactionManager +{ + private final PaimonMetadataFactory metadataFactory; + private final ClassLoader classLoader; + private final ConcurrentMap transactions = new ConcurrentHashMap<>(); + + @Inject + public PaimonTransactionManager(PaimonMetadataFactory metadataFactory) + { + this.metadataFactory = requireNonNull(metadataFactory, "metadataFactory is null"); + this.classLoader = Thread.currentThread().getContextClassLoader(); + } + + public void begin(ConnectorTransactionHandle transactionHandle) + { + MemoizedMetadata previousValue = transactions.putIfAbsent(transactionHandle, new MemoizedMetadata()); + checkState(previousValue == null); + } + + public PaimonMetadata get(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity) + { + return transactions.get(transactionHandle).get(identity); + } + + public void commit(ConnectorTransactionHandle transaction) + { + MemoizedMetadata transactionalMetadata = transactions.remove(transaction); + checkArgument(transactionalMetadata != null, "no such transaction: %s", transaction); + } + + public void rollback(ConnectorTransactionHandle transaction) + { + MemoizedMetadata transactionalMetadata = transactions.remove(transaction); + checkArgument(transactionalMetadata != null, "no such transaction: %s", transaction); + transactionalMetadata.optionalGet() + .ifPresent(metadata -> { + try (ThreadContextClassLoader _ = new ThreadContextClassLoader(classLoader)) { + metadata.rollback(); + } + }); + } + + private class MemoizedMetadata + { + @GuardedBy("this") + private PaimonMetadata metadata; + + private synchronized Optional optionalGet() + { + return Optional.ofNullable(metadata); + } + + private synchronized PaimonMetadata get(ConnectorIdentity identity) + { + if (metadata == null) { + try (ThreadContextClassLoader _ = new ThreadContextClassLoader(classLoader)) { + metadata = metadataFactory.create(identity); + } + } + return metadata; + } + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTypeUtils.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTypeUtils.java new file mode 100644 index 000000000000..d5ee3dca4d13 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/PaimonTypeUtils.java @@ -0,0 +1,351 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.spi.type.BigintType; +import io.trino.spi.type.IntegerType; +import io.trino.spi.type.RealType; +import io.trino.spi.type.RowType.Field; +import io.trino.spi.type.SmallintType; +import io.trino.spi.type.TimestampWithTimeZoneType; +import io.trino.spi.type.TinyintType; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeOperators; +import io.trino.spi.type.VarbinaryType; +import io.trino.spi.type.VarcharType; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BooleanType; +import org.apache.paimon.types.CharType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeDefaultVisitor; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.DateType; +import org.apache.paimon.types.DecimalType; +import org.apache.paimon.types.DoubleType; +import org.apache.paimon.types.FloatType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.LocalZonedTimestampType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.MultisetType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.SmallIntType; +import org.apache.paimon.types.TimeType; +import org.apache.paimon.types.TimestampType; +import org.apache.paimon.types.TinyIntType; +import org.apache.paimon.types.VarBinaryType; +import org.apache.paimon.types.VarCharType; + +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.spi.type.CharType.createCharType; + +public final class PaimonTypeUtils +{ + private PaimonTypeUtils() {} + + public static List fieldNames(RowType rowType) + { + return rowType.getFields().stream() + .map(DataField::name) + .map(String::toLowerCase) + .collect(toImmutableList()); + } + + public static Type toTrinoType(DataType type) + { + return type.accept(PaimonToTrinoTypeVistor.INSTANCE); + } + + public static DataType toPaimonType(Type trinoType) + { + return TrinoToPaimonTypeVisitor.INSTANCE.visit(trinoType); + } + + private static class PaimonToTrinoTypeVistor + extends DataTypeDefaultVisitor + { + private static final PaimonToTrinoTypeVistor INSTANCE = new PaimonToTrinoTypeVistor(); + + @Override + public Type visit(CharType charType) + { + return createCharType(Math.min(io.trino.spi.type.CharType.MAX_LENGTH, charType.getLength())); + } + + @Override + public Type visit(VarCharType varCharType) + { + if (varCharType.getLength() == VarCharType.MAX_LENGTH) { + return VarcharType.createUnboundedVarcharType(); + } + return VarcharType.createVarcharType( + varCharType.getLength()); + } + + @Override + public Type visit(BooleanType booleanType) + { + return io.trino.spi.type.BooleanType.BOOLEAN; + } + + @Override + public Type visit(BinaryType binaryType) + { + return VarbinaryType.VARBINARY; + } + + @Override + public Type visit(VarBinaryType varBinaryType) + { + return VarbinaryType.VARBINARY; + } + + @Override + public Type visit(DecimalType decimalType) + { + return io.trino.spi.type.DecimalType.createDecimalType( + decimalType.getPrecision(), decimalType.getScale()); + } + + @Override + public Type visit(TinyIntType tinyIntType) + { + return TinyintType.TINYINT; + } + + @Override + public Type visit(SmallIntType smallIntType) + { + return SmallintType.SMALLINT; + } + + @Override + public Type visit(IntType intType) + { + return IntegerType.INTEGER; + } + + @Override + public Type visit(BigIntType bigIntType) + { + return BigintType.BIGINT; + } + + @Override + public Type visit(FloatType floatType) + { + return RealType.REAL; + } + + @Override + public Type visit(DoubleType doubleType) + { + return io.trino.spi.type.DoubleType.DOUBLE; + } + + @Override + public Type visit(DateType dateType) + { + return io.trino.spi.type.DateType.DATE; + } + + @Override + public Type visit(TimeType timeType) + { + int precision = timeType.getPrecision(); + if (precision == 0) { + return io.trino.spi.type.TimeType.TIME_SECONDS; + } + if (precision <= 3) { + return io.trino.spi.type.TimeType.TIME_MILLIS; + } + if (precision <= 6) { + return io.trino.spi.type.TimeType.TIME_MICROS; + } + if (precision <= 9) { + return io.trino.spi.type.TimeType.TIME_NANOS; + } + return io.trino.spi.type.TimeType.TIME_PICOS; + } + + @Override + public Type visit(TimestampType timestampType) + { + int precision = timestampType.getPrecision(); + if (precision == 0) { + return io.trino.spi.type.TimestampType.TIMESTAMP_SECONDS; + } + if (precision <= 3) { + return io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; + } + if (precision <= 6) { + return io.trino.spi.type.TimestampType.TIMESTAMP_MICROS; + } + if (precision <= 9) { + return io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; + } + return io.trino.spi.type.TimestampType.TIMESTAMP_PICOS; + } + + @Override + public Type visit(LocalZonedTimestampType localZonedTimestampType) + { + int precision = localZonedTimestampType.getPrecision(); + if (precision <= 3) { + return TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS; + } + if (precision <= 6) { + return TimestampWithTimeZoneType.TIMESTAMP_TZ_MICROS; + } + if (precision <= 9) { + return TimestampWithTimeZoneType.TIMESTAMP_TZ_NANOS; + } + return TimestampWithTimeZoneType.TIMESTAMP_TZ_PICOS; + } + + @Override + public Type visit(ArrayType arrayType) + { + DataType elementType = arrayType.getElementType(); + return new io.trino.spi.type.ArrayType(elementType.accept(this)); + } + + @Override + public Type visit(MultisetType multisetType) + { + return new MapType(multisetType.getElementType(), new IntType()).accept(this); + } + + @Override + public Type visit(MapType mapType) + { + return new io.trino.spi.type.MapType( + mapType.getKeyType().accept(this), + mapType.getValueType().accept(this), + new TypeOperators()); + } + + @Override + public Type visit(RowType rowType) + { + List fields = + rowType.getFields().stream() + .map( + field -> + io.trino.spi.type.RowType.field( + field.name(), field.type().accept(this))) + .collect(Collectors.toList()); + return io.trino.spi.type.RowType.from(fields); + } + + @Override + protected Type defaultMethod(DataType logicalType) + { + throw new UnsupportedOperationException("Unsupported type: " + logicalType); + } + } + + private static class TrinoToPaimonTypeVisitor + { + private static final TrinoToPaimonTypeVisitor INSTANCE = new TrinoToPaimonTypeVisitor(); + + private final AtomicInteger currentIndex = new AtomicInteger(0); + + private DataType visit(Type trinoType) + { + switch (trinoType) { + case io.trino.spi.type.CharType charType -> { + return DataTypes.CHAR(Math.min(io.trino.spi.type.CharType.MAX_LENGTH, charType.getLength())); + } + case VarcharType varcharType -> { + Optional length = varcharType.getLength(); + if (length.isPresent()) { + return DataTypes.VARCHAR(Math.min(VarcharType.MAX_LENGTH, ((VarcharType) trinoType).getBoundedLength())); + } + return DataTypes.VARCHAR(VarcharType.MAX_LENGTH); + } + case io.trino.spi.type.BooleanType _ -> { + return DataTypes.BOOLEAN(); + } + case VarbinaryType _ -> { + return DataTypes.VARBINARY(Integer.MAX_VALUE); + } + case io.trino.spi.type.DecimalType decimalType -> { + return DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale()); + } + case TinyintType _ -> { + return DataTypes.TINYINT(); + } + case SmallintType _ -> { + return DataTypes.SMALLINT(); + } + case IntegerType _ -> { + return DataTypes.INT(); + } + case BigintType _ -> { + return DataTypes.BIGINT(); + } + case RealType _ -> { + return DataTypes.FLOAT(); + } + case io.trino.spi.type.DoubleType _ -> { + return DataTypes.DOUBLE(); + } + case io.trino.spi.type.DateType _ -> { + return DataTypes.DATE(); + } + case io.trino.spi.type.TimeType type -> { + if (type.getPrecision() > 9) { + throw new UnsupportedOperationException("Unsupported time type precision: " + type.getPrecision()); + } + // ERROR here, trino time type is long, but paimon time type is int + return new TimeType(type.getPrecision()); + } + case io.trino.spi.type.TimestampType timestampType -> { + int precision = timestampType.getPrecision(); + return new TimestampType(precision); + } + case TimestampWithTimeZoneType timestampType -> { + return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(timestampType.getPrecision()); + } + case io.trino.spi.type.ArrayType arrayType -> { + return DataTypes.ARRAY( + visit(arrayType.getElementType())); + } + case io.trino.spi.type.MapType mapType -> { + return DataTypes.MAP( + visit(mapType.getKeyType()), + visit(mapType.getValueType())); + } + case io.trino.spi.type.RowType rowType -> { + List dataFields = rowType.getFields().stream() + .map(field -> new DataField( + currentIndex.getAndIncrement(), + field.getName().get(), + visit(field.getType()))) + .collect(Collectors.toList()); + return new RowType(true, dataFields); + } + case null, default -> throw new UnsupportedOperationException("Unsupported type: " + trinoType); + } + } + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/AbstractPaimonTrinoCatalog.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/AbstractPaimonTrinoCatalog.java new file mode 100644 index 000000000000..40211702bbd0 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/AbstractPaimonTrinoCatalog.java @@ -0,0 +1,158 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.catalog; + +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.plugin.paimon.PaimonConfig; +import io.trino.plugin.paimon.fileio.PaimonFileIO; +import io.trino.spi.connector.ConnectorSession; +import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.FileStatus; +import org.apache.paimon.fs.Path; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Callable; + +import static java.util.Objects.requireNonNull; +import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH; + +public abstract class AbstractPaimonTrinoCatalog + implements TrinoCatalog +{ + protected final TrinoFileSystemFactory fileSystemFactory; + private final PaimonConfig config; + private final Path warehouse; + + protected AbstractPaimonTrinoCatalog( + PaimonConfig config, + TrinoFileSystemFactory fileSystemFactory) + { + requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.fileSystemFactory = fileSystemFactory; + this.config = config; + this.warehouse = new Path(config.getWarehouse()); + } + + protected static T uncheck(Callable callable) + { + try { + return callable.call(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected FileIO getFileIO(ConnectorSession session) + { + return new PaimonFileIO(fileSystemFactory.create(session), warehouse); + } + + @Override + public String warehouse() + { + return warehouse.toString(); + } + + protected List listDatabasesInFileSystem(ConnectorSession session) + { + FileIO fileIO = getFileIO(session); + return uncheck(() -> { + List databases = new ArrayList<>(); + for (FileStatus status : fileIO.listDirectories(warehouse)) { + Path path = status.getPath(); + if (status.isDir() && path.getName().endsWith(DB_SUFFIX)) { + String fileName = path.getName(); + databases.add(fileName.substring(0, fileName.length() - DB_SUFFIX.length())); + } + } + return databases; + }); + } + + protected List listTablesInFileSystem(ConnectorSession session, Path databasePath) + { + FileIO fileIO = getFileIO(session); + return uncheck(() -> { + List tables = new ArrayList<>(); + for (FileStatus status : fileIO.listDirectories(databasePath)) { + if (status.isDir() && tableExistsInFileSystem(fileIO, status.getPath())) { + tables.add(status.getPath().getName()); + } + } + return tables; + }); + } + + protected boolean tableExistsInFileSystem(FileIO fileIO, Path tablePath) + { + SchemaManager schemaManager = new SchemaManager(fileIO, tablePath); + + // in order to improve the performance, check the schema-0 firstly. + boolean schemaZeroExists = schemaManager.schemaExists(0); + if (schemaZeroExists) { + return true; + } + else { + // if schema-0 not exists, fallback to check other schemas + return !schemaManager.listAllIds().isEmpty(); + } + } + + protected Optional tableSchemaInFileSystem(FileIO fileIO, Path tablePath, String branchName) + { + Optional schema = + new SchemaManager(fileIO, tablePath, branchName).latest(); + if (!DEFAULT_MAIN_BRANCH.equals(branchName)) { + schema = + schema.map( + s -> { + Options branchOptions = new Options(s.options()); + branchOptions.set(CoreOptions.BRANCH, branchName); + return s.copy(branchOptions.toMap()); + }); + } + schema.ifPresent(s -> s.options().put(PATH.key(), tablePath.toString())); + return schema; + } + + protected Path newDatabasePathInFileSystem(String database) + { + return newDatabasePathInFileSystem(warehouse(), database); + } + + protected Path newDatabasePathInFileSystem(String warehouse, String database) + { + return new Path(warehouse, database + DB_SUFFIX); + } + + protected Path getTableLocationInFileSystem(Identifier identifier) + { + return new Path(newDatabasePathInFileSystem(identifier.getDatabaseName()), identifier.getTableName()); + } + + @Override + public PaimonConfig config() + { + return config; + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/PaimonCatalogModule.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/PaimonCatalogModule.java new file mode 100644 index 000000000000..67847bbc6466 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/PaimonCatalogModule.java @@ -0,0 +1,46 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.catalog; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.paimon.CatalogType; +import io.trino.plugin.paimon.PaimonConfig; +import io.trino.plugin.paimon.catalog.file.PaimonFileSystemCatalogModule; +import io.trino.plugin.paimon.catalog.hms.PaimonHiveCatalogModule; + +import static io.airlift.configuration.ConditionalModule.conditionalModule; +import static io.trino.plugin.paimon.CatalogType.FILESYSTEM; +import static io.trino.plugin.paimon.CatalogType.HIVE; + +public class PaimonCatalogModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + bindCatalogModule(FILESYSTEM, new PaimonFileSystemCatalogModule()); + bindCatalogModule(HIVE, new PaimonHiveCatalogModule()); + } + + private void bindCatalogModule(CatalogType catalogType, Module module) + { + // TODO: enable more types of catalog + install(conditionalModule( + PaimonConfig.class, + config -> config.getCatalogType() == catalogType, + module)); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/TrinoCatalog.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/TrinoCatalog.java new file mode 100644 index 000000000000..d70e4be0f0e7 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/TrinoCatalog.java @@ -0,0 +1,47 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.catalog; + +import io.trino.plugin.paimon.PaimonConfig; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import org.apache.paimon.table.Table; + +import java.util.List; +import java.util.Optional; + +/** + * An interface to allow different Paimon catalog implementations in PaimonMetadata. + *

+ * It mimics the Paimon catalog interface, with the following modifications: + *

    + *
  • ConnectorSession is added at the front of each method signature, similar to Iceberg's TrinoCatalog
  • + *
+ */ +public interface TrinoCatalog +{ + String DB_SUFFIX = ".db"; + + boolean databaseExists(ConnectorSession session, String database); + + List listDatabases(ConnectorSession session); + + Table loadTable(ConnectorSession session, SchemaTableName schemaTableName); + + List listTables(ConnectorSession session, Optional namespace); + + String warehouse(); + + PaimonConfig config(); +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/TrinoCatalogFactory.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/TrinoCatalogFactory.java new file mode 100644 index 000000000000..40021f82b1b9 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/TrinoCatalogFactory.java @@ -0,0 +1,21 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.catalog; + +import io.trino.spi.security.ConnectorIdentity; + +public interface TrinoCatalogFactory +{ + TrinoCatalog create(ConnectorIdentity identity); +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/PaimonFileSystemCatalogModule.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/PaimonFileSystemCatalogModule.java new file mode 100644 index 000000000000..3bfb42d6a03b --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/PaimonFileSystemCatalogModule.java @@ -0,0 +1,30 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.paimon.catalog.file; + +import com.google.inject.Binder; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.paimon.catalog.TrinoCatalogFactory; + +public class PaimonFileSystemCatalogModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + binder.bind(TrinoCatalogFactory.class).to(TrinoFileSystemCatalogFactory.class).in(Scopes.SINGLETON); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/TrinoFileSystemCatalogFactory.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/TrinoFileSystemCatalogFactory.java new file mode 100644 index 000000000000..4706e8d2cad0 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/TrinoFileSystemCatalogFactory.java @@ -0,0 +1,48 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.paimon.catalog.file; + +import com.google.inject.Inject; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.plugin.paimon.PaimonConfig; +import io.trino.plugin.paimon.catalog.TrinoCatalog; +import io.trino.plugin.paimon.catalog.TrinoCatalogFactory; +import io.trino.spi.security.ConnectorIdentity; + +import static java.util.Objects.requireNonNull; + +public class TrinoFileSystemCatalogFactory + implements TrinoCatalogFactory +{ + private final PaimonConfig paimonConfig; + private final TrinoFileSystemFactory fileSystemFactory; + + @Inject + public TrinoFileSystemCatalogFactory( + PaimonConfig config, + TrinoFileSystemFactory fileSystemFactory) + { + this.paimonConfig = requireNonNull(config, "config is null"); + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + } + + @Override + public TrinoCatalog create(ConnectorIdentity identity) + { + return new TrinoFileSystemPaimonCatalog( + paimonConfig, + fileSystemFactory); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/TrinoFileSystemPaimonCatalog.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/TrinoFileSystemPaimonCatalog.java new file mode 100644 index 000000000000..b13935f2828f --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/file/TrinoFileSystemPaimonCatalog.java @@ -0,0 +1,92 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.paimon.catalog.file; + +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.plugin.paimon.PaimonConfig; +import io.trino.plugin.paimon.catalog.AbstractPaimonTrinoCatalog; +import io.trino.plugin.paimon.fileio.PaimonFileIO; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.Table; + +import java.util.List; +import java.util.Optional; + +import static org.apache.paimon.CoreOptions.PATH; + +public class TrinoFileSystemPaimonCatalog + extends AbstractPaimonTrinoCatalog +{ + public TrinoFileSystemPaimonCatalog( + PaimonConfig config, + TrinoFileSystemFactory fileSystemFactory) + { + super(config, fileSystemFactory); + } + + @Override + public boolean databaseExists(ConnectorSession session, String database) + { + FileIO fileIO = getFileIO(session); + return uncheck(() -> fileIO.exists(newDatabasePathInFileSystem(database))); + } + + @Override + public List listDatabases(ConnectorSession session) + { + return listDatabasesInFileSystem(session); + } + + @Override + public List listTables(ConnectorSession session, Optional namespace) + { + if (namespace.isPresent()) { + return listTablesInFileSystem(session, newDatabasePathInFileSystem(namespace.get())); + } + else { + return listDatabases(session).stream() + .flatMap(db -> listTablesInFileSystem(session, newDatabasePathInFileSystem(db)).stream()) + .toList(); + } + } + + @Override + public Table loadTable(ConnectorSession session, SchemaTableName schemaTableName) + { + Identifier identifier = Identifier.create(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + Path tablePath = new Path(newDatabasePathInFileSystem(identifier.getDatabaseName()), identifier.getTableName()); + PaimonFileIO fileIO = new PaimonFileIO(fileSystemFactory.create(session), tablePath); + TableSchema schema = loadTableSchema(fileIO, identifier).orElseThrow(() -> new TableNotFoundException(schemaTableName)); + Path path = new Path(schema.options().get(PATH.key())); + + return FileStoreTableFactory.create(fileIO, path, schema); + } + + public Optional loadTableSchema(FileIO fileIO, Identifier identifier) + { + Path tablePath = new Path(newDatabasePathInFileSystem(identifier.getDatabaseName()), identifier.getObjectName()); + Optional schema = new SchemaManager(fileIO, tablePath).latest(); + schema.ifPresent(s -> s.options().put(PATH.key(), tablePath.toString())); + return schema; + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/PaimonHiveCatalogModule.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/PaimonHiveCatalogModule.java new file mode 100644 index 000000000000..a4e8132e9c5a --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/PaimonHiveCatalogModule.java @@ -0,0 +1,35 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.paimon.catalog.hms; + +import com.google.inject.Binder; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.hive.metastore.HiveMetastoreModule; +import io.trino.plugin.paimon.catalog.TrinoCatalogFactory; + +import java.util.Optional; + +public class PaimonHiveCatalogModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + binder.bind(TrinoCatalogFactory.class).to(TrinoHiveCatalogFactory.class).in(Scopes.SINGLETON); + + install(new HiveMetastoreModule(Optional.empty(), false)); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/TrinoHiveCatalogFactory.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/TrinoHiveCatalogFactory.java new file mode 100644 index 000000000000..a30cc48556ad --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/TrinoHiveCatalogFactory.java @@ -0,0 +1,59 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.paimon.catalog.hms; + +import com.google.inject.Inject; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.cache.CachingHiveMetastore; +import io.trino.plugin.paimon.PaimonConfig; +import io.trino.plugin.paimon.catalog.TrinoCatalog; +import io.trino.plugin.paimon.catalog.TrinoCatalogFactory; +import io.trino.spi.security.ConnectorIdentity; + +import java.util.Optional; + +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static java.util.Objects.requireNonNull; + +public class TrinoHiveCatalogFactory + implements TrinoCatalogFactory +{ + private final PaimonConfig paimonConfig; + private final HiveMetastoreFactory metastoreFactory; + private final TrinoFileSystemFactory fileSystemFactory; + + @Inject + public TrinoHiveCatalogFactory( + PaimonConfig config, + HiveMetastoreFactory metastoreFactory, + TrinoFileSystemFactory fileSystemFactory) + { + this.paimonConfig = requireNonNull(config, "config is null"); + this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null"); + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + } + + @Override + public TrinoCatalog create(ConnectorIdentity identity) + { + CachingHiveMetastore metastore = createPerTransactionCache(metastoreFactory.createMetastore(Optional.of(identity)), 1000); + + return new TrinoHivePaimonCatalog( + paimonConfig, + metastore, + fileSystemFactory); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/TrinoHivePaimonCatalog.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/TrinoHivePaimonCatalog.java new file mode 100644 index 000000000000..a7df9e72bd18 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/catalog/hms/TrinoHivePaimonCatalog.java @@ -0,0 +1,166 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.catalog.hms; + +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.Database; +import io.trino.metastore.cache.CachingHiveMetastore; +import io.trino.plugin.hive.TableType; +import io.trino.plugin.paimon.PaimonConfig; +import io.trino.plugin.paimon.catalog.AbstractPaimonTrinoCatalog; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaNotFoundException; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.TableMetadata; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.CatalogEnvironment; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.Table; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static java.util.Objects.requireNonNull; +import static org.apache.paimon.CoreOptions.PATH; + +public class TrinoHivePaimonCatalog + extends AbstractPaimonTrinoCatalog +{ + private static final String INPUT_FORMAT_CLASS_NAME = + "org.apache.paimon.hive.mapred.PaimonInputFormat"; + private static final String OUTPUT_FORMAT_CLASS_NAME = + "org.apache.paimon.hive.mapred.PaimonOutputFormat"; + + private final CachingHiveMetastore metastore; + + public TrinoHivePaimonCatalog( + PaimonConfig config, + CachingHiveMetastore metastore, + TrinoFileSystemFactory fileSystemFactory) + { + super( + config, + fileSystemFactory); + + this.metastore = requireNonNull(metastore, "metastore is null"); + } + + private static boolean isPaimonTable(io.trino.metastore.Table table) + { + return INPUT_FORMAT_CLASS_NAME.equals(table.getStorage().getStorageFormat().getInputFormat()) + && OUTPUT_FORMAT_CLASS_NAME.equals(table.getStorage().getStorageFormat().getOutputFormat()); + } + + @Override + public boolean databaseExists(ConnectorSession session, String database) + { + return metastore.getDatabase(database).isPresent(); + } + + @Override + public List listDatabases(ConnectorSession session) + { + return metastore.getAllDatabases(); + } + + @Override + public Table loadTable(ConnectorSession session, SchemaTableName schemaTableName) + { + Identifier identifier = Identifier.create(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + FileIO fileIO = getFileIO(session); + TableMetadata tableMetadata = loadTableMetadata(fileIO, identifier); + Path path = new Path(tableMetadata.schema().options().get(PATH.key())); + CatalogEnvironment catalogEnv = + new CatalogEnvironment( + identifier, + tableMetadata.uuid(), + null, + null, + null, + false); + return FileStoreTableFactory.create(fileIO, path, tableMetadata.schema(), catalogEnv); + } + + @Override + public List listTables(ConnectorSession session, Optional namespace) + { + if (namespace.isPresent()) { + String databaseName = namespace.get(); + if (!databaseExists(session, databaseName)) { + throw new SchemaNotFoundException("Database does not exist: " + databaseName); + } + return metastore.getTables(databaseName).stream().map(t -> t.tableName().getTableName()).collect(Collectors.toList()); + } + + return metastore.getAllDatabases().stream().flatMap(name -> metastore.getTables(name).stream()) + .map(t -> t.tableName().getTableName()) + .collect(Collectors.toList()); + } + + protected TableMetadata loadTableMetadata(FileIO fileIO, Identifier identifier) + { + return loadTableMetadata(fileIO, identifier, getHmsTable(identifier)); + } + + public io.trino.metastore.Table getHmsTable(Identifier identifier) + { + return metastore.getTable(identifier.getDatabaseName(), identifier.getTableName()).orElseThrow(() -> new TableNotFoundException(SchemaTableName.schemaTableName(identifier.getDatabaseName(), identifier.getTableName()))); + } + + private TableMetadata loadTableMetadata(FileIO fileIO, Identifier identifier, io.trino.metastore.Table table) + { + return new TableMetadata( + loadTableSchema(fileIO, identifier, table), + isExternalTable(table), + identifier.getFullName() + "." + table.getParameters().get("transient_lastDdlTime")); + } + + private TableSchema loadTableSchema(FileIO fileIO, Identifier identifier, io.trino.metastore.Table table) + { + if (isPaimonTable(table)) { + return tableSchemaInFileSystem( + fileIO, + getTableLocation(identifier, table), + identifier.getBranchNameOrDefault()) + .orElseThrow(() -> new TableNotFoundException(SchemaTableName.schemaTableName(identifier.getDatabaseName(), identifier.getTableName()))); + } + + throw new TableNotFoundException(SchemaTableName.schemaTableName(identifier.getDatabaseName(), identifier.getTableName())); + } + + private boolean isExternalTable(io.trino.metastore.Table table) + { + return table != null && TableType.EXTERNAL_TABLE.name().equals(table.getTableType()); + } + + private Path getTableLocation(Identifier identifier, @Nullable io.trino.metastore.Table table) + { + Optional tableLocationString = table.getStorage().getOptionalLocation(); + if (tableLocationString.isPresent()) { + return new Path(tableLocationString.get()); + } + String databaseName = identifier.getDatabaseName(); + String tableName = identifier.getTableName(); + Database database = metastore.getDatabase(databaseName).orElseThrow(() -> new SchemaNotFoundException("Database does not exist: " + databaseName)); + Optional tableLocationByDatabase = database.getLocation().map(databaseLocation -> new Path(databaseLocation, tableName)); + return tableLocationByDatabase.orElse(super.getTableLocationInFileSystem(identifier)); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonDirectoryFileStatus.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonDirectoryFileStatus.java new file mode 100644 index 000000000000..4a173abfbd88 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonDirectoryFileStatus.java @@ -0,0 +1,56 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.fileio; + +import org.apache.paimon.fs.FileStatus; +import org.apache.paimon.fs.Path; + +import static java.util.Objects.requireNonNull; + +public class PaimonDirectoryFileStatus + implements FileStatus +{ + private final Path path; + + public PaimonDirectoryFileStatus(Path path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public long getLen() + { + // can't get len by trino file system + return -1; + } + + @Override + public boolean isDir() + { + return true; + } + + @Override + public Path getPath() + { + return path; + } + + @Override + public long getModificationTime() + { + // can't get modification time by trino file system + return -1; + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonFileIO.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonFileIO.java new file mode 100644 index 000000000000..1fceb7c7f353 --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonFileIO.java @@ -0,0 +1,251 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.fileio; + +import io.trino.filesystem.FileEntry; +import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoInputFile; +import io.trino.filesystem.TrinoInputStream; +import io.trino.filesystem.TrinoOutputFile; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.FileStatus; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.PositionOutputStream; +import org.apache.paimon.fs.SeekableInputStream; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.FileAlreadyExistsException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.locks.ReentrantLock; + +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class PaimonFileIO + implements FileIO +{ + private static final ReentrantLock RENAME_LOCK = new ReentrantLock(); + + private final boolean objectStore; + + private final TrinoFileSystem fileSystem; + + public PaimonFileIO( + TrinoFileSystem fileSystem, + @Nullable Path path) + { + this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); + this.objectStore = path == null || checkObjectStore(path.toUri().getScheme()); + } + + private static boolean checkObjectStore(String scheme) + { + scheme = scheme.toLowerCase(ENGLISH); + if (!scheme.startsWith("s3") + && !scheme.startsWith("emr") + && !scheme.startsWith("oss") + && !scheme.startsWith("wasb")) { + return scheme.startsWith("http") || scheme.startsWith("ftp"); + } + return true; + } + + @Override + public boolean isObjectStore() + { + return objectStore; + } + + @Override + public void configure(CatalogContext catalogContext) {} + + @Override + public SeekableInputStream newInputStream(Path path) + throws IOException + { + return new PaimonInputStreamWrapper(fileSystem.newInputFile(Location.of(path.toString())).newStream()); + } + + @Override + public PositionOutputStream newOutputStream(Path path, boolean overwrite) + throws IOException + { + TrinoOutputFile outputFile = fileSystem.newOutputFile(Location.of(path.toString())); + try { + return new PositionOutputStreamWrapper(outputFile.create(), 0); + } + catch (FileAlreadyExistsException e) { + if (overwrite) { + fileSystem.deleteFile(Location.of(path.toString())); + return new PositionOutputStreamWrapper(outputFile.create(), 0); + } + throw e; + } + } + + @Override + public FileStatus getFileStatus(Path path) + throws IOException + { + return status(path); + } + + private FileStatus status(Path path) + throws IOException + { + if (fileSystem.directoryExists(Location.of(path.toString())).orElse(false)) { + return new PaimonDirectoryFileStatus(path); + } + TrinoInputFile trinoInputFile = fileSystem.newInputFile(Location.of(path.toString())); + return new PaimonFileStatus(trinoInputFile.length(), path, trinoInputFile.lastModified().getEpochSecond()); + } + + @Override + public FileStatus[] listStatus(Path path) + throws IOException + { + List fileStatusList = new ArrayList<>(); + Location location = Location.of(path.toString()); + if (fileSystem.directoryExists(location).orElse(false)) { + FileIterator fileIterator = fileSystem.listFiles(location); + while (fileIterator.hasNext()) { + FileEntry fileEntry = fileIterator.next(); + fileStatusList.add( + new PaimonFileStatus( + fileEntry.length(), + new Path(fileEntry.location().toString()), + fileEntry.lastModified().getEpochSecond())); + } + fileSystem.listDirectories(Location.of(path.toString())) + .forEach(directory -> fileStatusList.add(new PaimonDirectoryFileStatus(new Path(directory.toString())))); + } + return fileStatusList.toArray(new FileStatus[0]); + } + + @Override + public FileStatus[] listDirectories(Path path) + throws IOException + { + return fileSystem.listDirectories(Location.of(path.toString())).stream() + .map(location -> new PaimonDirectoryFileStatus(new Path(location.toString()))) + .toArray(FileStatus[]::new); + } + + @Override + public boolean exists(Path path) + throws IOException + { + return fileSystem.directoryExists(Location.of(path.toString())).orElse(false) + || existFile(Location.of(path.toString())); + } + + private boolean existFile(Location location) + throws IOException + { + try { + return fileSystem.newInputFile(location).exists(); + } + catch (IllegalArgumentException e) { + return false; + } + } + + @Override + public boolean delete(Path path, boolean recursive) + throws IOException + { + Location location = Location.of(path.toString()); + if (fileSystem.directoryExists(location).orElse(false)) { + if (!recursive) { + if (fileSystem.listFiles(location).hasNext()) { + throw new IOException("Directory " + location + " is not empty"); + } + } + fileSystem.deleteDirectory(location); + return true; + } + else if (existFile(location)) { + fileSystem.deleteFile(location); + return true; + } + + return false; + } + + @Override + public boolean mkdirs(Path path) + throws IOException + { + fileSystem.createDirectory(Location.of(path.toString())); + return true; + } + + @Override + public boolean rename(Path source, Path target) + throws IOException + { + boolean local = "file".equals(source.toUri().getScheme()); + try { + if (local) { + RENAME_LOCK.lock(); + } + Location sourceLocation = Location.of(source.toString()); + Location targetLocation = Location.of(target.toString()); + if (fileSystem.directoryExists(sourceLocation).orElse(false)) { + fileSystem.renameDirectory(sourceLocation, targetLocation); + } + else { + renameFileAnyway(sourceLocation, targetLocation); + } + } + catch (IOException e) { + if (e.getMessage().contains("Target location already exists") || e.getMessage().contains("rename failed")) { + return false; + } + throw e; + } + finally { + if (local) { + RENAME_LOCK.unlock(); + } + } + return true; + } + + private void renameFileAnyway(Location source, Location target) + throws IOException + { + try { + fileSystem.renameFile(source, target); + } + catch (IOException e) { + try (TrinoInputStream input = fileSystem.newInputFile(source).newStream(); + OutputStream outputStream = fileSystem.newOutputFile(target).create()) { + byte[] btyes = new byte[1024 * 8]; + int len; + while ((len = input.read(btyes)) != -1) { + outputStream.write(btyes, 0, len); + } + } + fileSystem.deleteFile(source); + } + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonFileStatus.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonFileStatus.java new file mode 100644 index 000000000000..91c70fbf78fc --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonFileStatus.java @@ -0,0 +1,58 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.fileio; + +import org.apache.paimon.fs.FileStatus; +import org.apache.paimon.fs.Path; + +import static java.util.Objects.requireNonNull; + +public class PaimonFileStatus + implements FileStatus +{ + private final long len; + private final Path path; + private final long modificationTime; + + public PaimonFileStatus(long length, Path path, long modificationTime) + { + this.len = length; + this.path = requireNonNull(path, "path is null"); + this.modificationTime = modificationTime; + } + + @Override + public long getLen() + { + return len; + } + + @Override + public boolean isDir() + { + return false; + } + + @Override + public Path getPath() + { + return path; + } + + @Override + public long getModificationTime() + { + return modificationTime; + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonInputStreamWrapper.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonInputStreamWrapper.java new file mode 100644 index 000000000000..8c3a2e5da08a --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PaimonInputStreamWrapper.java @@ -0,0 +1,143 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.fileio; + +import io.trino.filesystem.TrinoInputStream; +import org.apache.paimon.fs.SeekableInputStream; + +import java.io.IOException; +import java.io.OutputStream; + +import static java.util.Objects.requireNonNull; + +public class PaimonInputStreamWrapper + extends SeekableInputStream +{ + private final TrinoInputStream inputStream; + + public PaimonInputStreamWrapper(TrinoInputStream inputStream) + { + this.inputStream = requireNonNull(inputStream, "inputStream is null"); + } + + @Override + public void seek(long l) + throws IOException + { + inputStream.seek(l); + } + + @Override + public long getPos() + throws IOException + { + return inputStream.getPosition(); + } + + @Override + public int read() + throws IOException + { + return inputStream.read(); + } + + @Override + public int read(byte[] bytes, int off, int len) + throws IOException + { + return inputStream.read(bytes, off, len); + } + + @Override + public void close() + throws IOException + { + inputStream.close(); + } + + @Override + public byte[] readNBytes(int len) + throws IOException + { + return inputStream.readNBytes(len); + } + + @Override + public int readNBytes(byte[] b, int off, int len) + throws IOException + { + return inputStream.readNBytes(b, off, len); + } + + @Override + public int read(byte[] b) + throws IOException + { + return inputStream.read(b); + } + + @Override + public long skip(long n) + throws IOException + { + return inputStream.skip(n); + } + + @Override + public void skipNBytes(long n) + throws IOException + { + inputStream.skipNBytes(n); + } + + @Override + public byte[] readAllBytes() + throws IOException + { + return inputStream.readAllBytes(); + } + + @Override + public int available() + throws IOException + { + return inputStream.available(); + } + + @Override + public void mark(int readlimit) + { + inputStream.mark(readlimit); + } + + @Override + public void reset() + throws IOException + { + inputStream.reset(); + } + + @Override + public boolean markSupported() + { + return inputStream.markSupported(); + } + + @Override + public long transferTo(OutputStream out) + throws IOException + { + return inputStream.transferTo(out); + } +} diff --git a/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PositionOutputStreamWrapper.java b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PositionOutputStreamWrapper.java new file mode 100644 index 000000000000..5ad951ebb65f --- /dev/null +++ b/plugin/trino-paimon/src/main/java/io/trino/plugin/paimon/fileio/PositionOutputStreamWrapper.java @@ -0,0 +1,79 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.fileio; + +import org.apache.paimon.fs.PositionOutputStream; + +import java.io.IOException; +import java.io.OutputStream; + +import static java.util.Objects.requireNonNull; + +public class PositionOutputStreamWrapper + extends PositionOutputStream +{ + private final OutputStream outputStream; + + private long position; + + public PositionOutputStreamWrapper(OutputStream outputStream, long startPosition) + { + this.outputStream = requireNonNull(outputStream, "outputStream is null"); + this.position = startPosition; + } + + @Override + public long getPos() + { + return position; + } + + @Override + public void write(int b) + throws IOException + { + position++; + outputStream.write(b); + } + + @Override + public void write(byte[] bytes) + throws IOException + { + position += bytes.length; + outputStream.write(bytes); + } + + @Override + public void write(byte[] bytes, int off, int len) + throws IOException + { + position += len; + outputStream.write(bytes, off, len); + } + + @Override + public void flush() + throws IOException + { + outputStream.flush(); + } + + @Override + public void close() + throws IOException + { + outputStream.close(); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/PaimonQueryRunner.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/PaimonQueryRunner.java new file mode 100644 index 000000000000..ad8df693a4d8 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/PaimonQueryRunner.java @@ -0,0 +1,134 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.errorprone.annotations.CanIgnoreReturnValue; +import io.airlift.log.Logger; +import io.trino.Session; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; +import io.trino.plugin.paimon.testing.PaimonTablesInitializer; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.tpch.TpchTable; + +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; + +import static io.airlift.testing.Closeables.closeAllSuppress; +import static io.trino.testing.TestingSession.testSessionBuilder; +import static io.trino.testing.containers.Minio.MINIO_REGION; +import static io.trino.testing.containers.Minio.MINIO_ROOT_PASSWORD; +import static io.trino.testing.containers.Minio.MINIO_ROOT_USER; + +final class PaimonQueryRunner +{ + private static final String PAIMON_CATALOG = "paimon"; + private static final String SCHEMA_NAME = "tpch"; + + private PaimonQueryRunner() {} + + public static Builder builder() + { + return new Builder(); + } + + public static Builder builder(Hive3MinioDataLake hiveMinioDataLake) + { + return new Builder() + .setWarehouse("s3://" + hiveMinioDataLake.getBucketName() + "/") + .addConnectorProperty("paimon.catalog.type", "hive") + .addConnectorProperty("hive.metastore.uri", hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint().toString()) + .addConnectorProperty("fs.native-s3.enabled", "true") + .addConnectorProperty("s3.aws-access-key", MINIO_ROOT_USER) + .addConnectorProperty("s3.aws-secret-key", MINIO_ROOT_PASSWORD) + .addConnectorProperty("s3.region", MINIO_REGION) + .addConnectorProperty("s3.endpoint", hiveMinioDataLake.getMinio().getMinioAddress()) + .addConnectorProperty("s3.path-style-access", "true"); + } + + public static class Builder + extends DistributedQueryRunner.Builder + { + private final Map connectorProperties = new HashMap<>(); + private String warehouse; + private PaimonTablesInitializer dataLoader; + + protected Builder() + { + super(testSessionBuilder() + .setCatalog("paimon") + .setSchema(SCHEMA_NAME) + .build()); + } + + @CanIgnoreReturnValue + public Builder setWarehouse(String warehouse) + { + this.warehouse = warehouse; + return this; + } + + @CanIgnoreReturnValue + public Builder setDataLoader(PaimonTablesInitializer dataLoader) + { + this.dataLoader = dataLoader; + return this; + } + + @CanIgnoreReturnValue + public Builder addConnectorProperty(String key, String value) + { + this.connectorProperties.put(key, value); + return this; + } + + @Override + public DistributedQueryRunner build() + throws Exception + { + Session session = testSessionBuilder().setCatalog(PAIMON_CATALOG).setSchema(SCHEMA_NAME).build(); + DistributedQueryRunner queryRunner = super.build(); + try { + queryRunner.installPlugin(new TestingPaimonPlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("/"))); + if (warehouse == null) { + Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("paimon_data"); + warehouse = dataDir.toFile().toURI().toString(); + } + connectorProperties.put("paimon.warehouse", warehouse); + queryRunner.createCatalog(PAIMON_CATALOG, PAIMON_CATALOG, connectorProperties); + + dataLoader.initializeTables(session, queryRunner, SCHEMA_NAME); + return queryRunner; + } + catch (Throwable e) { + closeAllSuppress(e, queryRunner); + throw e; + } + } + } + + public static void main(String[] args) + throws Exception + { + //noinspection resource + QueryRunner queryRunner = builder() + .addCoordinatorProperty("http-server.http.port", "8080") + .setDataLoader(new PaimonTablesInitializer(TpchTable.getTables())) + .build(); + Logger log = Logger.get(PaimonQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/PaimonTestUtils.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/PaimonTestUtils.java new file mode 100644 index 000000000000..49d701185001 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/PaimonTestUtils.java @@ -0,0 +1,74 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.fs.Path; +import org.apache.paimon.options.Options; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.CharType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VarCharType; +import org.apache.paimon.utils.InstantiationUtil; + +import java.nio.file.Files; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +import static org.apache.paimon.data.BinaryString.fromString; + +final class PaimonTestUtils +{ + private PaimonTestUtils() {} + + public static byte[] getSerializedTable() + throws Exception + { + String warehouse = Files.createTempDirectory(UUID.randomUUID().toString()).toUri().toString(); + Path tablePath = new Path(warehouse, "test.db/user"); + SimpleTableTestHelper testHelper = createTestHelper(tablePath); + testHelper.write(GenericRow.of(1, 2L, fromString("1"), fromString("1"))); + testHelper.write(GenericRow.of(3, 4L, fromString("2"), fromString("2"))); + testHelper.write(GenericRow.of(5, 6L, fromString("3"), fromString("3"))); + testHelper.write(GenericRow.ofKind(RowKind.DELETE, 3, 4L, fromString("2"), fromString("2"))); + testHelper.commit(); + Map config = new HashMap<>(); + config.put("warehouse", warehouse); + Catalog catalog = CatalogFactory.createCatalog(CatalogContext.create(Options.fromMap(config))); + Identifier tablePath2 = new Identifier("test", "user"); + return InstantiationUtil.serializeObject(catalog.getTable(tablePath2)); + } + + private static SimpleTableTestHelper createTestHelper(Path tablePath) + throws Exception + { + RowType rowType = new RowType( + Arrays.asList( + new DataField(0, "a", new IntType()), + new DataField(1, "b", new BigIntType()), + // test field name has upper case + new DataField(2, "aCa", new VarCharType()), + new DataField(3, "d", new CharType(1)))); + return new SimpleTableTestHelper(tablePath, rowType); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/SimpleTableTestHelper.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/SimpleTableTestHelper.java new file mode 100644 index 000000000000..694c65c356d5 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/SimpleTableTestHelper.java @@ -0,0 +1,63 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.sink.InnerTableCommit; +import org.apache.paimon.table.sink.InnerTableWrite; +import org.apache.paimon.types.RowType; + +final class SimpleTableTestHelper +{ + private static final String USER = "user"; + + private final InnerTableWrite writer; + private final InnerTableCommit commit; + private final FileStoreTable table; + + public SimpleTableTestHelper(Path path, RowType rowType) + throws Exception + { + Schema schema = new Schema(rowType.getFields(), ImmutableList.of(), ImmutableList.of("a"), ImmutableMap.of("bucket", "1"), ""); + new SchemaManager(LocalFileIO.create(), path).createTable(schema); + table = FileStoreTableFactory.create(LocalFileIO.create(), path); + this.writer = table.newWrite(USER); + this.commit = table.newCommit(USER); + } + + public void write(InternalRow row) + throws Exception + { + writer.write(row); + } + + public void createTag(String name) + { + table.createTag(name); + } + + public void commit() + throws Exception + { + commit.commit(0, writer.prepareCommit(true, 0)); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonConfig.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonConfig.java new file mode 100644 index 000000000000..f6051e96e498 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonConfig.java @@ -0,0 +1,19 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +final class TestPaimonConfig +{ + // TODO Add tests +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonConnectorTest.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonConnectorTest.java new file mode 100644 index 000000000000..736ff39ef990 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonConnectorTest.java @@ -0,0 +1,60 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.plugin.paimon.testing.PaimonTablesInitializer; +import io.trino.testing.BaseConnectorTest; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; + +final class TestPaimonConnectorTest + extends BaseConnectorTest +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return PaimonQueryRunner.builder() + .setDataLoader(new PaimonTablesInitializer(REQUIRED_TPCH_TABLES)) + .build(); + } + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + return switch (connectorBehavior) { + case SUPPORTS_ADD_COLUMN, + SUPPORTS_ARRAY, + SUPPORTS_COMMENT_ON_COLUMN, + SUPPORTS_COMMENT_ON_TABLE, + SUPPORTS_CREATE_MATERIALIZED_VIEW, + SUPPORTS_CREATE_SCHEMA, + SUPPORTS_CREATE_TABLE, + SUPPORTS_CREATE_VIEW, + SUPPORTS_LIMIT_PUSHDOWN, + SUPPORTS_DELETE, + SUPPORTS_DROP_NOT_NULL_CONSTRAINT, + SUPPORTS_INSERT, + SUPPORTS_MAP_TYPE, + SUPPORTS_MERGE, + SUPPORTS_RENAME_COLUMN, + SUPPORTS_RENAME_SCHEMA, + SUPPORTS_RENAME_TABLE, + SUPPORTS_ROW_TYPE, + SUPPORTS_TOPN_PUSHDOWN, + SUPPORTS_UPDATE -> false; + default -> super.hasBehavior(connectorBehavior); + }; + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonITCase.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonITCase.java new file mode 100644 index 000000000000..47c63bf20c66 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonITCase.java @@ -0,0 +1,686 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.MaterializedResult; +import io.trino.testing.QueryRunner; +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericMap; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.disk.IOManagerImpl; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.sink.InnerTableCommit; +import org.apache.paimon.table.sink.InnerTableWrite; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.CharType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VarCharType; +import org.junit.jupiter.api.Test; + +import java.nio.file.Files; +import java.time.Instant; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static io.airlift.testing.Closeables.closeAllSuppress; +import static io.trino.testing.TestingSession.testSessionBuilder; +import static java.time.ZoneOffset.UTC; +import static org.apache.paimon.data.BinaryString.fromString; +import static org.assertj.core.api.Assertions.assertThat; + +// TODO Merge into TestPaimonConnectorTest +final class TestPaimonITCase + extends AbstractTestQueryFramework +{ + private static final String CATALOG = "paimon"; + private static final String DB = "default"; + + private static SimpleTableTestHelper createTestHelper(Path tablePath) + throws Exception + { + RowType rowType = new RowType( + Arrays.asList( + new DataField(0, "a", new IntType()), + new DataField(1, "b", new BigIntType()), + // test field name has upper case + new DataField(2, "aCa", new VarCharType()), + new DataField(3, "d", new CharType(1)))); + return new SimpleTableTestHelper(tablePath, rowType); + } + + private static String timestampLiteral(long epochMilliSeconds, int precision) + { + return DateTimeFormatter.ofPattern("''yyyy-MM-dd HH:mm:ss." + "S".repeat(precision) + " VV''") + .format(Instant.ofEpochMilli(epochMilliSeconds).atZone(UTC)); + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + String warehouse = Files.createTempDirectory(UUID.randomUUID().toString()).toUri().toString(); + + Path tablePath1 = new Path(warehouse, DB + ".db/t1"); + SimpleTableTestHelper testHelper1 = createTestHelper(tablePath1); + testHelper1.write(GenericRow.of(1, 2L, fromString("1"), fromString("1"))); + testHelper1.write(GenericRow.of(3, 4L, fromString("2"), fromString("2"))); + testHelper1.write(GenericRow.of(5, 6L, fromString("3"), fromString("3"))); + testHelper1.write(GenericRow.ofKind(RowKind.DELETE, 3, 4L, fromString("2"), fromString("2"))); + testHelper1.commit(); + + Path tablePath2 = new Path(warehouse, "default.db/t2"); + SimpleTableTestHelper testHelper2 = createTestHelper(tablePath2); + testHelper2.write(GenericRow.of(1, 2L, fromString("1"), fromString("1"))); + testHelper2.write(GenericRow.of(3, 4L, fromString("2"), fromString("2"))); + testHelper2.commit(); + testHelper2.createTag("1"); + testHelper2.write(GenericRow.of(5, 6L, fromString("3"), fromString("3"))); + testHelper2.write(GenericRow.of(7, 8L, fromString("4"), fromString("4"))); + testHelper2.commit(); + testHelper2.createTag("tag-2"); + + { + Path tablePath3 = new Path(warehouse, "default.db/t3"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "pt", DataTypes.STRING()), + new DataField(1, "a", new IntType()), + new DataField(2, "b", new BigIntType()), + new DataField(3, "c", new BigIntType()), + new DataField(4, "d", new IntType()))); + new SchemaManager(LocalFileIO.create(), tablePath3) + .createTable( + new Schema( + rowType.getFields(), + Collections.singletonList("pt"), + Collections.emptyList(), + new HashMap<>(), + "")); + FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath3); + InnerTableWrite writer = table.newWrite("user"); + InnerTableCommit commit = table.newCommit("user"); + writer.write(GenericRow.of(fromString("1"), 1, 1L, 1L, 1)); + writer.write(GenericRow.of(fromString("1"), 1, 2L, 2L, 2)); + writer.write(GenericRow.of(fromString("2"), 3, 3L, 3L, 3)); + commit.commit(0, writer.prepareCommit(true, 0)); + } + + { + Path tablePath = new Path(warehouse, "default.db/empty_t"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(1, "a", new IntType()), + new DataField(2, "b", new BigIntType()))); + new SchemaManager(LocalFileIO.create(), tablePath) + .createTable( + new Schema( + rowType.getFields(), + Collections.emptyList(), + Collections.emptyList(), + new HashMap<>(), + "")); + } + + { + Path tablePath4 = new Path(warehouse, "default.db/t4"); + List innerRowFields = new ArrayList<>(); + innerRowFields.add(new DataField(4, "innercol1", new IntType())); + innerRowFields.add( + new DataField(5, "innercol2", new VarCharType(VarCharType.MAX_LENGTH))); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "i", new IntType()), + new DataField( + 1, + "map", + new MapType( + new VarCharType(VarCharType.MAX_LENGTH), + new VarCharType(VarCharType.MAX_LENGTH))), + new DataField(2, "innerrow", new RowType(true, innerRowFields)), + new DataField(3, "array", new ArrayType(new IntType())))); + new SchemaManager(LocalFileIO.create(), tablePath4) + .createTable( + new Schema( + rowType.getFields(), + Collections.emptyList(), + Collections.singletonList("i"), + Collections.singletonMap("bucket", "1"), + "")); + FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath4); + InnerTableWrite writer = table.newWrite("user"); + InnerTableCommit commit = table.newCommit("user"); + writer.write( + GenericRow.of( + 1, + new GenericMap( + new HashMap<>(ImmutableMap.of(fromString("1"), fromString("2")))), + GenericRow.of(2, fromString("male")), + new GenericArray(new int[] {1, 2, 3}))); + commit.commit(0, writer.prepareCommit(true, 0)); + } + + { + Path tablePath6 = new Path(warehouse, "default.db/t99"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "boolean", DataTypes.BOOLEAN()), + new DataField(1, "tinyint", DataTypes.TINYINT()), + new DataField(2, "smallint", DataTypes.SMALLINT()), + new DataField(3, "int", DataTypes.INT()), + new DataField(4, "bigint", DataTypes.BIGINT()), + new DataField(5, "float", DataTypes.FLOAT()), + new DataField(6, "double", DataTypes.DOUBLE()), + new DataField(7, "char", DataTypes.CHAR(5)), + new DataField(8, "varchar", DataTypes.VARCHAR(100)), + new DataField(9, "date", DataTypes.DATE()), + new DataField(10, "timestamp_0", DataTypes.TIMESTAMP(0)), + new DataField(11, "timestamp_3", DataTypes.TIMESTAMP(3)), + new DataField(12, "timestamp_6", DataTypes.TIMESTAMP(6)), + new DataField( + 13, + "timestamp_tz", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), + new DataField(14, "decimal", DataTypes.DECIMAL(10, 5)), + new DataField(15, "varbinary", DataTypes.VARBINARY(10)), + new DataField(16, "array", DataTypes.ARRAY(DataTypes.INT())), + new DataField( + 17, + "map", + DataTypes.MAP(DataTypes.INT(), DataTypes.INT())), + new DataField( + 18, + "row", + DataTypes.ROW( + DataTypes.FIELD(100, "q1", DataTypes.INT()), + DataTypes.FIELD(101, "q2", DataTypes.INT()))))); + new SchemaManager(LocalFileIO.create(), tablePath6) + .createTable( + new Schema( + rowType.getFields(), + List.of( + "boolean", + "tinyint", + "smallint", + "int", + "bigint", + "float", + "double", + "char", + "varchar", + "date", + "timestamp_0", + "timestamp_3", + "timestamp_6", + "timestamp_tz", + "decimal"), + List.of( + "boolean", + "tinyint", + "smallint", + "int", + "bigint", + "float", + "double", + "char", + "varchar", + "date", + "timestamp_0", + "timestamp_3", + "timestamp_6", + "timestamp_tz", + "decimal", + "varbinary"), + Collections.singletonMap("bucket", "1"), + "")); + FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath6); + InnerTableWrite writer = table.newWrite("user"); + InnerTableCommit commit = table.newCommit("user"); + writer.write( + GenericRow.of( + true, + (byte) 1, + (short) 1, + 1, + 1L, + 1.0f, + 1.0d, + BinaryString.fromString("char1"), + BinaryString.fromString("varchar1"), + 0, + Timestamp.fromMicros(1694505288000000L), + Timestamp.fromMicros(1694505288001000L), + Timestamp.fromMicros(1694505288001001L), + Timestamp.fromMicros(1694505288002001L), + Decimal.fromUnscaledLong(10000, 10, 5), + new byte[] {0x01, 0x02, 0x03}, + new GenericArray(new int[] {1, 1, 1}), + new GenericMap(Map.of(1, 1)), + GenericRow.of(1, 1))); + commit.commit(0, writer.prepareCommit(true, 0)); + } + + { + Path tablePath7 = new Path(warehouse, "default.db/t100"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "boolean", DataTypes.BOOLEAN()), + new DataField(1, "tinyint", DataTypes.TINYINT()), + new DataField(2, "smallint", DataTypes.SMALLINT()), + new DataField(3, "int", DataTypes.INT()), + new DataField(4, "bigint", DataTypes.BIGINT()), + new DataField(5, "float", DataTypes.FLOAT()), + new DataField(6, "double", DataTypes.DOUBLE()), + new DataField(7, "char", DataTypes.CHAR(5)), + new DataField(8, "varchar", DataTypes.VARCHAR(100)), + new DataField(9, "date", DataTypes.DATE()), + new DataField(10, "timestamp_0", DataTypes.TIMESTAMP(3)), + new DataField(11, "timestamp_3", DataTypes.TIMESTAMP(3)), + new DataField(12, "timestamp_6", DataTypes.TIMESTAMP(6)), + new DataField(13, "decimal", DataTypes.DECIMAL(10, 5)), + new DataField(14, "varbinary", DataTypes.VARBINARY(10)), + new DataField(15, "array", DataTypes.ARRAY(DataTypes.INT())), + new DataField( + 16, + "map", + DataTypes.MAP(DataTypes.INT(), DataTypes.INT())), + new DataField( + 17, + "row", + DataTypes.ROW( + DataTypes.FIELD(100, "q1", DataTypes.INT()), + DataTypes.FIELD(101, "q2", DataTypes.INT()))))); + new SchemaManager(LocalFileIO.create(), tablePath7) + .createTable( + new Schema( + rowType.getFields(), + Collections.emptyList(), + Collections.emptyList(), + Collections.singletonMap("bucket", "-1"), + "")); + FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath7); + InnerTableWrite writer = table.newWrite("user"); + InnerTableCommit commit = table.newCommit("user"); + writer.write( + GenericRow.of( + true, + (byte) 1, + (short) 1, + 1, + 1L, + 1.0f, + 1.0d, + BinaryString.fromString("char1"), + BinaryString.fromString("varchar1"), + 0, + Timestamp.fromMicros(1694505288000000L), + Timestamp.fromMicros(1694505288001000L), + Timestamp.fromMicros(1694505288001001L), + Decimal.fromUnscaledLong(10000, 10, 5), + new byte[] {0x01, 0x02, 0x03}, + new GenericArray(new int[] {1, 1, 1}), + new GenericMap(Map.of(1, 1)), + GenericRow.of(1, 1))); + commit.commit(0, writer.prepareCommit(true, 0)); + + new SchemaManager(LocalFileIO.create(), tablePath7) + .commitChanges(SchemaChange.dropColumn("smallint")); + table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath7); + writer = table.newWrite("user"); + commit = table.newCommit("user"); + writer.write( + GenericRow.of( + true, + (byte) 1, + 1, + 1L, + 1.0f, + 1.0d, + BinaryString.fromString("char1"), + BinaryString.fromString("varchar1"), + 0, + Timestamp.fromMicros(1694505288000000L), + Timestamp.fromMicros(1694505288001000L), + Timestamp.fromMicros(1694505288001001L), + Decimal.fromUnscaledLong(10000, 10, 5), + new byte[] {0x01, 0x02, 0x03}, + new GenericArray(new int[] {1, 1, 1}), + new GenericMap(Map.of(1, 1)), + GenericRow.of(1, 1))); + commit.commit(1, writer.prepareCommit(true, 1)); + + new SchemaManager(LocalFileIO.create(), tablePath7) + .commitChanges(SchemaChange.addColumn("smallint", DataTypes.SMALLINT())); + table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath7); + writer = table.newWrite("user"); + commit = table.newCommit("user"); + writer.write( + GenericRow.of( + true, + (byte) 1, + 1, + 1L, + 1.0f, + 1.0d, + BinaryString.fromString("char1"), + BinaryString.fromString("varchar1"), + 0, + Timestamp.fromMicros(1694505288000000L), + Timestamp.fromMicros(1694505288001000L), + Timestamp.fromMicros(1694505288001001L), + Decimal.fromUnscaledLong(10000, 10, 5), + new byte[] {0x01, 0x02, 0x03}, + new GenericArray(new int[] {1, 1, 1}), + new GenericMap(Map.of(1, 1)), + GenericRow.of(1, 1), + (short) 1)); + commit.commit(1, writer.prepareCommit(true, 1)); + + new SchemaManager(LocalFileIO.create(), tablePath7) + .commitChanges(SchemaChange.updateColumnType("smallint", DataTypes.STRING())); + table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath7); + writer = table.newWrite("user"); + commit = table.newCommit("user"); + writer.write( + GenericRow.of( + true, + (byte) 1, + 1, + 1L, + 1.0f, + 1.0d, + BinaryString.fromString("char1"), + BinaryString.fromString("varchar1"), + 0, + Timestamp.fromMicros(1694505288000000L), + Timestamp.fromMicros(1694505288001000L), + Timestamp.fromMicros(1694505288001001L), + Decimal.fromUnscaledLong(10000, 10, 5), + new byte[] {0x01, 0x02, 0x03}, + new GenericArray(new int[] {1, 1, 1}), + new GenericMap(Map.of(1, 1)), + GenericRow.of(1, 1), + BinaryString.fromString("10086"))); + commit.commit(1, writer.prepareCommit(true, 1)); + } + + { + Path tablePath6 = new Path(warehouse, "default.db/t101"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "a", DataTypes.STRING()), + new DataField(1, "b", DataTypes.INT()), + new DataField(2, "c", DataTypes.INT()))); + new SchemaManager(LocalFileIO.create(), tablePath6) + .createTable( + new Schema( + rowType.getFields(), + Collections.emptyList(), + List.of("a"), + new HashMap<>(ImmutableMap.of(CoreOptions.BUCKET.key(), "1", CoreOptions.DELETION_VECTORS_ENABLED.key(), "true")), + "")); + FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath6); + InnerTableWrite writer = table.newWrite("user"); + writer.withIOManager(new IOManagerImpl("/tmp")); + InnerTableCommit commit = table.newCommit("user"); + for (int i = 0; i < 10; i++) { + writer.write(GenericRow.of(BinaryString.fromString("a" + i), i, i)); + } + commit.commit(0, writer.prepareCommit(true, 0)); + + writer.write(GenericRow.ofKind(RowKind.DELETE, BinaryString.fromString("a0"), 0, 0)); + commit.commit(1, writer.prepareCommit(true, 1)); + } + + { + Path tablePath = new Path(warehouse, "default.db/t102"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "a", DataTypes.STRING()), + new DataField(1, "b", DataTypes.INT()), + new DataField(2, "c", DataTypes.INT()))); + new SchemaManager(LocalFileIO.create(), tablePath) + .createTable( + new Schema( + rowType.getFields(), + Collections.emptyList(), + Collections.emptyList(), + new HashMap<>(ImmutableMap.of("file-index.bloom-filter.columns", "a,b,c")), + "")); + FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath); + InnerTableWrite writer = table.newWrite("user"); + writer.withIOManager(new IOManagerImpl("/tmp")); + InnerTableCommit commit = table.newCommit("user"); + for (int i = 0; i < 100; i = i + 3) { + writer.write(GenericRow.of(BinaryString.fromString("a" + i), i, i)); + } + commit.commit(0, writer.prepareCommit(true, 0)); + + for (int i = 1; i < 100; i = i + 3) { + writer.write(GenericRow.of(BinaryString.fromString("a" + i), i, i)); + } + commit.commit(1, writer.prepareCommit(true, 1)); + + for (int i = 2; i < 100; i = i + 3) { + writer.write(GenericRow.of(BinaryString.fromString("a" + i), i, i)); + } + commit.commit(2, writer.prepareCommit(true, 2)); + } + + { + Path tablePath = new Path(warehouse, "default.db/fixed_bucket_table_wi_pk"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "id", DataTypes.INT()), + new DataField(1, "name", DataTypes.STRING()))); + new SchemaManager(LocalFileIO.create(), tablePath) + .createTable( + new Schema( + rowType.getFields(), + Collections.emptyList(), + Collections.emptyList(), + new HashMap<>(ImmutableMap.of("file.format", "orc", "primary-key", "id", "bucket", "2")), + "")); + } + + { + Path tablePath = new Path(warehouse, "default.db/fixed_bucket_table_wo_pk"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "id", DataTypes.INT()), + new DataField(1, "name", DataTypes.STRING()))); + new SchemaManager(LocalFileIO.create(), tablePath) + .createTable( + new Schema( + rowType.getFields(), + Collections.emptyList(), + Collections.emptyList(), + new HashMap<>(ImmutableMap.of("file.format", "orc", "bucket", "2", "bucket-key", "id")), + "")); + } + + { + Path tablePath = new Path(warehouse, "default.db/unaware_table"); + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "id", DataTypes.INT()), + new DataField(1, "name", DataTypes.STRING()))); + new SchemaManager(LocalFileIO.create(), tablePath) + .createTable( + new Schema( + rowType.getFields(), + Collections.emptyList(), + Collections.emptyList(), + new HashMap<>(ImmutableMap.of("file.format", "orc")), + "")); + } + + DistributedQueryRunner queryRunner = null; + try { + queryRunner = + DistributedQueryRunner.builder( + testSessionBuilder().setCatalog(CATALOG).setSchema(DB).build()) + .build(); + queryRunner.installPlugin(new PaimonPlugin()); + Map options = new HashMap<>(); + options.put("paimon.warehouse", warehouse); + options.put("paimon.catalog.type", "filesystem"); + options.put("fs.hadoop.enabled", "true"); + queryRunner.createCatalog(CATALOG, CATALOG, options); + return queryRunner; + } + catch (Throwable e) { + closeAllSuppress(e, queryRunner); + throw e; + } + } + + @Test + void testComplexTypes() + { + assertThat(execute("SELECT * FROM paimon.default.t4")) + .isEqualTo("[[1, {1=2}, [2, male], [1, 2, 3]]]"); + } + + @Test + void testEmptyTable() + { + assertThat(execute("SELECT * FROM paimon.default.empty_t")).isEqualTo("[]"); + } + + @Test + void testProjection() + { + assertThat(execute("SELECT * FROM paimon.default.t1")) + .isEqualTo("[[1, 2, 1, 1], [5, 6, 3, 3]]"); + assertThat(execute("SELECT a, aCa FROM paimon.default.t1")).isEqualTo("[[1, 1], [5, 3]]"); + assertThat(execute("SELECT SUM(b) FROM paimon.default.t1")).isEqualTo("[[8]]"); + } + + @Test + void testLimit() + { + assertThat(execute("SELECT * FROM paimon.default.t1 LIMIT 1")).isEqualTo("[[1, 2, 1, 1]]"); + assertThat(execute("SELECT * FROM paimon.default.t1 WHERE a = 5 LIMIT 1")) + .isEqualTo("[[5, 6, 3, 3]]"); + } + + @Test + void testFilter() + { + assertThat(execute("SELECT a, aCa FROM paimon.default.t2 WHERE a < 4")) + .isEqualTo("[[1, 1], [3, 2]]"); + } + + @Test + void testGroupByWithCast() + { + assertThat( + execute( + "SELECT pt, a, SUM(b), SUM(d) FROM paimon.default.t3 GROUP BY pt, a ORDER BY pt, a")) + .isEqualTo("[[1, 1, 3, 3], [2, 3, 3, 3]]"); + } + + @Test + void testLimitWithPartition() + { + assertThat(execute("SELECT * FROM paimon.default.t3 WHERE pt = '1' LIMIT 1")) + .isEqualTo("[[1, 1, 1, 1, 1]]"); + + assertThat(execute("SELECT * FROM paimon.default.t3 WHERE pt = '1' AND b = 2 LIMIT 1")) + .isEqualTo("[[1, 1, 2, 2, 2]]"); + } + + @Test + void testAllType() + { + assertThat( + execute( + "SELECT boolean, tinyint, smallint,int,bigint,float,double,char,varchar, date,timestamp_0, " + + "timestamp_3, timestamp_6, decimal, to_hex(varbinary), array, map, row FROM paimon.default.t99")) + .isEqualTo( + "[[true, 1, 1, 1, 1, 1.0, 1.0, char1, varchar1, 1970-01-01, " + + "2023-09-12T07:54:48, 2023-09-12T07:54:48.001, 2023-09-12T07:54:48.001001, " + + "0.10000, 010203, [1, 1, 1], {1=1}, [1, 1]]]"); + } + + @Test + void testSchemaEvolution() + { + assertThat( + execute( + "SELECT boolean, tinyint, smallint, int, bigint,float,double,char,varchar, date,timestamp_0, " + + "timestamp_3, timestamp_6, decimal, to_hex(varbinary), array, map, row FROM paimon.default.t100")) + .isEqualTo( + "[[true, 1, null, 1, 1, 1.0, 1.0, char1, varchar1, 1970-01-01, 2023-09-12T07:54:48, 2023-09-12T07:54:48.001, 2023-09-12T07:54:48.001001, 0.10000, 010203, [1, 1, 1], {1=1}, [1, 1]], " + + "[true, 1, null, 1, 1, 1.0, 1.0, char1, varchar1, 1970-01-01, 2023-09-12T07:54:48, 2023-09-12T07:54:48.001, 2023-09-12T07:54:48.001001, 0.10000, 010203, [1, 1, 1], {1=1}, [1, 1]], " + + "[true, 1, 1, 1, 1, 1.0, 1.0, char1, varchar1, 1970-01-01, 2023-09-12T07:54:48, 2023-09-12T07:54:48.001, 2023-09-12T07:54:48.001001, 0.10000, 010203, [1, 1, 1], {1=1}, [1, 1]], " + + "[true, 1, 10086, 1, 1, 1.0, 1.0, char1, varchar1, 1970-01-01, 2023-09-12T07:54:48, 2023-09-12T07:54:48.001, 2023-09-12T07:54:48.001001, 0.10000, 010203, [1, 1, 1], {1=1}, [1, 1]]]"); + } + + @Test + void testDeletionFile() + { + assertThat(execute("SELECT * FROM paimon.default.t101 WHERE b > 0")) + .isEqualTo( + "[[a1, 1, 1], [a2, 2, 2], [a3, 3, 3], [a4, 4, 4], [a5, 5, 5], [a6, 6, 6], [a7, 7, 7], [a8, 8, 8], [a9, 9, 9]]"); + } + + @Test + void testFileIndex() + { + assertThat(execute("SELECT * FROM paimon.default.t102 where c = 2")).isEqualTo("[[a2, 2, 2]]"); + } + + private String execute(String sql) + { + MaterializedResult result = getQueryRunner().execute(sql); + // TODO Use assertThat(query()) instead + return result.getMaterializedRows().toString(); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonMinioConnectorSmokeTest.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonMinioConnectorSmokeTest.java new file mode 100644 index 000000000000..1e5cdb674a87 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonMinioConnectorSmokeTest.java @@ -0,0 +1,64 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.plugin.hive.containers.Hive3MinioDataLake; +import io.trino.plugin.paimon.testing.PaimonTablesInitializer; +import io.trino.testing.BaseConnectorSmokeTest; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; + +import static io.trino.plugin.hive.containers.HiveHadoop.HIVE3_IMAGE; +import static io.trino.testing.TestingNames.randomNameSuffix; + +final class TestPaimonMinioConnectorSmokeTest + extends BaseConnectorSmokeTest +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + String bucketName = "test-paimon-connector-" + randomNameSuffix(); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName, HIVE3_IMAGE)); + hiveMinioDataLake.start(); + hiveMinioDataLake.getMinioClient().ensureBucketExists(bucketName); + + return PaimonQueryRunner.builder(hiveMinioDataLake) + .setDataLoader(new PaimonTablesInitializer(REQUIRED_TPCH_TABLES)) + .build(); + } + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + return switch (connectorBehavior) { + case SUPPORTS_ADD_COLUMN, + SUPPORTS_ARRAY, + SUPPORTS_CREATE_MATERIALIZED_VIEW, + SUPPORTS_CREATE_SCHEMA, + SUPPORTS_CREATE_TABLE, + SUPPORTS_CREATE_VIEW, + SUPPORTS_DELETE, + SUPPORTS_INSERT, + SUPPORTS_MAP_TYPE, + SUPPORTS_MERGE, + SUPPORTS_RENAME_COLUMN, + SUPPORTS_RENAME_SCHEMA, + SUPPORTS_RENAME_TABLE, + SUPPORTS_ROW_TYPE, + SUPPORTS_UPDATE -> false; + default -> super.hasBehavior(connectorBehavior); + }; + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonPlugin.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonPlugin.java new file mode 100644 index 000000000000..6eff7f4e1029 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonPlugin.java @@ -0,0 +1,46 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.Plugin; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.testing.TestingConnectorContext; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.UUID; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static java.nio.file.Files.createTempDirectory; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestPaimonPlugin +{ + @Test + void testCreatePaimonConnector() + throws IOException + { + String warehouse = createTempDirectory(UUID.randomUUID().toString()).toUri().toString(); + Plugin plugin = new PaimonPlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + Connector connector = factory.create( + "paimon", + ImmutableMap.of("paimon.warehouse", warehouse, "paimon.catalog.type", "filesystem"), + new TestingConnectorContext()); + assertThat(connector).isNotNull(); + connector.shutdown(); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonRow.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonRow.java new file mode 100644 index 000000000000..7e0727522294 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonRow.java @@ -0,0 +1,106 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.airlift.slice.Slices; +import io.trino.spi.Page; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.types.RowType; +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; +import java.time.LocalDateTime; + +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.Decimals.encodeScaledValue; +import static io.trino.spi.type.Decimals.encodeShortScaledValue; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MICROS; +import static io.trino.spi.type.TypeUtils.writeNativeValue; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.type.DateTimes.MICROSECONDS_PER_MILLISECOND; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.paimon.data.Timestamp.fromLocalDateTime; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +final class TestPaimonRow +{ + @Test + void testPaimonRow() + { + Page singlePage = new Page( + 1, + writeNativeValue(BOOLEAN, null), + writeNativeValue(BOOLEAN, false), + writeNativeValue(VARBINARY, Slices.wrappedBuffer((byte) 22)), + writeNativeValue(SMALLINT, 356L), + writeNativeValue(INTEGER, 4L), + writeNativeValue(BIGINT, 23567222L), + writeNativeValue(REAL, (long) Float.floatToIntBits(1213.33f)), + writeNativeValue(DOUBLE, 121.3d), + writeNativeValue(VARCHAR, Slices.wrappedBuffer("rfyu".getBytes(UTF_8))), + writeNativeValue(createDecimalType(2, 2), encodeShortScaledValue(BigDecimal.valueOf(0.21), 2)), + writeNativeValue(createDecimalType(38, 2), encodeScaledValue(BigDecimal.valueOf(65782123123.01), 2)), + writeNativeValue(createDecimalType(10, 1), encodeShortScaledValue(BigDecimal.valueOf(62123123.5), 1)), + writeNativeValue(TIMESTAMP_MICROS, fromLocalDateTime(LocalDateTime.parse("2007-12-03T10:15:30")).getMillisecond() * MICROSECONDS_PER_MILLISECOND), + writeNativeValue(VARBINARY, Slices.wrappedBuffer("varbinary_v".getBytes(UTF_8)))); + + RowType rowType = RowType.builder().fields( + DataTypes.BOOLEAN(), + DataTypes.BOOLEAN(), + DataTypes.SMALLINT(), + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.STRING(), + DataTypes.DECIMAL(2, 2), + DataTypes.DECIMAL(38, 2), + DataTypes.DECIMAL(10, 1), + DataTypes.TIMESTAMP(6), + DataTypes.VARBINARY(Integer.MAX_VALUE)).build(); + + PaimonRow paimonRow = new PaimonRow(rowType, singlePage, RowKind.INSERT); + + assertThat(paimonRow.getRowKind()).isEqualTo(RowKind.INSERT); + assertThat(paimonRow.isNullAt(0)).isEqualTo(true); + assertThat(paimonRow.getBoolean(1)).isEqualTo(false); + assertThat(paimonRow.getBinary(2)[0]).isEqualTo((byte) 22); + assertThat(paimonRow.getShort(3)).isEqualTo((short) 356); + assertThat(paimonRow.getInt(4)).isEqualTo(4); + assertThat(paimonRow.getLong(5)).isEqualTo(23567222L); + assertThat(paimonRow.getFloat(6)).isEqualTo(1213.33f); + assertThat(paimonRow.getDouble(7)).isEqualTo(121.3d); + assertThat(paimonRow.getString(8)).isEqualTo(BinaryString.fromString("rfyu")); + assertThat(paimonRow.getDecimal(9, 2, 2)) + .isEqualTo(Decimal.fromBigDecimal(BigDecimal.valueOf(0.21), 2, 2)); + assertThat(paimonRow.getDecimal(10, 38, 2)) + .isEqualTo(Decimal.fromBigDecimal(BigDecimal.valueOf(65782123123.01), 38, 2)); + assertThat(paimonRow.getDecimal(11, 10, 1)) + .isEqualTo(Decimal.fromBigDecimal(BigDecimal.valueOf(62123123.5), 10, 1)); + assertThat(paimonRow.getTimestamp(12, 6)) + .isEqualTo(fromLocalDateTime(LocalDateTime.parse("2007-12-03T10:15:30"))); + assertThat(paimonRow.getBinary(13)) + .isEqualTo("varbinary_v".getBytes(UTF_8)); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonSplit.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonSplit.java new file mode 100644 index 000000000000..ea28db04dc46 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonSplit.java @@ -0,0 +1,37 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.airlift.json.JsonCodec; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; + +final class TestPaimonSplit +{ + private final JsonCodec codec = JsonCodec.jsonCodec(PaimonSplit.class); + + @Test + void testJsonRoundTrip() + throws Exception + { + byte[] serializedTable = PaimonTestUtils.getSerializedTable(); + PaimonSplit expected = new PaimonSplit(Arrays.toString(serializedTable), 0.1); + String json = codec.toJson(expected); + PaimonSplit actual = codec.fromJson(json); + assertThat(actual.splitSerialized()).isEqualTo(expected.splitSerialized()); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonTrinoType.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonTrinoType.java new file mode 100644 index 000000000000..74620bcdc875 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestPaimonTrinoType.java @@ -0,0 +1,119 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.BooleanType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DateType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.DoubleType; +import io.trino.spi.type.IntegerType; +import io.trino.spi.type.MapType; +import io.trino.spi.type.RealType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.SmallintType; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TimestampWithTimeZoneType; +import io.trino.spi.type.TinyintType; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeOperators; +import io.trino.spi.type.VarbinaryType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.TimeType; +import org.apache.paimon.types.VarCharType; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static io.trino.plugin.paimon.PaimonTypeUtils.toPaimonType; +import static io.trino.plugin.paimon.PaimonTypeUtils.toTrinoType; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestPaimonTrinoType +{ + @Test + void testToTrinoType() + { + assertThat(toTrinoType(DataTypes.CHAR(1)).getDisplayName()).isEqualTo("char(1)"); + assertThat(toTrinoType(DataTypes.VARCHAR(10)).getDisplayName()).isEqualTo("varchar(10)"); + assertThat(toTrinoType(DataTypes.BOOLEAN()).getDisplayName()).isEqualTo("boolean"); + assertThat(toTrinoType(DataTypes.BINARY(10)).getDisplayName()).isEqualTo("varbinary"); + assertThat(toTrinoType(DataTypes.VARBINARY(10)).getDisplayName()).isEqualTo("varbinary"); + assertThat(toTrinoType(DataTypes.DECIMAL(38, 0)).getDisplayName()).isEqualTo("decimal(38,0)"); + assertThat(toTrinoType(DataTypes.DECIMAL(2, 2)).getDisplayName()).isEqualTo("decimal(2,2)"); + assertThat(toTrinoType(DataTypes.TINYINT()).getDisplayName()).isEqualTo("tinyint"); + assertThat(toTrinoType(DataTypes.SMALLINT()).getDisplayName()).isEqualTo("smallint"); + assertThat(toTrinoType(DataTypes.INT()).getDisplayName()).isEqualTo("integer"); + assertThat(toTrinoType(DataTypes.BIGINT()).getDisplayName()).isEqualTo("bigint"); + assertThat(toTrinoType(DataTypes.FLOAT()).getDisplayName()).isEqualTo("real"); + assertThat(toTrinoType(DataTypes.DOUBLE()).getDisplayName()).isEqualTo("double"); + assertThat(toTrinoType(DataTypes.DATE()).getDisplayName()).isEqualTo("date"); + assertThat(toTrinoType(new TimeType(0)).getDisplayName()).isEqualTo("time(0)"); + assertThat(toTrinoType(new TimeType(3)).getDisplayName()).isEqualTo("time(3)"); + assertThat(toTrinoType(new TimeType(6)).getDisplayName()).isEqualTo("time(6)"); + assertThat(toTrinoType(new TimeType(9)).getDisplayName()).isEqualTo("time(9)"); + assertThat(toTrinoType(DataTypes.TIMESTAMP()).getDisplayName()).isEqualTo("timestamp(6)"); + assertThat(toTrinoType(new org.apache.paimon.types.TimestampType(3)).getDisplayName()).isEqualTo("timestamp(3)"); + assertThat(toTrinoType(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()).getDisplayName()).isEqualTo("timestamp(6) with time zone"); + assertThat(toTrinoType(DataTypes.ARRAY(DataTypes.STRING())).getDisplayName()).isEqualTo("array(varchar)"); + assertThat(toTrinoType(DataTypes.MULTISET(DataTypes.STRING())).getDisplayName()).isEqualTo("map(varchar, integer)"); + assertThat(toTrinoType(DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING())).getDisplayName()).isEqualTo("map(bigint, varchar)"); + assertThat(toTrinoType(DataTypes.ROW( + new DataField(0, "id", new IntType()), + new DataField(1, "name", new VarCharType(Integer.MAX_VALUE)))).getDisplayName()) + .isEqualTo("row(\"id\" integer, \"name\" varchar)"); + } + + @Test + void testToPaimonType() + { + assertThat(toPaimonType(CharType.createCharType(1)).asSQLString()).isEqualTo("CHAR(1)"); + assertThat(toPaimonType(VARCHAR).asSQLString()).isEqualTo("VARCHAR(2147483646)"); + assertThat(toPaimonType(BooleanType.BOOLEAN).asSQLString()).isEqualTo("BOOLEAN"); + assertThat(toPaimonType(VarbinaryType.VARBINARY).asSQLString()).isEqualTo("BYTES"); + assertThat(toPaimonType(DecimalType.createDecimalType(2, 2)).asSQLString()).isEqualTo("DECIMAL(2, 2)"); + assertThat(toPaimonType(TinyintType.TINYINT).asSQLString()).isEqualTo("TINYINT"); + assertThat(toPaimonType(SmallintType.SMALLINT).asSQLString()).isEqualTo("SMALLINT"); + assertThat(toPaimonType(IntegerType.INTEGER).asSQLString()).isEqualTo("INT"); + assertThat(toPaimonType(BigintType.BIGINT).asSQLString()).isEqualTo("BIGINT"); + assertThat(toPaimonType(RealType.REAL).asSQLString()).isEqualTo("FLOAT"); + assertThat(toPaimonType(DoubleType.DOUBLE).asSQLString()).isEqualTo("DOUBLE"); + assertThat(toPaimonType(DateType.DATE).asSQLString()).isEqualTo("DATE"); + assertThat(toPaimonType(io.trino.spi.type.TimeType.TIME_SECONDS).asSQLString()).isEqualTo("TIME(0)"); + assertThat(toPaimonType(io.trino.spi.type.TimeType.TIME_MILLIS).asSQLString()).isEqualTo("TIME(3)"); + assertThat(toPaimonType(TimestampType.TIMESTAMP_SECONDS).asSQLString()).isEqualTo("TIMESTAMP(0)"); + assertThat(toPaimonType(TimestampType.TIMESTAMP_MILLIS).asSQLString()).isEqualTo("TIMESTAMP(3)"); + assertThat(toPaimonType(TimestampType.TIMESTAMP_MICROS).asSQLString()).isEqualTo("TIMESTAMP(6)"); + assertThat(toPaimonType(TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS).asSQLString()).isEqualTo("TIMESTAMP(3) WITH LOCAL TIME ZONE"); + assertThat(toPaimonType(new ArrayType(IntegerType.INTEGER)).asSQLString()).isEqualTo("ARRAY"); + assertThat(toPaimonType(new MapType( + IntegerType.INTEGER, + VARCHAR, + new TypeOperators())).asSQLString()).isEqualTo("MAP"); + List fields = new ArrayList<>(); + fields.add(new RowType.Field(Optional.of("id"), IntegerType.INTEGER)); + fields.add(new RowType.Field(Optional.of("name"), VARCHAR)); + Type type = RowType.from(fields); + DataType rowType = toPaimonType(type); + assertThat(rowType.asSQLString()).isEqualTo("ROW<`id` INT, `name` VARCHAR(2147483646)>"); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestingPaimonConnectorFactory.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestingPaimonConnectorFactory.java new file mode 100644 index 000000000000..eaef1e19950b --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestingPaimonConnectorFactory.java @@ -0,0 +1,59 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.local.LocalFileSystemFactory; +import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorFactory; + +import java.nio.file.Path; +import java.util.Map; +import java.util.Optional; + +import static com.google.inject.multibindings.MapBinder.newMapBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class TestingPaimonConnectorFactory + implements ConnectorFactory +{ + private final Path localFileSystemRootPath; + + public TestingPaimonConnectorFactory(Path localFileSystemRootPath) + { + localFileSystemRootPath.toFile().mkdirs(); + this.localFileSystemRootPath = localFileSystemRootPath; + } + + @Override + public String getName() + { + return "paimon"; + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + ImmutableMap.Builder configBuilder = ImmutableMap.builder() + .putAll(config); + return PaimonConnectorFactory.createConnector(catalogName, configBuilder.buildOrThrow(), context, Optional.of(binder -> { + newMapBinder(binder, String.class, TrinoFileSystemFactory.class) + .addBinding("file").toInstance(new LocalFileSystemFactory(localFileSystemRootPath)); + configBinder(binder).bindConfigDefaults(FileHiveMetastoreConfig.class, metastoreConfig -> metastoreConfig.setCatalogDirectory("file:///managed/")); + })); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestingPaimonPlugin.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestingPaimonPlugin.java new file mode 100644 index 000000000000..4a5e60a24482 --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/TestingPaimonPlugin.java @@ -0,0 +1,43 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.connector.ConnectorFactory; + +import java.nio.file.Path; +import java.util.List; + +import static com.google.common.base.Verify.verify; +import static java.util.Objects.requireNonNull; + +public class TestingPaimonPlugin + extends PaimonPlugin +{ + private final Path localFileSystemRootPath; + + public TestingPaimonPlugin(Path localFileSystemRootPath) + { + this.localFileSystemRootPath = requireNonNull(localFileSystemRootPath, "localFileSystemRootPath is null"); + } + + @Override + public Iterable getConnectorFactories() + { + List connectorFactories = ImmutableList.copyOf(super.getConnectorFactories()); + verify(connectorFactories.size() == 1, "Unexpected connector factories: %s", connectorFactories); + + return ImmutableList.of(new TestingPaimonConnectorFactory(localFileSystemRootPath)); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/PaimonTablesInitializer.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/PaimonTablesInitializer.java new file mode 100644 index 000000000000..3cd7f80e62ad --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/PaimonTablesInitializer.java @@ -0,0 +1,201 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.testing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.trino.Session; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; +import io.trino.plugin.paimon.CatalogType; +import io.trino.plugin.paimon.PaimonConnector; +import io.trino.plugin.paimon.PaimonMetadata; +import io.trino.plugin.paimon.PaimonMetadataFactory; +import io.trino.plugin.paimon.catalog.TrinoCatalog; +import io.trino.plugin.paimon.fileio.PaimonFileIO; +import io.trino.plugin.tpch.TpchMetadata; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.plugin.tpch.TpchTableHandle; +import io.trino.spi.connector.CatalogSchemaName; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.testing.MaterializedResult; +import io.trino.testing.MaterializedRow; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorSession; +import io.trino.tpch.TpchTable; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.Database; +import org.apache.paimon.catalog.FileSystemCatalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.FileIOLoader; +import org.apache.paimon.fs.Path; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.sink.BatchTableCommit; +import org.apache.paimon.table.sink.BatchTableWrite; +import org.apache.paimon.table.sink.BatchWriteBuilder; +import org.intellij.lang.annotations.Language; + +import java.time.LocalDate; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static io.trino.plugin.paimon.testing.PaimonTestingUtils.createPaimonTable; +import static java.lang.String.format; + +public class PaimonTablesInitializer +{ + private static final CatalogSchemaName TPCH_TINY = new CatalogSchemaName("tpch", "tiny"); + private static final Logger log = Logger.get(PaimonTablesInitializer.class); + + private final List> tpchTables; + + public PaimonTablesInitializer(List> tpchTables) + { + this.tpchTables = ImmutableList.copyOf(tpchTables); + } + + private static void createTableWithData(Catalog catalog, String databaseName, TpchTable tpchTable, QueryRunner queryRunner) + throws Exception + { + @Language("SQL") String sql = generateScanSql(TPCH_TINY, tpchTable); + log.info("Executing %s", sql); + MaterializedResult result = queryRunner.execute(sql); + + Table paimonTable = catalog.getTable(Identifier.create(databaseName, tpchTable.getTableName())); + BatchWriteBuilder builder = paimonTable.newBatchWriteBuilder(); + + try (BatchTableWrite write = builder.newWrite(); + BatchTableCommit commit = builder.newCommit()) { + for (MaterializedRow row : result.getMaterializedRows()) { + write.write(toPaimonRow(row)); + } + commit.commit(write.prepareCommit()); + } + } + + private static InternalRow toPaimonRow(MaterializedRow row) + { + GenericRow genericRow = new GenericRow(row.getFields().size()); + for (int i = 0; i < row.getFields().size(); i++) { + Object field = row.getField(i); + switch (field) { + case String string -> genericRow.setField(i, BinaryString.fromString(string)); + case LocalDate date -> genericRow.setField(i, (int) date.toEpochDay()); + default -> genericRow.setField(i, field); + } + } + return genericRow; + } + + private static String generateScanSql(CatalogSchemaName catalogSchemaName, TpchTable table) + { + StringBuilder builder = new StringBuilder(); + builder.append("SELECT "); + String columnList = table.getColumns().stream() + .map(column -> quote(column.getSimplifiedColumnName())) + .collect(Collectors.joining(", ")); + builder.append(columnList); + String tableName = format("%s.%s", catalogSchemaName.toString(), table.getTableName()); + builder.append(" FROM ").append(tableName); + return builder.toString(); + } + + private static String quote(String name) + { + return "\"" + name + "\""; + } + + public void initializeTables(Session session, QueryRunner queryRunner, String schemaName) + throws Exception + { + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog(TPCH_TINY.getCatalogName(), "tpch", ImmutableMap.of()); + ConnectorIdentity connectorIdentity = ConnectorIdentity.ofUser(session.getUser()); + ConnectorSession connectorSession = TestingConnectorSession.SESSION; + + PaimonMetadata paimonMetadata = ((PaimonConnector) queryRunner.getCoordinator().getConnector("paimon")).getInjector() + .getInstance(PaimonMetadataFactory.class) + .create(connectorIdentity); + + TrinoFileSystem fileSystem = ((PaimonConnector) queryRunner.getCoordinator().getConnector("paimon")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(connectorIdentity); + + PaimonFileIO paimonFileIO = new PaimonFileIO(fileSystem, new Path(paimonMetadata.catalog().warehouse())); + + TrinoCatalog paimonTrinoCatalog = paimonMetadata.catalog(); + + Options options = new Options(); + options.set(CatalogOptions.WAREHOUSE, paimonTrinoCatalog.warehouse()); + CatalogContext context = CatalogContext.create(options, new FileIOLoader() + { + @Override + public String getScheme() + { + return "s3"; + } + + @Override + public FileIO load(Path path) + { + return paimonFileIO; + } + }); + + Catalog paimonCatalog = new FileSystemCatalog(paimonFileIO, new Path(paimonTrinoCatalog.warehouse()), context.options()) + { + @Override + public Database getDatabaseImpl(String name) + { + return Database.of(name); + } + }; + + if (paimonMetadata.catalog().config().getCatalogType() == CatalogType.HIVE) { + HiveMetastoreFactory hiveMetastoreFactory = ((PaimonConnector) queryRunner.getCoordinator().getConnector("paimon")).getInjector() + .getInstance(HiveMetastoreFactory.class); + HiveMetastore hiveMetaStore = hiveMetastoreFactory.createMetastore(Optional.of(connectorIdentity)); + paimonCatalog = new TrinoHiveCatalog(hiveMetaStore, paimonCatalog); + } + + paimonCatalog.createDatabase(schemaName, true); + TpchMetadata tpchMetadata = (TpchMetadata) queryRunner.getCoordinator().getConnector("tpch").getMetadata(null, null); + + for (TpchTable tpchTable : tpchTables) { + TpchTableHandle tpchTableHandle = tpchMetadata.getTableHandle(connectorSession, SchemaTableName.schemaTableName(TPCH_TINY.getSchemaName(), tpchTable.getTableName()), Optional.empty(), Optional.empty()); + ConnectorTableMetadata metadata = tpchMetadata.getTableMetadata(connectorSession, tpchTableHandle); + ConnectorTableMetadata paimonTableMeta = new ConnectorTableMetadata( + new SchemaTableName(schemaName, metadata.getTable().getTableName()), + metadata.getColumns().stream().filter(c -> !c.isHidden()).collect(Collectors.toList()), + metadata.getProperties(), + metadata.getComment()); + createPaimonTable(paimonCatalog, paimonTableMeta); + createTableWithData(paimonCatalog, session.getSchema().orElse(schemaName), tpchTable, queryRunner); + } + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/PaimonTestingUtils.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/PaimonTestingUtils.java new file mode 100644 index 000000000000..13f6b55ba21a --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/PaimonTestingUtils.java @@ -0,0 +1,79 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.testing; + +import io.trino.plugin.paimon.PaimonTableOptionUtils; +import io.trino.plugin.paimon.PaimonTableOptions; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.SchemaTableName; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.schema.Schema; + +import java.util.HashMap; +import java.util.Map; + +import static io.trino.plugin.paimon.PaimonTypeUtils.toPaimonType; +import static io.trino.spi.StandardErrorCode.SCHEMA_NOT_FOUND; +import static java.lang.String.format; + +/** + * Utility class for Paimon connector testing. + * This class provides helper methods for creating tables in tests. + */ +public final class PaimonTestingUtils +{ + private PaimonTestingUtils() {} + + /** + * Create a Paimon table for testing purposes. + * This method is intended for use in tests only. + */ + public static void createPaimonTable( + Catalog catalog, + ConnectorTableMetadata tableMetadata) + { + SchemaTableName table = tableMetadata.getTable(); + Identifier identifier = Identifier.create(table.getSchemaName(), table.getTableName()); + + try { + catalog.createTable(identifier, prepareSchema(tableMetadata), true); + } + catch (Catalog.DatabaseNotExistException e) { + throw new TrinoException(SCHEMA_NOT_FOUND, format("Schema %s not found", table.getSchemaName()), e); + } + catch (Catalog.TableAlreadyExistException _) { + // Table was already created by a previous test setup step. + } + } + + private static Schema prepareSchema(ConnectorTableMetadata tableMetadata) + { + Map properties = new HashMap<>(tableMetadata.getProperties()); + Schema.Builder builder = Schema.newBuilder() + .primaryKey(PaimonTableOptions.getPrimaryKeys(properties)) + .partitionKeys(PaimonTableOptions.getPartitionedKeys(properties)) + .comment(tableMetadata.getComment().orElse(null)); + + for (ColumnMetadata column : tableMetadata.getColumns()) { + builder.column(column.getName(), toPaimonType(column.getType()), column.getComment().orElse(null)); + } + + PaimonTableOptionUtils.buildOptions(builder, properties); + + return builder.build(); + } +} diff --git a/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/TrinoHiveCatalog.java b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/TrinoHiveCatalog.java new file mode 100644 index 000000000000..88c1ca8d9b2c --- /dev/null +++ b/plugin/trino-paimon/src/test/java/io/trino/plugin/paimon/testing/TrinoHiveCatalog.java @@ -0,0 +1,401 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.paimon.testing; + +import io.trino.metastore.Column; +import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveType; +import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.StorageFormat; +import org.apache.paimon.PagedList; +import org.apache.paimon.Snapshot; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; +import org.apache.paimon.catalog.Database; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; +import org.apache.paimon.function.Function; +import org.apache.paimon.function.FunctionChange; +import org.apache.paimon.partition.Partition; +import org.apache.paimon.partition.PartitionStatistics; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.table.Instant; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.TableSnapshot; +import org.apache.paimon.types.DataField; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.metastore.type.TypeInfoUtils.getTypeInfoFromTypeString; +import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; +import static java.util.Objects.requireNonNull; + +public class TrinoHiveCatalog + implements Catalog +{ + private static final String INPUT_FORMAT_CLASS_NAME = "org.apache.paimon.hive.mapred.PaimonInputFormat"; + private static final String OUTPUT_FORMAT_CLASS_NAME = "org.apache.paimon.hive.mapred.PaimonOutputFormat"; + private static final String SERDE_CLASS_NAME = "org.apache.paimon.hive.PaimonSerDe"; + + public static final StorageFormat PAIMON_METASTORE_STORAGE_FORMAT = StorageFormat.create( + SERDE_CLASS_NAME, + INPUT_FORMAT_CLASS_NAME, + OUTPUT_FORMAT_CLASS_NAME); + + private final HiveMetastore hiveMetastore; + private final Catalog baseCatalog; + + public TrinoHiveCatalog(HiveMetastore hiveMetastore, Catalog baseCatalog) + { + this.hiveMetastore = requireNonNull(hiveMetastore, "hiveMetastore is null"); + this.baseCatalog = requireNonNull(baseCatalog, "baseCatalog is null"); + } + + public static List toHiveColumns(List columns) + { + return columns.stream() + .map(column -> new Column( + column.name(), + HiveType.fromTypeInfo(getTypeInfoFromTypeString(column.type().asSQLString().toLowerCase(Locale.ROOT))), + Optional.ofNullable(column.description()), + Map.of())) + .collect(toImmutableList()); + } + + @Override + public Map options() + { + return baseCatalog.options(); + } + + @Override + public CatalogLoader catalogLoader() + { + return baseCatalog.catalogLoader(); + } + + @Override + public boolean caseSensitive() + { + return baseCatalog.caseSensitive(); + } + + @Override + public List listDatabases() + { + return hiveMetastore.getAllDatabases(); + } + + @Override + public PagedList listDatabasesPaged(@Nullable Integer integer, @Nullable String s, @Nullable String s1) + { + return null; + } + + @Override + public void createDatabase(String name, boolean ignoreIfExists, Map options) + throws DatabaseAlreadyExistException + { + if (databaseExists(name)) { + if (ignoreIfExists) { + return; + } + throw new DatabaseAlreadyExistException(name); + } + + hiveMetastore.createDatabase(io.trino.metastore.Database.builder().setDatabaseName(name).setOwnerName(Optional.empty()).setOwnerType(Optional.empty()).setParameters(options).build()); + baseCatalog.createDatabase(name, ignoreIfExists); + } + + private boolean databaseExists(String databaseName) + { + return hiveMetastore.getDatabase(databaseName).isPresent(); + } + + @Override + public Database getDatabase(String name) + throws DatabaseNotExistException + { + Optional database = hiveMetastore.getDatabase(name); + if (database.isEmpty()) { + throw new DatabaseNotExistException(name); + } + + return new Database.DatabaseImpl(name, database.get().getParameters(), database.get().getComment().orElse(null)); + } + + @Override + public void dropDatabase(String databaseName, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException + { + if (!databaseExists(databaseName)) { + if (ignoreIfNotExists) { + return; + } + throw new DatabaseNotExistException(databaseName); + } + + hiveMetastore.dropDatabase(databaseName, false); + baseCatalog.dropDatabase(databaseName, ignoreIfNotExists, cascade); + } + + @Override + public void alterDatabase(String databaseName, List propertyChanges, boolean ignoreIfNotExists) + throws DatabaseNotExistException + { + if (!databaseExists(databaseName)) { + if (ignoreIfNotExists) { + return; + } + throw new DatabaseNotExistException(databaseName); + } + + baseCatalog.alterDatabase(databaseName, propertyChanges, ignoreIfNotExists); + } + + @Override + public Table getTable(Identifier identifier) + throws TableNotExistException + { + return baseCatalog.getTable(identifier); + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException + { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(databaseName); + } + return hiveMetastore.getTables(databaseName).stream().map(t -> t.tableName().getTableName()).collect(Collectors.toList()); + } + + @Override + public PagedList listTablesPaged(String s, @Nullable Integer integer, @Nullable String s1, @Nullable String s2, @Nullable String s3) + throws DatabaseNotExistException + { + throw new UnsupportedOperationException("Alter table is not supported yet"); + } + + @Override + public PagedList listTableDetailsPaged(String s, @Nullable Integer integer, @Nullable String s1, @Nullable String s2, @Nullable String s3) + { + throw new UnsupportedOperationException("Alter table is not supported yet"); + } + + @Override + public void dropTable(Identifier identifier, boolean ignoreIfNotExists) + throws TableNotExistException + { + hiveMetastore.dropTable(identifier.getDatabaseName(), identifier.getTableName(), false); + baseCatalog.dropTable(identifier, ignoreIfNotExists); + } + + @Override + public void createTable(Identifier identifier, Schema schema, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException + { + if (!databaseExists(identifier.getDatabaseName())) { + throw new DatabaseNotExistException(identifier.getDatabaseName()); + } + + if (hiveMetastore.getTable(identifier.getDatabaseName(), identifier.getTableName()).isPresent()) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(identifier); + } + } + + hiveMetastore.createTable(io.trino.metastore.Table.builder() + .setDatabaseName(identifier.getDatabaseName()) + .setTableName(identifier.getTableName()) + .setDataColumns(toHiveColumns(schema.fields())) + .setOwner(Optional.of("paimon")) + .setTableType(EXTERNAL_TABLE.name()) + .withStorage(storage -> storage.setLocation(baseCatalog.options().get("warehouse") + "/" + identifier.getDatabaseName() + ".db/" + identifier.getTableName())) + .withStorage(storage -> storage.setStorageFormat(PAIMON_METASTORE_STORAGE_FORMAT)) + .setParameter("EXTERNAL", "TRUE") + .setParameters(schema.options()) + .build(), PrincipalPrivileges.NO_PRIVILEGES); + + baseCatalog.createTable(identifier, schema, ignoreIfExists); + } + + @Override + public void renameTable(Identifier identifier, Identifier newIdentifier, boolean b) + throws TableNotExistException, TableAlreadyExistException + { + hiveMetastore.renameTable(identifier.getDatabaseName(), identifier.getTableName(), newIdentifier.getDatabaseName(), newIdentifier.getTableName()); + baseCatalog.renameTable(identifier, newIdentifier, b); + } + + @Override + public void alterTable(Identifier identifier, List list, boolean b) + { + throw new UnsupportedOperationException("Alter table is not supported yet"); + } + + @Override + public void markDonePartitions(Identifier identifier, List> list) + { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitions(Identifier identifier) + throws TableNotExistException + { + return baseCatalog.listPartitions(identifier); + } + + @Override + public PagedList listPartitionsPaged(Identifier identifier, @Nullable Integer integer, @Nullable String s, @Nullable String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean supportsListObjectsPaged() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean supportsVersionManagement() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean commitSnapshot(Identifier identifier, @Nullable String s, Snapshot snapshot, List list) + { + throw new UnsupportedOperationException(); + } + + @Override + public Optional loadSnapshot(Identifier identifier) + { + throw new UnsupportedOperationException(); + } + + @Override + public Optional loadSnapshot(Identifier identifier, String s) + { + throw new UnsupportedOperationException(); + } + + @Override + public PagedList listSnapshotsPaged(Identifier identifier, @Nullable Integer integer, @Nullable String s) + { + throw new UnsupportedOperationException(); + } + + @Override + public void rollbackTo(Identifier identifier, Instant instant) + { + throw new UnsupportedOperationException(); + } + + @Override + public void createBranch(Identifier identifier, String s, @Nullable String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public void dropBranch(Identifier identifier, String s) + { + throw new UnsupportedOperationException(); + } + + @Override + public void fastForward(Identifier identifier, String s) + { + throw new UnsupportedOperationException(); + } + + @Override + public List listBranches(Identifier identifier) + { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartitions(Identifier identifier, List> list) + { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartitions(Identifier identifier, List> list) + { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitions(Identifier identifier, List list) + { + throw new UnsupportedOperationException(); + } + + @Override + public List listFunctions(String s) + { + throw new UnsupportedOperationException(); + } + + @Override + public Function getFunction(Identifier identifier) + { + throw new UnsupportedOperationException(); + } + + @Override + public void createFunction(Identifier identifier, Function function, boolean b) + { + throw new UnsupportedOperationException(); + } + + @Override + public void dropFunction(Identifier identifier, boolean b) + { + throw new UnsupportedOperationException(); + } + + @Override + public void alterFunction(Identifier identifier, List list, boolean b) + { + throw new UnsupportedOperationException(); + } + + @Override + public List authTableQuery(Identifier identifier, @Nullable List list) + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + throws Exception + { + baseCatalog.close(); + } +} diff --git a/pom.xml b/pom.xml index 451ecbf5f0cf..9f2247b09642 100644 --- a/pom.xml +++ b/pom.xml @@ -101,6 +101,7 @@ plugin/trino-openlineage plugin/trino-opensearch plugin/trino-oracle + plugin/trino-paimon plugin/trino-password-authenticators plugin/trino-pinot plugin/trino-postgresql @@ -201,6 +202,7 @@ 1.10.1 5.18.1 0.13.0 + 1.3.1 1.20.0 5.3.2 3.17.0 @@ -2033,6 +2035,12 @@ shaded-protobuf + + org.apache.paimon + paimon-bundle + ${dep.paimon.version} + + org.apache.parquet parquet-avro