diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 36daf72144b01..5a82d7f49f27c 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -1208,3 +1208,25 @@ jobs: FLINK_PROFILE: ${{ matrix.flinkProfile }} run: mvn test -Punit-tests -Djava17 -Djava.version=17 -D"$SCALA_PROFILE" -D"$FLINK_PROFILE" -pl hudi-examples/hudi-examples-flink $MVN_ARGS + + test-hudi-trino-plugin: + runs-on: ubuntu-latest + steps: + - name: Checkout repository + uses: actions/checkout@v3 + - name: Set up JDK 23 + uses: actions/setup-java@v3 + with: + # Note: We are not caching here again, as we want to use the .m2 repository populated by + # the previous step + java-version: '23' + distribution: 'temurin' + architecture: x64 + - name: Build hudi-trino-plugin with JDK 23 + working-directory: ./hudi-trino-plugin + run: + mvn clean install -DskipTests + - name: Test hudi-trino-plugin with JDK 23 + working-directory: ./hudi-trino-plugin + run: + mvn test diff --git a/hudi-trino-plugin/.mvn/modernizer/violations-production-code-only.xml b/hudi-trino-plugin/.mvn/modernizer/violations-production-code-only.xml new file mode 100644 index 0000000000000..4653d63fafc28 --- /dev/null +++ b/hudi-trino-plugin/.mvn/modernizer/violations-production-code-only.xml @@ -0,0 +1,86 @@ + + + + + java/util/concurrent/ThreadPoolExecutor."<init>":(IIJLjava/util/concurrent/TimeUnit;Ljava/util/concurrent/BlockingQueue;)V + 1.1 + Use constructor that takes ThreadFactory and name the threads + + + + java/util/concurrent/ThreadPoolExecutor."<init>":(IIJLjava/util/concurrent/TimeUnit;Ljava/util/concurrent/BlockingQueue;Ljava/util/concurrent/RejectedExecutionHandler;)V + 1.1 + Use constructor that takes ThreadFactory and name the threads + + + + java/util/concurrent/ScheduledThreadPoolExecutor."<init>":(I)V + 1.1 + Use constructor that takes ThreadFactory and name the threads + + + + java/util/concurrent/ScheduledThreadPoolExecutor."<init>":(ILjava/util/concurrent/RejectedExecutionHandler;)V + 1.1 + Use constructor that takes ThreadFactory and name the threads + + + + java/util/concurrent/Executors.newFixedThreadPool:(I)Ljava/util/concurrent/ExecutorService; + 1.1 + Use factory method that takes ThreadFactory and name the threads + + + + java/util/concurrent/Executors.newWorkStealingPool:()Ljava/util/concurrent/ExecutorService; + 1.1 + Use factory method that takes ThreadFactory and name the threads + + + + java/util/concurrent/Executors.newWorkStealingPool:(I)Ljava/util/concurrent/ExecutorService; + 1.1 + Use factory method that takes ThreadFactory and name the threads + + + + java/util/concurrent/Executors.newSingleThreadExecutor:()Ljava/util/concurrent/ExecutorService; + 1.1 + Use factory method that takes ThreadFactory and name the threads + + + + java/util/concurrent/Executors.newCachedThreadPool:()Ljava/util/concurrent/ExecutorService; + 1.1 + Use factory method that takes ThreadFactory and name the threads + + + + java/util/concurrent/Executors.newSingleThreadScheduledExecutor:()Ljava/util/concurrent/ScheduledExecutorService; + 1.1 + Use factory method that takes ThreadFactory and name the threads + + + + java/util/concurrent/Executors.newScheduledThreadPool:(I)Ljava/util/concurrent/ScheduledExecutorService; + 1.1 + Use factory method that takes ThreadFactory and name the threads + + diff --git a/hudi-trino-plugin/.mvn/modernizer/violations.xml b/hudi-trino-plugin/.mvn/modernizer/violations.xml new file mode 100644 index 0000000000000..eae8ea39b5916 --- /dev/null +++ b/hudi-trino-plugin/.mvn/modernizer/violations.xml @@ -0,0 +1,365 @@ + + + + + java/lang/Class.newInstance:()Ljava/lang/Object; + 1.1 + Prefer Class.getConstructor().newInstance() + + + + java/lang/String."<init>":([B)V + 1.1 + Prefer new String(byte[], Charset) + + + + java/lang/String.getBytes:()[B + 1.1 + Prefer String.getBytes(Charset) + + + + java/lang/String.toString:()Ljava/lang/String; + 1.1 + Call to toString() is redundant + + + + + java/io/File.toString:()Ljava/lang/String; + 1.1 + Prefer File.getPath() + + + + java/lang/Thread$Builder.factory:()Ljava/util/concurrent/ThreadFactory; + 1.1 + Use io.airlift.concurrent.Threads's thread factories, as the set thread context class loader + + + java/lang/Thread$Builder$OfPlatform.factory:()Ljava/util/concurrent/ThreadFactory; + 1.1 + Use io.airlift.concurrent.Threads's thread factories, as the set thread context class loader + + + java/lang/Thread$Builder$OfVirtual.factory:()Ljava/util/concurrent/ThreadFactory; + 1.1 + Use io.airlift.concurrent.Threads's thread factories, as the set thread context class loader + + + + com/google/common/primitives/Ints.checkedCast:(J)I + 1.8 + Prefer Math.toIntExact(long) + + + + com/google/common/collect/ImmutableMap$Builder.build:()Lcom/google/common/collect/ImmutableMap; + 1.8 + Use buildOrThrow() instead, as it makes it clear that it will throw on duplicated values + + + com/google/common/collect/ImmutableTable$Builder.build:()Lcom/google/common/collect/ImmutableTable; + 1.8 + Use buildOrThrow() instead, as it makes it clear that it will throw on duplicated values + + + + com/google/common/collect/ImmutableBiMap$Builder."<init>":()V + 1.8 + Use builder() static factory method instead + + + com/google/common/collect/ImmutableList$Builder."<init>":()V + 1.8 + Use builder() static factory method instead + + + com/google/common/collect/ImmutableMap$Builder."<init>":()V + 1.8 + Use builder() static factory method instead + + + com/google/common/collect/ImmutableMultimap$Builder."<init>":()V + 1.8 + Use builder() static factory method instead + + + com/google/common/collect/ImmutableMultiset$Builder."<init>":()V + 1.8 + Use builder() static factory method instead + + + com/google/common/collect/ImmutableSet$Builder."<init>":()V + 1.8 + Use builder() static factory method instead + + + com/google/common/collect/ImmutableSortedMap$Builder."<init>":()V + 1.8 + Use orderedBy() static factory method instead + + + com/google/common/collect/ImmutableSortedSet$Builder."<init>":()V + 1.8 + Use orderedBy() static factory method instead + + + com/google/common/collect/ImmutableTable$Builder."<init>":()V + 1.8 + Use builder() static factory method instead + + + + com/google/common/cache/CacheBuilder.build:()Lcom/google/common/cache/Cache; + 1.8 + Guava Cache has concurrency issues around invalidation and ongoing loads. Use EvictableCacheBuilder or SafeCaches to build caches. + See https://github.com/trinodb/trino/issues/10512 for more information and see https://github.com/trinodb/trino/issues/10512#issuecomment-1016221168 + for why Caffeine does not solve the problem. + + + + com/google/common/cache/CacheBuilder.build:(Lcom/google/common/cache/CacheLoader;)Lcom/google/common/cache/LoadingCache; + 1.8 + Guava LoadingCache has concurrency issues around invalidation and ongoing loads. Use EvictableCacheBuilder or SafeCaches to build caches. + See https://github.com/trinodb/trino/issues/10512 for more information and see https://github.com/trinodb/trino/issues/10512#issuecomment-1016221168 + for why Caffeine does not solve the problem. + + + + org/testng/Assert.assertEquals:(Ljava/lang/Iterable;Ljava/lang/Iterable;)V + 1.8 + Use AssertJ or QueryAssertions due to TestNG #543 + + + + org/testng/Assert.assertEquals:(Ljava/lang/Iterable;Ljava/lang/Iterable;Ljava/lang/String;)V + 1.8 + Use AssertJ or QueryAssertions due to TestNG #543 + + + + org/testng/Assert.assertThrows:(Lorg/testng/Assert$ThrowingRunnable;)V + 1.8 + Use AssertJ's assertThatThrownBy, see https://github.com/trinodb/trino/issues/5320 for rationale + + + + org/testng/Assert.assertThrows:(Ljava/lang/Class;Lorg/testng/Assert$ThrowingRunnable;)V + 1.8 + Use AssertJ's assertThatThrownBy, see https://github.com/trinodb/trino/issues/5320 for rationale + + + + com/amazonaws/services/glue/model/Table.getStorageDescriptor:()Lcom/amazonaws/services/glue/model/StorageDescriptor; + 1.1 + Storage descriptor is nullable in Glue model, which is too easy to forget about. Prefer GlueToTrinoConverter.getStorageDescriptor + + + + com/amazonaws/services/glue/model/Table.getTableType:()Ljava/lang/String; + 1.1 + Table type is nullable in Glue model, which is too easy to forget about. Prefer GlueToTrinoConverter.getTableType + + + + com/amazonaws/services/glue/model/Column.getParameters:()Ljava/util/Map; + 1.1 + Column parameters map is nullable in Glue model, which is too easy to forget about. Prefer GlueToTrinoConverter.getColumnParameters + + + + com/amazonaws/services/glue/model/Table.getParameters:()Ljava/util/Map; + 1.1 + Table parameters map is nullable in Glue model, which is too easy to forget about. Prefer GlueToTrinoConverter.getTableParameters + + + + com/amazonaws/services/glue/model/Partition.getParameters:()Ljava/util/Map; + 1.1 + Partition parameters map is nullable in Glue model, which is too easy to forget about. Prefer GlueToTrinoConverter.getPartitionParameters + + + + com/amazonaws/services/glue/model/SerDeInfo.getParameters:()Ljava/util/Map; + 1.1 + SerDeInfo parameters map is nullable in Glue model, which is too easy to forget about. Prefer GlueToTrinoConverter.getSerDeInfoParameters + + + + org/apache/hadoop/fs/FileSystem.close:()V + 1.1 + Hadoop FileSystem instances are shared and should not be closed + + + + java/util/TimeZone.getTimeZone:(Ljava/lang/String;)Ljava/util/TimeZone; + 1.8 + Avoid TimeZone.getTimeZone as it returns GMT for a zone not supported by the JVM. Use TimeZone.getTimeZone(ZoneId.of(..)) instead, or TimeZone.getTimeZone(..., false). + + + + org/joda/time/DateTimeZone.toTimeZone:()Ljava/util/TimeZone; + 1.8 + Avoid DateTimeZone.toTimeZone as it returns GMT for a zone not supported by the JVM. Use TimeZone.getTimeZone(ZoneId.of(dtz.getId())) instead. + + + + com/esri/core/geometry/ogc/OGCGeometry.equals:(Lcom/esri/core/geometry/ogc/OGCGeometry;)Z + 1.6 + Prefer OGCGeometry.Equals(OGCGeometry) + + + + com/esri/core/geometry/ogc/OGCGeometry.equals:(Ljava/lang/Object;)Z + 1.6 + Prefer OGCGeometry.Equals(OGCGeometry) + + + + io/airlift/units/DataSize."<init>":(DLio/airlift/units/DataSize$Unit;)V + 1.8 + Use io.airlift.units.DataSize.of(long, DataSize.Unit) + + + + io/airlift/units/DataSize.succinctDataSize:(DLio/airlift/units/DataSize$Unit;)Lio/airlift/units/DataSize; + 1.8 + Use io.airlift.units.DataSize.of(long, DataSize.Unit).succinct() -- Note that succinct conversion only affects toString() results + + + + io/airlift/units/DataSize.getValue:()D + 1.8 + Use io.airlift.units.DataSize.toBytes() and Unit.inBytes() for conversion + + + + io/airlift/units/DataSize.getValue:(Lio/airlift/units/DataSize$Unit;)D + 1.8 + Use io.airlift.units.DataSize.toBytes() and Unit.inBytes() for conversion + + + + io/airlift/units/DataSize.roundTo:(Lio/airlift/units/DataSize$Unit;)J + 1.8 + Method is deprecated for removal + + + + io/airlift/units/DataSize.convertTo:(Lio/airlift/units/DataSize$Unit;)Lio/airlift/units/DataSize; + 1.8 + Use io.airlift.units.DataSize.to(DataSize.Unit) + + + + io/airlift/units/DataSize.convertToMostSuccinctDataSize:()Lio/airlift/units/DataSize; + 1.8 + Use io.airlift.units.DataSize.succinct() + + + + io/airlift/testing/Closeables.closeQuietly:([Ljava/io/Closeable;)V + 1.0 + Use Closeables.closeAll() or Closer. + + + + com/google/inject/util/Modules.combine:(Ljava/lang/Iterable;)Lcom/google/inject/Module; + 1.8 + Use io.airlift.configuration.ConfigurationAwareModule.combine + + + + com/google/inject/util/Modules.combine:([Lcom/google/inject/Module;)Lcom/google/inject/Module; + 1.8 + Use io.airlift.configuration.ConfigurationAwareModule.combine + + + + io/jsonwebtoken/Jwts.builder:()Lio/jsonwebtoken/JwtBuilder; + 1.8 + Use io.trino.server.security.jwt.JwtsUtil or equivalent + + + + io/jsonwebtoken/Jwts.parserBuilder:()Lio/jsonwebtoken/JwtParserBuilder; + 1.8 + Use io.trino.server.security.jwt.JwtsUtil or equivalent + + + + org/openjdk/jol/info/ClassLayout.instanceSize:()J + 1.8 + Use io.airlift.slice.SizeOf.instanceSize + + + + org/testng/annotations/BeforeTest + 1.8 + Prefer org.testng.annotations.BeforeClass + + + + org/testng/annotations/AfterTest + 1.8 + Prefer org.testng.annotations.AfterClass + + + + com/fasterxml/jackson/core/JsonFactory."<init>":()V + 1.8 + Use io.trino.plugin.base.util.JsonUtils.jsonFactory() + + + + com/fasterxml/jackson/core/JsonFactoryBuilder."<init>":()V + 1.8 + Use io.trino.plugin.base.util.JsonUtils.jsonFactoryBuilder() instead + + + + software/amazon/awssdk/services/glue/model/Table.tableType:()Ljava/lang/String; + 1.8 + Table type is nullable in Glue model, which is too easy to forget about. Prefer GlueConverter.getTableTypeNullable + + + + org/assertj/core/util/Files.newTemporaryFile:()Ljava/io/File; + 1.1 + Use @TempDir instead + + + org/assertj/core/util/Files.newTemporaryFolder:()Ljava/io/File; + 1.1 + Use @TempDir instead + + + org/assertj/core/util/Files.temporaryFolder:()Ljava/io/File; + 1.1 + Use @TempDir instead + + + org/assertj/core/util/Files.temporaryFolderPath:()Ljava/lang/String; + 1.1 + Use @TempDir instead + + diff --git a/hudi-trino-plugin/pom.xml b/hudi-trino-plugin/pom.xml new file mode 100644 index 0000000000000..d819b3093e239 --- /dev/null +++ b/hudi-trino-plugin/pom.xml @@ -0,0 +1,460 @@ + + + + 4.0.0 + + + io.trino + trino-root + 472 + + + + + trino-hudi + trino-plugin + Trino - Hudi connector + + + true + 1.0.2 + + + + + + com.esotericsoftware + kryo + 4.0.2 + + + + com.google.errorprone + error_prone_annotations + true + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + io.airlift + bootstrap + + + + io.airlift + concurrent + + + + io.airlift + configuration + + + + io.airlift + json + + + + io.airlift + log + + + + io.airlift + units + + + + io.trino + trino-filesystem + + + + io.trino + trino-filesystem-manager + + + + io.trino + trino-hive + + + + io.trino + trino-memory-context + + + + io.trino + trino-metastore + + + + io.trino + trino-parquet + + + + io.trino + trino-plugin-toolkit + + + + jakarta.validation + jakarta.validation-api + + + + joda-time + joda-time + + + + org.apache.avro + avro + + + + org.apache.hudi + hudi-common + ${dep.hudi.version} + + + io.dropwizard.metrics + * + + + org.apache.hbase + * + + + org.apache.httpcomponents + * + + + org.apache.orc + * + + + + + + org.apache.hudi + hudi-io + ${dep.hudi.version} + + + com.google.protobuf + protobuf-java + + + + + + org.apache.parquet + parquet-column + + + + org.weakref + jmxutils + + + + 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-context + provided + + + + io.trino + trino-spi + provided + + + + org.openjdk.jol + jol-core + provided + + + + com.github.ben-manes.caffeine + caffeine + runtime + + + + io.airlift + log-manager + runtime + + + + io.dropwizard.metrics + metrics-core + runtime + + + + io.trino + trino-hive-formats + runtime + + + + org.jetbrains + annotations + runtime + + + + 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 + + + + io.trino + trino-main + test-jar + test + + + + io.trino + trino-parser + test + + + + io.trino + trino-spi + 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.hadoop + hadoop-apache + test + + + + io.trino.tpch + tpch + test + + + + org.apache.hudi + hudi-client-common + ${dep.hudi.version} + test + + + * + * + + + + + + org.apache.hudi + hudi-hadoop-common + ${dep.hudi.version} + test + + + * + * + + + + + + org.apache.hudi + hudi-java-client + ${dep.hudi.version} + test + + + org.apache.hudi + * + + + + + + org.apache.parquet + parquet-avro + test + + + + org.apache.parquet + parquet-hadoop + test + + + + org.assertj + assertj-core + test + + + + org.json + json + 20250107 + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.jupiter + junit-jupiter-params + test + + + + + + + org.basepom.maven + duplicate-finder-maven-plugin + + + + log4j.properties + log4j-surefire.properties + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/ForHudiSplitManager.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/ForHudiSplitManager.java new file mode 100644 index 0000000000000..0f1987594a1ac --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/ForHudiSplitManager.java @@ -0,0 +1,29 @@ +/* + * 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.hudi; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@BindingAnnotation +public @interface ForHudiSplitManager {} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/ForHudiSplitSource.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/ForHudiSplitSource.java new file mode 100644 index 0000000000000..801b1e0309407 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/ForHudiSplitSource.java @@ -0,0 +1,29 @@ +/* + * 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.hudi; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@BindingAnnotation +public @interface ForHudiSplitSource {} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiBaseFileOnlyPageSource.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiBaseFileOnlyPageSource.java new file mode 100644 index 0000000000000..1180638a7cc47 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiBaseFileOnlyPageSource.java @@ -0,0 +1,126 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.util.SynthesizedColumnHandler; +import io.trino.spi.Page; +import io.trino.spi.block.Block; +import io.trino.spi.connector.ConnectorPageSource; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +/** + * This page source is for reading data columns in the parquet format. + * This page source also avoids costly avro IndexRecord serialization. + */ +public class HudiBaseFileOnlyPageSource + implements ConnectorPageSource +{ + private final ConnectorPageSource dataPageSource; + private final List allOutputColumns; + private final SynthesizedColumnHandler synthesizedColumnHandler; + // Maps output channel to physical source channel, or -1 if synthesized + private final int[] physicalSourceChannelMap; + + public HudiBaseFileOnlyPageSource( + ConnectorPageSource dataPageSource, + List allOutputColumns, + // Columns provided by dataPageSource + List dataColumns, + // Handler to manage synthesized/virtual in Hudi tables such as partition columns and metadata, i.e. file size (not hudi metadata) + SynthesizedColumnHandler synthesizedColumnHandler) + { + this.dataPageSource = requireNonNull(dataPageSource, "dataPageSource is null"); + this.allOutputColumns = ImmutableList.copyOf(requireNonNull(allOutputColumns, "allOutputColumns is null")); + this.synthesizedColumnHandler = requireNonNull(synthesizedColumnHandler, "synthesizedColumnHandler is null"); + + // Create a mapping from the channel index in the output page to the channel index in the physicalDataPageSource's page + this.physicalSourceChannelMap = new int[allOutputColumns.size()]; + Map physicalColumnNameToChannel = new HashMap<>(); + for (int i = 0; i < dataColumns.size(); i++) { + physicalColumnNameToChannel.put(dataColumns.get(i).getName().toLowerCase(Locale.ENGLISH), i); + } + + for (int i = 0; i < allOutputColumns.size(); i++) { + this.physicalSourceChannelMap[i] = physicalColumnNameToChannel.getOrDefault(allOutputColumns.get(i).getName().toLowerCase(Locale.ENGLISH), -1); + } + } + + @Override + public long getCompletedBytes() + { + return dataPageSource.getCompletedBytes(); + } + + @Override + public long getReadTimeNanos() + { + return dataPageSource.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return dataPageSource.isFinished(); + } + + @Override + public Page getNextPage() + { + Page physicalSourcePage = dataPageSource.getNextPage(); + if (physicalSourcePage == null) { + return null; + } + + int positionCount = physicalSourcePage.getPositionCount(); + if (positionCount == 0 && synthesizedColumnHandler.getSynthesizedColumnCount() == 0) { + // If only physical columns and page is empty + return physicalSourcePage; + } + + Block[] outputBlocks = new Block[allOutputColumns.size()]; + for (int i = 0; i < allOutputColumns.size(); i++) { + HiveColumnHandle outputColumn = allOutputColumns.get(i); + if (physicalSourceChannelMap[i] != -1) { + outputBlocks[i] = physicalSourcePage.getBlock(physicalSourceChannelMap[i]); + } + else { + // Column is synthesized + outputBlocks[i] = synthesizedColumnHandler.createRleSynthesizedBlock(outputColumn, positionCount); + } + } + return new Page(outputBlocks); + } + + @Override + public long getMemoryUsage() + { + return dataPageSource.getMemoryUsage(); + } + + @Override + public void close() + throws IOException + { + dataPageSource.close(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConfig.java new file mode 100644 index 0000000000000..5138caea39c8d --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -0,0 +1,410 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.DefunctConfig; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; +import jakarta.validation.constraints.DecimalMax; +import jakarta.validation.constraints.DecimalMin; +import jakarta.validation.constraints.Min; +import jakarta.validation.constraints.NotNull; + +import java.util.List; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static java.util.Locale.ENGLISH; +import static java.util.concurrent.TimeUnit.SECONDS; + +@DefunctConfig({ + "hudi.min-partition-batch-size", + "hudi.max-partition-batch-size" +}) +public class HudiConfig +{ + private List columnsToHide = ImmutableList.of(); + private boolean tableStatisticsEnabled = true; + private int tableStatisticsExecutorParallelism = 4; + private boolean metadataEnabled; + private boolean shouldUseParquetColumnNames = true; + private boolean shouldUseParquetColumnIndex; + private boolean sizeBasedSplitWeightsEnabled = true; + private DataSize standardSplitWeightSize = DataSize.of(128, MEGABYTE); + private double minimumAssignedSplitWeight = 0.05; + private DataSize targetSplitSize = DataSize.of(128, MEGABYTE); + private int maxSplitsPerSecond = Integer.MAX_VALUE; + private int maxOutstandingSplits = 1000; + private int splitLoaderParallelism = 4; + private int splitGeneratorParallelism = 4; + private long perTransactionMetastoreCacheMaximumSize = 2000; + private boolean queryPartitionFilterRequired; + private boolean ignoreAbsentPartitions; + private Duration dynamicFilteringWaitTimeout = new Duration(1, SECONDS); + + // Internal configuration for debugging and testing + private boolean isRecordLevelIndexEnabled = true; + private boolean isSecondaryIndexEnabled = true; + private boolean isColumnStatsIndexEnabled = true; + private boolean isPartitionStatsIndexEnabled = true; + private Duration columnStatsWaitTimeout = new Duration(1, SECONDS); + private Duration recordIndexWaitTimeout = new Duration(2, SECONDS); + private Duration secondaryIndexWaitTimeout = new Duration(2, SECONDS); + + public List getColumnsToHide() + { + return columnsToHide; + } + + @Config("hudi.columns-to-hide") + @ConfigDescription("List of column names that will be hidden from the query output. " + + "It can be used to hide Hudi meta fields. By default, no fields are hidden.") + public HudiConfig setColumnsToHide(List columnsToHide) + { + this.columnsToHide = columnsToHide.stream() + .map(s -> s.toLowerCase(ENGLISH)) + .collect(toImmutableList()); + return this; + } + + @Config("hudi.table-statistics-enabled") + @ConfigDescription("Enable table statistics for query planning.") + public HudiConfig setTableStatisticsEnabled(boolean tableStatisticsEnabled) + { + this.tableStatisticsEnabled = tableStatisticsEnabled; + return this; + } + + public boolean isTableStatisticsEnabled() + { + return this.tableStatisticsEnabled; + } + + @Min(1) + public int getTableStatisticsExecutorParallelism() + { + return tableStatisticsExecutorParallelism; + } + + @Config("hudi.table-statistics-executor-parallelism") + @ConfigDescription("Number of threads to asynchronously generate table statistics.") + public HudiConfig setTableStatisticsExecutorParallelism(int parallelism) + { + this.tableStatisticsExecutorParallelism = parallelism; + return this; + } + + @Config("hudi.metadata-enabled") + @ConfigDescription("Fetch the list of file names and sizes from Hudi metadata table rather than storage.") + public HudiConfig setMetadataEnabled(boolean metadataEnabled) + { + this.metadataEnabled = metadataEnabled; + return this; + } + + public boolean isMetadataEnabled() + { + return this.metadataEnabled; + } + + @Config("hudi.parquet.use-column-names") + @ConfigDescription("Access Parquet columns using names from the file. If disabled, then columns are accessed using index." + + "Only applicable to Parquet file format.") + public HudiConfig setUseParquetColumnNames(boolean shouldUseParquetColumnNames) + { + this.shouldUseParquetColumnNames = shouldUseParquetColumnNames; + return this; + } + + public boolean getUseParquetColumnNames() + { + return this.shouldUseParquetColumnNames; + } + + @Config("hudi.parquet.use-column-index") + @ConfigDescription("Enable using Parquet column indexes") + public HudiConfig setUseParquetColumnIndex(boolean shouldUseParquetColumnIndex) + { + this.shouldUseParquetColumnIndex = shouldUseParquetColumnIndex; + return this; + } + + public boolean isUseParquetColumnIndex() + { + return this.shouldUseParquetColumnIndex; + } + + @Config("hudi.size-based-split-weights-enabled") + @ConfigDescription("Unlike uniform splitting, size-based splitting ensures that each batch of splits has enough data to process. " + + "By default, it is enabled to improve performance.") + public HudiConfig setSizeBasedSplitWeightsEnabled(boolean sizeBasedSplitWeightsEnabled) + { + this.sizeBasedSplitWeightsEnabled = sizeBasedSplitWeightsEnabled; + return this; + } + + public boolean isSizeBasedSplitWeightsEnabled() + { + return sizeBasedSplitWeightsEnabled; + } + + @Config("hudi.standard-split-weight-size") + @ConfigDescription("The split size corresponding to the standard weight (1.0) " + + "when size based split weights are enabled.") + public HudiConfig setStandardSplitWeightSize(DataSize standardSplitWeightSize) + { + this.standardSplitWeightSize = standardSplitWeightSize; + return this; + } + + @NotNull + public DataSize getStandardSplitWeightSize() + { + return standardSplitWeightSize; + } + + @Config("hudi.minimum-assigned-split-weight") + @ConfigDescription("Minimum weight that a split can be assigned when size based split weights are enabled.") + public HudiConfig setMinimumAssignedSplitWeight(double minimumAssignedSplitWeight) + { + this.minimumAssignedSplitWeight = minimumAssignedSplitWeight; + return this; + } + + @DecimalMax("1") + @DecimalMin(value = "0", inclusive = false) + public double getMinimumAssignedSplitWeight() + { + return minimumAssignedSplitWeight; + } + + @Config("hudi.target-split-size") + @ConfigDescription("The target split size") + public HudiConfig setTargetSplitSize(DataSize targetSplitSize) + { + this.targetSplitSize = targetSplitSize; + return this; + } + + @NotNull + public DataSize getTargetSplitSize() + { + return targetSplitSize; + } + + @Min(1) + public int getMaxSplitsPerSecond() + { + return maxSplitsPerSecond; + } + + @Config("hudi.max-splits-per-second") + @ConfigDescription("Rate at which splits are enqueued for processing. The queue will throttle if this rate limit is breached.") + public HudiConfig setMaxSplitsPerSecond(int maxSplitsPerSecond) + { + this.maxSplitsPerSecond = maxSplitsPerSecond; + return this; + } + + @Min(1) + public int getMaxOutstandingSplits() + { + return maxOutstandingSplits; + } + + @Config("hudi.max-outstanding-splits") + @ConfigDescription("Maximum outstanding splits in a batch enqueued for processing.") + public HudiConfig setMaxOutstandingSplits(int maxOutstandingSplits) + { + this.maxOutstandingSplits = maxOutstandingSplits; + return this; + } + + @Min(1) + public int getSplitGeneratorParallelism() + { + return splitGeneratorParallelism; + } + + @Config("hudi.split-generator-parallelism") + @ConfigDescription("Number of threads to generate splits from partitions.") + public HudiConfig setSplitGeneratorParallelism(int splitGeneratorParallelism) + { + this.splitGeneratorParallelism = splitGeneratorParallelism; + return this; + } + + @Min(1) + public int getSplitLoaderParallelism() + { + return splitLoaderParallelism; + } + + @Config("hudi.split-loader-parallelism") + @ConfigDescription("Number of threads to run background split loader. A single background split loader is needed per query.") + public HudiConfig setSplitLoaderParallelism(int splitLoaderParallelism) + { + this.splitLoaderParallelism = splitLoaderParallelism; + return this; + } + + @Min(1) + public long getPerTransactionMetastoreCacheMaximumSize() + { + return perTransactionMetastoreCacheMaximumSize; + } + + @Config("hudi.per-transaction-metastore-cache-maximum-size") + public HudiConfig setPerTransactionMetastoreCacheMaximumSize(long perTransactionMetastoreCacheMaximumSize) + { + this.perTransactionMetastoreCacheMaximumSize = perTransactionMetastoreCacheMaximumSize; + return this; + } + + @Config("hudi.query-partition-filter-required") + @ConfigDescription("Require a filter on at least one partition column") + public HudiConfig setQueryPartitionFilterRequired(boolean queryPartitionFilterRequired) + { + this.queryPartitionFilterRequired = queryPartitionFilterRequired; + return this; + } + + public boolean isQueryPartitionFilterRequired() + { + return queryPartitionFilterRequired; + } + + @Config("hudi.ignore-absent-partitions") + public HudiConfig setIgnoreAbsentPartitions(boolean ignoreAbsentPartitions) + { + this.ignoreAbsentPartitions = ignoreAbsentPartitions; + return this; + } + + public boolean isIgnoreAbsentPartitions() + { + return ignoreAbsentPartitions; + } + + @Config("hudi.index.record-level-index-enabled") + @ConfigDescription("Internal configuration to control whether record level index is enabled for debugging/testing.") + public HudiConfig setRecordLevelIndexEnabled(boolean isRecordLevelIndexEnabled) + { + this.isRecordLevelIndexEnabled = isRecordLevelIndexEnabled; + return this; + } + + public boolean isRecordLevelIndexEnabled() + { + return isRecordLevelIndexEnabled; + } + + @Config("hudi.index.secondary-index-enabled") + @ConfigDescription("Internal configuration to control whether secondary index is enabled for debugging/testing.") + public HudiConfig setSecondaryIndexEnabled(boolean isSecondaryIndexEnabled) + { + this.isSecondaryIndexEnabled = isSecondaryIndexEnabled; + return this; + } + + public boolean isSecondaryIndexEnabled() + { + return isSecondaryIndexEnabled; + } + + @Config("hudi.index.column-stats-index-enabled") + @ConfigDescription("Internal configuration to control whether column stats index is enabled for debugging/testing.") + public HudiConfig setColumnStatsIndexEnabled(boolean isColumnStatsIndexEnabled) + { + this.isColumnStatsIndexEnabled = isColumnStatsIndexEnabled; + return this; + } + + public boolean isColumnStatsIndexEnabled() + { + return isColumnStatsIndexEnabled; + } + + @Config("hudi.index.partition-stats-index-enabled") + @ConfigDescription("Internal configuration to control whether partition stats index is enabled for debugging/testing.") + public HudiConfig setPartitionStatsIndexEnabled(boolean isPartitionStatsIndexEnabled) + { + this.isPartitionStatsIndexEnabled = isPartitionStatsIndexEnabled; + return this; + } + + public boolean isPartitionStatsIndexEnabled() + { + return isPartitionStatsIndexEnabled; + } + + @Config("hudi.dynamic-filtering.wait-timeout") + @ConfigDescription("Maximum timeout to wait for dynamic filtering, e.g. 1000ms, 20s, 2m, 1h") + public HudiConfig setDynamicFilteringWaitTimeout(Duration dynamicFilteringWaitTimeout) + { + this.dynamicFilteringWaitTimeout = dynamicFilteringWaitTimeout; + return this; + } + + @NotNull + public Duration getDynamicFilteringWaitTimeout() + { + return dynamicFilteringWaitTimeout; + } + + @Config("hudi.index.column-stats.wait-timeout") + @ConfigDescription("Maximum timeout to wait for loading column stats, e.g. 1000ms, 20s") + public HudiConfig setColumnStatsWaitTimeout(Duration columnStatusWaitTimeout) + { + this.columnStatsWaitTimeout = columnStatusWaitTimeout; + return this; + } + + @NotNull + public Duration getColumnStatsWaitTimeout() + { + return columnStatsWaitTimeout; + } + + @Config("hudi.index.record-index.wait-timeout") + @ConfigDescription("Maximum timeout to wait for loading record index, e.g. 1000ms, 20s") + public HudiConfig setRecordIndexWaitTimeout(Duration recordIndexWaitTimeout) + { + this.recordIndexWaitTimeout = recordIndexWaitTimeout; + return this; + } + + @NotNull + public Duration getRecordIndexWaitTimeout() + { + return recordIndexWaitTimeout; + } + + @Config("hudi.index.secondary-index.wait-timeout") + @ConfigDescription("Maximum timeout to wait for loading secondary index, e.g. 1000ms, 20s") + public HudiConfig setSecondaryIndexWaitTimeout(Duration secondaryIndexWaitTimeout) + { + this.secondaryIndexWaitTimeout = secondaryIndexWaitTimeout; + return this; + } + + @NotNull + public Duration getSecondaryIndexWaitTimeout() + { + return secondaryIndexWaitTimeout; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConnector.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConnector.java new file mode 100644 index 0000000000000..072005cd3ffc4 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConnector.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.hudi; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.inject.Injector; +import io.airlift.bootstrap.LifeCycleManager; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.HiveTransactionHandle; +import io.trino.spi.classloader.ThreadContextClassLoader; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorNodePartitioningProvider; +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.connector.SystemTable; +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.ImmutableList.toImmutableList; +import static io.trino.spi.transaction.IsolationLevel.SERIALIZABLE; +import static io.trino.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +public class HudiConnector + implements Connector +{ + private final Injector injector; + private final LifeCycleManager lifeCycleManager; + private final HudiTransactionManager transactionManager; + private final ConnectorSplitManager splitManager; + private final ConnectorPageSourceProvider pageSourceProvider; + private final ConnectorNodePartitioningProvider nodePartitioningProvider; + private final Set systemTables; + private final List> sessionProperties; + private final List> tableProperties; + + public HudiConnector( + Injector injector, + LifeCycleManager lifeCycleManager, + HudiTransactionManager transactionManager, + ConnectorSplitManager splitManager, + ConnectorPageSourceProvider pageSourceProvider, + ConnectorNodePartitioningProvider nodePartitioningProvider, + Set systemTables, + Set sessionPropertiesProviders, + List> tableProperties) + { + 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.nodePartitioningProvider = requireNonNull(nodePartitioningProvider, "nodePartitioningProvider is null"); + this.systemTables = ImmutableSet.copyOf(requireNonNull(systemTables, "systemTables is null")); + this.sessionProperties = requireNonNull(sessionPropertiesProviders, "sessionPropertiesProviders is null").stream() + .flatMap(sessionPropertiesProvider -> sessionPropertiesProvider.getSessionProperties().stream()) + .collect(toImmutableList()); + this.tableProperties = ImmutableList.copyOf(requireNonNull(tableProperties, "tableProperties is null")); + } + + @Override + public ConnectorMetadata getMetadata(ConnectorSession session, ConnectorTransactionHandle transactionHandle) + { + ConnectorMetadata metadata = transactionManager.get(transactionHandle, session.getIdentity()); + return new ClassLoaderSafeConnectorMetadata(metadata, getClass().getClassLoader()); + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return pageSourceProvider; + } + + @Override + public ConnectorNodePartitioningProvider getNodePartitioningProvider() + { + return nodePartitioningProvider; + } + + @Override + public Set getSystemTables() + { + return systemTables; + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } + + @Override + public List> getTableProperties() + { + return tableProperties; + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) + { + checkConnectorSupports(SERIALIZABLE, isolationLevel); + ConnectorTransactionHandle transaction = new HiveTransactionHandle(true); + try (ThreadContextClassLoader _ = new ThreadContextClassLoader(getClass().getClassLoader())) { + transactionManager.put(transaction); + } + return transaction; + } + + @Override + public void commit(ConnectorTransactionHandle transaction) + { + transactionManager.commit(transaction); + } + + @Override + public void rollback(ConnectorTransactionHandle transaction) + { + transactionManager.rollback(transaction); + } + + @Override + public final void shutdown() + { + lifeCycleManager.stop(); + } + + @VisibleForTesting + public Injector getInjector() + { + return injector; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java new file mode 100644 index 0000000000000..f7dd96d767991 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.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.hudi; + +import com.google.common.collect.ImmutableSet; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.json.JsonModule; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.trace.Tracer; +import io.trino.filesystem.manager.FileSystemModule; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.trino.plugin.base.classloader.ClassLoaderSafeNodePartitioningProvider; +import io.trino.plugin.base.jmx.MBeanServerModule; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.NodeVersion; +import io.trino.plugin.hive.metastore.HiveMetastoreModule; +import io.trino.spi.NodeManager; +import io.trino.spi.catalog.CatalogName; +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 io.trino.spi.connector.ConnectorNodePartitioningProvider; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.type.TypeManager; +import org.weakref.jmx.guice.MBeanModule; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.inject.util.Modules.EMPTY_MODULE; +import static io.trino.plugin.base.Versions.checkStrictSpiVersionMatch; + +public class HudiConnectorFactory + implements ConnectorFactory +{ + @Override + public String getName() + { + return "hudi"; + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + checkStrictSpiVersionMatch(context, this); + return createConnector(catalogName, config, context, Optional.empty()); + } + + public static Connector createConnector( + String catalogName, + Map config, + ConnectorContext context, + Optional module) + { + ClassLoader classLoader = HudiConnectorFactory.class.getClassLoader(); + try (ThreadContextClassLoader _ = new ThreadContextClassLoader(classLoader)) { + Bootstrap app = new Bootstrap( + new MBeanModule(), + new JsonModule(), + new HudiModule(), + new HiveMetastoreModule(Optional.empty()), + new FileSystemModule(catalogName, context.getNodeManager(), context.getOpenTelemetry(), false), + new MBeanServerModule(), + module.orElse(EMPTY_MODULE), + binder -> { + binder.bind(OpenTelemetry.class).toInstance(context.getOpenTelemetry()); + binder.bind(Tracer.class).toInstance(context.getTracer()); + binder.bind(NodeVersion.class).toInstance(new NodeVersion(context.getNodeManager().getCurrentNode().getVersion())); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); + }); + + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + LifeCycleManager lifeCycleManager = injector.getInstance(LifeCycleManager.class); + HudiTransactionManager transactionManager = injector.getInstance(HudiTransactionManager.class); + ConnectorSplitManager splitManager = injector.getInstance(ConnectorSplitManager.class); + ConnectorPageSourceProvider connectorPageSource = injector.getInstance(ConnectorPageSourceProvider.class); + ConnectorNodePartitioningProvider connectorDistributionProvider = injector.getInstance(ConnectorNodePartitioningProvider.class); + Set sessionPropertiesProviders = injector.getInstance(new Key<>() {}); + HudiTableProperties hudiTableProperties = injector.getInstance(HudiTableProperties.class); + + return new HudiConnector( + injector, + lifeCycleManager, + transactionManager, + new ClassLoaderSafeConnectorSplitManager(splitManager, classLoader), + new ClassLoaderSafeConnectorPageSourceProvider(connectorPageSource, classLoader), + new ClassLoaderSafeNodePartitioningProvider(connectorDistributionProvider, classLoader), + ImmutableSet.of(), + sessionPropertiesProviders, + hudiTableProperties.getTableProperties()); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java new file mode 100644 index 0000000000000..6b34fedc8853c --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiErrorCode.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.hudi; + +import io.trino.spi.ErrorCode; +import io.trino.spi.ErrorCodeSupplier; +import io.trino.spi.ErrorType; + +import static io.trino.spi.ErrorType.EXTERNAL; + +public enum HudiErrorCode + implements ErrorCodeSupplier +{ + // HUDI_UNKNOWN_TABLE_TYPE(0, EXTERNAL), + HUDI_INVALID_PARTITION_VALUE(1, EXTERNAL), + HUDI_BAD_DATA(2, EXTERNAL), + // HUDI_MISSING_DATA(3, EXTERNAL) is deprecated + HUDI_CANNOT_OPEN_SPLIT(4, EXTERNAL), + HUDI_UNSUPPORTED_FILE_FORMAT(5, EXTERNAL), + HUDI_CURSOR_ERROR(6, EXTERNAL), + HUDI_FILESYSTEM_ERROR(7, EXTERNAL), + HUDI_PARTITION_NOT_FOUND(8, EXTERNAL), + HUDI_UNSUPPORTED_TABLE_TYPE(9, EXTERNAL), + HUDI_NO_VALID_COMMIT(10, EXTERNAL), + HUDI_META_CLIENT_ERROR(11, EXTERNAL), + /**/; + + private final ErrorCode errorCode; + + HudiErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0507_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiFileStatus.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiFileStatus.java new file mode 100644 index 0000000000000..56d585db87721 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiFileStatus.java @@ -0,0 +1,26 @@ +/* + * 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.hudi; + +import io.trino.filesystem.Location; + +import static java.util.Objects.requireNonNull; + +public record HudiFileStatus(Location location, boolean isDirectory, long length, long modificationTime, long blockSize) +{ + public HudiFileStatus + { + requireNonNull(location, "location is null"); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiMetadata.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiMetadata.java new file mode 100644 index 0000000000000..bbe5564b1d154 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiMetadata.java @@ -0,0 +1,498 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.Column; +import io.trino.metastore.HiveMetastore; +import io.trino.metastore.Table; +import io.trino.metastore.TableInfo; +import io.trino.plugin.base.classloader.ClassLoaderSafeSystemTable; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.stats.HudiTableStatistics; +import io.trino.plugin.hudi.stats.TableStatisticsReader; +import io.trino.plugin.hudi.util.HudiTableTypeUtils; +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.Constraint; +import io.trino.spi.connector.ConstraintApplicationResult; +import io.trino.spi.connector.RelationColumnsMetadata; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SchemaTablePrefix; +import io.trino.spi.connector.SystemTable; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.statistics.Estimate; +import io.trino.spi.statistics.TableStatistics; +import io.trino.spi.type.TypeManager; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantComparatorV2; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.util.Lazy; + +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.function.Function; +import java.util.function.UnaryOperator; +import java.util.stream.Stream; + +import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.metastore.Table.TABLE_COMMENT; +import static io.trino.plugin.hive.HiveTimestampPrecision.NANOSECONDS; +import static io.trino.plugin.hive.util.HiveUtil.columnMetadataGetter; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeyColumnHandles; +import static io.trino.plugin.hive.util.HiveUtil.hiveColumnHandles; +import static io.trino.plugin.hive.util.HiveUtil.isHiveSystemSchema; +import static io.trino.plugin.hive.util.HiveUtil.isHudiTable; +import static io.trino.plugin.hudi.HudiSessionProperties.getColumnsToHide; +import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataTableEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.isQueryPartitionFilterRequired; +import static io.trino.plugin.hudi.HudiSessionProperties.isTableStatisticsEnabled; +import static io.trino.plugin.hudi.HudiTableProperties.LOCATION_PROPERTY; +import static io.trino.plugin.hudi.HudiTableProperties.PARTITIONED_BY_PROPERTY; +import static io.trino.plugin.hudi.HudiUtil.buildTableMetaClient; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.StandardErrorCode.QUERY_REJECTED; +import static io.trino.spi.StandardErrorCode.UNSUPPORTED_TABLE_TYPE; +import static io.trino.spi.connector.SchemaTableName.schemaTableName; +import static java.lang.String.format; +import static java.util.Collections.singletonList; +import static java.util.Objects.requireNonNull; +import static java.util.function.Function.identity; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLUSTERING_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.INDEXING_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; + +public class HudiMetadata + implements ConnectorMetadata +{ + private static final Logger log = Logger.get(HudiMetadata.class); + private static final Map tableStatisticsCache = new ConcurrentHashMap<>(); + private static final Set refreshingKeysInProgress = ConcurrentHashMap.newKeySet(); + private final HiveMetastore metastore; + private final TrinoFileSystemFactory fileSystemFactory; + private final TypeManager typeManager; + private final ExecutorService tableStatisticsExecutor; + + public HudiMetadata( + HiveMetastore metastore, + TrinoFileSystemFactory fileSystemFactory, + TypeManager typeManager, + ExecutorService tableStatisticsExecutor) + { + this.metastore = requireNonNull(metastore, "metastore is null"); + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.tableStatisticsExecutor = requireNonNull(tableStatisticsExecutor, "tableStatisticsExecutor is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return metastore.getAllDatabases().stream() + .filter(schemaName -> !isHiveSystemSchema(schemaName)) + .collect(toImmutableList()); + } + + @Override + public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName, Optional startVersion, Optional endVersion) + { + log.info("Creating new HudiTableHandle for %s", tableName); + if (startVersion.isPresent() || endVersion.isPresent()) { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support versioned tables"); + } + + if (isHiveSystemSchema(tableName.getSchemaName())) { + return null; + } + Optional tableOpt = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + if (tableOpt.isEmpty()) { + return null; + } + + Table table = tableOpt.get(); + if (!isHudiTable(table)) { + throw new TrinoException(UNSUPPORTED_TABLE_TYPE, format("Not a Hudi table: %s", tableName)); + } + String basePath = table.getStorage().getLocation(); + TrinoFileSystem fileSystem = fileSystemFactory.create(session); + String inputFormat = table.getStorage().getStorageFormat().getInputFormat(); + HoodieTableType hoodieTableType = HudiTableTypeUtils.fromInputFormat(inputFormat); + + return new HudiTableHandle( + Optional.of(table), + Optional.of(Lazy.lazily(() -> buildTableMetaClient(fileSystem, tableName.toString(), basePath))), + tableName.getSchemaName(), + tableName.getTableName(), + table.getStorage().getLocation(), + hoodieTableType, + getPartitionKeyColumnHandles(table, typeManager), + ImmutableSet.of(), + TupleDomain.all(), + TupleDomain.all()); + } + + @Override + public Optional getSystemTable(ConnectorSession session, SchemaTableName tableName) + { + return getRawSystemTable(tableName, session) + .map(systemTable -> new ClassLoaderSafeSystemTable(systemTable, getClass().getClassLoader())); + } + + private Optional getRawSystemTable(SchemaTableName tableName, ConnectorSession session) + { + Optional nameOptional = HudiTableName.from(tableName.getTableName()); + if (nameOptional.isEmpty()) { + return Optional.empty(); + } + HudiTableName name = nameOptional.get(); + if (name.tableType() == TableType.DATA) { + return Optional.empty(); + } + + Optional
tableOptional = metastore.getTable(tableName.getSchemaName(), name.tableName()); + if (tableOptional.isEmpty()) { + return Optional.empty(); + } + if (!isHudiTable(tableOptional.get())) { + return Optional.empty(); + } + return switch (name.tableType()) { + case DATA -> throw new AssertionError(); + case TIMELINE -> { + SchemaTableName systemTableName = new SchemaTableName(tableName.getSchemaName(), name.tableNameWithType()); + yield Optional.of(new TimelineTable(fileSystemFactory.create(session), systemTableName, tableOptional.get())); + } + }; + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + { + HudiTableHandle hudiTableHandle = (HudiTableHandle) table; + return getTableMetadata(hudiTableHandle.getSchemaTableName(), getColumnsToHide(session)); + } + + @Override + public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle tableHandle, Constraint constraint) + { + HudiTableHandle handle = (HudiTableHandle) tableHandle; + HudiPredicates predicates = HudiPredicates.from(constraint.getSummary()); + TupleDomain regularColumnPredicates = predicates.getRegularColumnPredicates(); + TupleDomain partitionColumnPredicates = predicates.getPartitionColumnPredicates(); + + // TODO Since the constraint#predicate isn't utilized during split generation. So, + // Let's not add constraint#predicateColumns to newConstraintColumns. + Set newConstraintColumns = Stream.concat( + Stream.concat( + regularColumnPredicates.getDomains().stream() + .map(Map::keySet) + .flatMap(Collection::stream), + partitionColumnPredicates.getDomains().stream() + .map(Map::keySet) + .flatMap(Collection::stream)), + handle.getConstraintColumns().stream()) + .collect(toImmutableSet()); + + HudiTableHandle newHudiTableHandle = handle.applyPredicates( + newConstraintColumns, + partitionColumnPredicates, + regularColumnPredicates); + + if (handle.getPartitionPredicates().equals(newHudiTableHandle.getPartitionPredicates()) + && handle.getRegularPredicates().equals(newHudiTableHandle.getRegularPredicates()) + && handle.getConstraintColumns().equals(newHudiTableHandle.getConstraintColumns())) { + return Optional.empty(); + } + + return Optional.of(new ConstraintApplicationResult<>( + newHudiTableHandle, + newHudiTableHandle.getRegularPredicates().transformKeys(ColumnHandle.class::cast), + constraint.getExpression(), + false)); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + HudiTableHandle hudiTableHandle = (HudiTableHandle) tableHandle; + Table table = metastore.getTable(hudiTableHandle.getSchemaName(), hudiTableHandle.getTableName()) + .orElseThrow(() -> new TableNotFoundException(schemaTableName(hudiTableHandle.getSchemaName(), hudiTableHandle.getTableName()))); + return hiveColumnHandles(table, typeManager, NANOSECONDS).stream() + .collect(toImmutableMap(HiveColumnHandle::getName, identity())); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + return ((HiveColumnHandle) columnHandle).getColumnMetadata(); + } + + @Override + public Optional getInfo(ConnectorTableHandle tableHandle) + { + HudiTableHandle table = (HudiTableHandle) tableHandle; + return Optional.of(new HudiTableInfo(table.getSchemaTableName(), table.getTableType().name(), table.getBasePath())); + } + + @Override + public List listTables(ConnectorSession session, Optional optionalSchemaName) + { + ImmutableList.Builder tableNames = ImmutableList.builder(); + for (String schemaName : listSchemas(session, optionalSchemaName)) { + for (TableInfo tableInfo : metastore.getTables(schemaName)) { + tableNames.add(tableInfo.tableName()); + } + } + return tableNames.build(); + } + + @Override + public Iterator streamRelationColumns( + ConnectorSession session, + Optional schemaName, + UnaryOperator> relationFilter) + { + SchemaTablePrefix prefix = schemaName.map(SchemaTablePrefix::new) + .orElseGet(SchemaTablePrefix::new); + List tables = prefix.getTable() + .map(_ -> singletonList(prefix.toSchemaTableName())) + .orElseGet(() -> listTables(session, prefix.getSchema())); + + Map relationColumns = tables.stream() + .map(table -> getTableColumnMetadata(session, table)) + .flatMap(Optional::stream) + .collect(toImmutableMap(RelationColumnsMetadata::name, Function.identity())); + return relationFilter.apply(relationColumns.keySet()).stream() + .map(relationColumns::get) + .iterator(); + } + + @Override + public TableStatistics getTableStatistics(ConnectorSession session, ConnectorTableHandle tableHandle) + { + if (!isTableStatisticsEnabled(session) || !isHudiMetadataTableEnabled(session)) { + return TableStatistics.empty(); + } + + List columnHandles = getColumnHandles(session, tableHandle) + .values().stream() + .map(e -> (HiveColumnHandle) e) + .filter(e -> !e.isHidden()) + .toList(); + return getTableStatisticsFromCache( + (HudiTableHandle) tableHandle, columnHandles, tableStatisticsCache, refreshingKeysInProgress, tableStatisticsExecutor); + } + + @Override + public void validateScan(ConnectorSession session, ConnectorTableHandle handle) + { + HudiTableHandle hudiTableHandle = (HudiTableHandle) handle; + if (isQueryPartitionFilterRequired(session)) { + if (!hudiTableHandle.getPartitionColumns().isEmpty()) { + Set partitionColumns = hudiTableHandle.getPartitionColumns().stream() + .map(HiveColumnHandle::getName) + .collect(toImmutableSet()); + Set constraintColumns = hudiTableHandle.getConstraintColumns().stream() + .map(HiveColumnHandle::getBaseColumnName) + .collect(toImmutableSet()); + if (Collections.disjoint(constraintColumns, partitionColumns)) { + throw new TrinoException( + QUERY_REJECTED, + format("Filter required on %s for at least one of the partition columns: %s", hudiTableHandle.getSchemaTableName(), String.join(", ", partitionColumns))); + } + } + } + } + + @Override + public boolean allowSplittingReadIntoMultipleSubQueries(ConnectorSession session, ConnectorTableHandle tableHandle) + { + // hudi supports only a columnar (parquet) storage format + return true; + } + + HiveMetastore getMetastore() + { + return metastore; + } + + private Optional getTableColumnMetadata(ConnectorSession session, SchemaTableName table) + { + try { + List columns = getTableMetadata(table, getColumnsToHide(session)).getColumns(); + return Optional.of(RelationColumnsMetadata.forTable(table, columns)); + } + catch (TableNotFoundException _) { + return Optional.empty(); + } + } + + private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName, Collection columnsToHide) + { + Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + Function metadataGetter = columnMetadataGetter(table); + List columns = hiveColumnHandles(table, typeManager, NANOSECONDS).stream() + .filter(column -> !columnsToHide.contains(column.getName())) + .map(metadataGetter) + .collect(toImmutableList()); + + ImmutableMap.Builder properties = ImmutableMap.builder(); + // Location property + String location = table.getStorage().getOptionalLocation().orElse(null); + if (!isNullOrEmpty(location)) { + properties.put(LOCATION_PROPERTY, location); + } + + // Partitioning property + List partitionedBy = table.getPartitionColumns().stream() + .map(Column::getName) + .collect(toImmutableList()); + if (!partitionedBy.isEmpty()) { + properties.put(PARTITIONED_BY_PROPERTY, partitionedBy); + } + + Optional comment = Optional.ofNullable(table.getParameters().get(TABLE_COMMENT)); + return new ConnectorTableMetadata(tableName, columns, properties.buildOrThrow(), comment); + } + + private List listSchemas(ConnectorSession session, Optional schemaName) + { + return schemaName + .filter(name -> !isHiveSystemSchema(name)) + .map(Collections::singletonList) + .orElseGet(() -> listSchemaNames(session)); + } + + private static TableStatistics getTableStatisticsFromCache( + HudiTableHandle tableHandle, + List columnHandles, + Map cache, + Set refreshingKeysInProgress, + ExecutorService tableStatisticsExecutor) + { + TableStatisticsCacheKey key = new TableStatisticsCacheKey(tableHandle.getBasePath()); + HudiTableStatistics cachedValue = cache.get(key); + TableStatistics statisticsToReturn = TableStatistics.empty(); + if (cachedValue != null) { + // Here we avoid checking the latest commit which requires loading the meta client and timeline + // which can block query planning. We assume that the cache result might be stale but close + // enough for CBO. + log.info("Returning cached table statistics for table: %s, latest commit in cache: %s", + tableHandle.getSchemaTableName(), cachedValue.latestCommit()); + statisticsToReturn = cachedValue.tableStatistics(); + } + + triggerAsyncStatsRefresh(tableHandle, columnHandles, cache, key, refreshingKeysInProgress, tableStatisticsExecutor); + return statisticsToReturn; + } + + private static void triggerAsyncStatsRefresh( + HudiTableHandle tableHandle, + List columnHandles, + Map cache, + TableStatisticsCacheKey key, + Set refreshingKeysInProgress, + ExecutorService tableStatisticsExecutor) + { + if (refreshingKeysInProgress.add(key)) { + tableStatisticsExecutor.submit(() -> { + HoodieTimer refreshTimer = HoodieTimer.start(); + try { + log.info("Starting async statistics calculation for table: %s", tableHandle.getSchemaTableName()); + HoodieTableMetaClient metaClient = tableHandle.getMetaClient(); + Option latestCommitOption = metaClient.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createSet( + COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION, INDEXING_ACTION)) + .filterCompletedInstants().lastInstant(); + + if (latestCommitOption.isEmpty()) { + log.info("Putting table statistics of 0 row in %s ms for empty table: %s", + refreshTimer.endTimer(), tableHandle.getSchemaTableName()); + cache.put(key, new HudiTableStatistics( + // A dummy instant that does not match any commit + new HoodieInstant(COMPLETED, COMMIT_ACTION, "", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + TableStatistics.builder().setRowCount(Estimate.of(0)).build())); + return; + } + + HoodieInstant latestCommit = latestCommitOption.get(); + HudiTableStatistics oldValue = cache.get(key); + if (oldValue != null && latestCommit.equals(oldValue.latestCommit())) { + log.info("Table statistics is still valid for table: %s (checked in %s ms)", + tableHandle.getSchemaTableName(), refreshTimer.endTimer()); + return; + } + + if (!metaClient.getTableConfig().isMetadataTableAvailable() + || !metaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.COLUMN_STATS)) { + log.info("Putting empty table statistics in %s ms as metadata table or " + + "column stats is not available for table: %s", + refreshTimer.endTimer(), tableHandle.getSchemaTableName()); + cache.put(key, new HudiTableStatistics(latestCommit, TableStatistics.empty())); + return; + } + + TableStatistics newStatistics = TableStatisticsReader.create(metaClient) + .getTableStatistics(latestCommit, columnHandles); + HudiTableStatistics newValue = new HudiTableStatistics(latestCommit, newStatistics); + cache.put(key, newValue); + log.info("Async table statistics calculation finished in %s ms for table: %s, commit: %s", + refreshTimer.endTimer(), tableHandle.getSchemaTableName(), latestCommit); + } + catch (Throwable e) { + log.error(e, "Error calculating table statistics asynchronously for table %s", tableHandle.getSchemaTableName()); + } + finally { + refreshingKeysInProgress.remove(key); + } + }); + } + else { + log.debug("Table statistics refresh already in progress for table: %s", tableHandle.getSchemaTableName()); + } + } + + private record TableStatisticsCacheKey(String basePath) {} +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java new file mode 100644 index 0000000000000..ca11b48a61a95 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.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.hudi; + +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.hudi.stats.ForHudiTableStatistics; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.spi.type.TypeManager; + +import java.util.Optional; +import java.util.concurrent.ExecutorService; + +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static java.util.Objects.requireNonNull; + +public class HudiMetadataFactory +{ + private final HiveMetastoreFactory metastoreFactory; + private final TrinoFileSystemFactory fileSystemFactory; + private final TypeManager typeManager; + private final long perTransactionMetastoreCacheMaximumSize; + private final ExecutorService tableStatisticsExecutor; + + @Inject + public HudiMetadataFactory( + HiveMetastoreFactory metastoreFactory, + TrinoFileSystemFactory fileSystemFactory, + TypeManager typeManager, + HudiConfig hudiConfig, + @ForHudiTableStatistics ExecutorService tableStatisticsExecutor) + { + this.metastoreFactory = requireNonNull(metastoreFactory, "metastore is null"); + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.perTransactionMetastoreCacheMaximumSize = hudiConfig.getPerTransactionMetastoreCacheMaximumSize(); + this.tableStatisticsExecutor = requireNonNull(tableStatisticsExecutor, "tableStatisticsExecutor is null"); + } + + public HudiMetadata create(ConnectorIdentity identity) + { + CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastoreFactory.createMetastore(Optional.of(identity)), perTransactionMetastoreCacheMaximumSize); + return new HudiMetadata(cachingHiveMetastore, fileSystemFactory, typeManager, tableStatisticsExecutor); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiModule.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiModule.java new file mode 100644 index 0000000000000..929cd834c844b --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiModule.java @@ -0,0 +1,117 @@ +/* + * 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.hudi; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.Singleton; +import io.trino.metastore.HiveMetastore; +import io.trino.plugin.base.metrics.FileFormatDataSourceStats; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.HideDeltaLakeTables; +import io.trino.plugin.hive.HiveNodePartitioningProvider; +import io.trino.plugin.hive.HiveTransactionHandle; +import io.trino.plugin.hive.metastore.thrift.TranslateHiveViews; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.plugin.hive.parquet.ParquetWriterConfig; +import io.trino.plugin.hudi.stats.ForHudiTableStatistics; +import io.trino.spi.connector.ConnectorNodePartitioningProvider; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.security.ConnectorIdentity; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.BiFunction; + +import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.trino.plugin.base.ClosingBinder.closingBinder; +import static java.util.concurrent.Executors.newCachedThreadPool; +import static java.util.concurrent.Executors.newScheduledThreadPool; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class HudiModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(HudiTransactionManager.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(HudiConfig.class); + + binder.bind(boolean.class).annotatedWith(TranslateHiveViews.class).toInstance(false); + binder.bind(boolean.class).annotatedWith(HideDeltaLakeTables.class).toInstance(false); + + newSetBinder(binder, SessionPropertiesProvider.class).addBinding().to(HudiSessionProperties.class).in(Scopes.SINGLETON); + binder.bind(HudiTableProperties.class).in(Scopes.SINGLETON); + + binder.bind(ConnectorSplitManager.class).to(HudiSplitManager.class).in(Scopes.SINGLETON); + binder.bind(ConnectorPageSourceProvider.class).to(HudiPageSourceProvider.class).in(Scopes.SINGLETON); + binder.bind(ConnectorNodePartitioningProvider.class).to(HiveNodePartitioningProvider.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(ParquetReaderConfig.class); + configBinder(binder).bindConfig(ParquetWriterConfig.class); + + binder.bind(HudiMetadataFactory.class).in(Scopes.SINGLETON); + + binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(FileFormatDataSourceStats.class).withGeneratedName(); + + closingBinder(binder).registerExecutor(Key.get(ExecutorService.class, ForHudiTableStatistics.class)); + closingBinder(binder).registerExecutor(Key.get(ExecutorService.class, ForHudiSplitManager.class)); + closingBinder(binder).registerExecutor(Key.get(ScheduledExecutorService.class, ForHudiSplitSource.class)); + } + + @Provides + @Singleton + @ForHudiTableStatistics + public ExecutorService createTableStatisticsExecutor(HudiConfig hudiConfig) + { + return newScheduledThreadPool( + hudiConfig.getTableStatisticsExecutorParallelism(), + daemonThreadsNamed("hudi-table-statistics-executor-%s")); + } + + @Provides + @Singleton + @ForHudiSplitManager + public ExecutorService createExecutorService() + { + return newCachedThreadPool(daemonThreadsNamed("hudi-split-manager-%s")); + } + + @Provides + @Singleton + @ForHudiSplitSource + public ScheduledExecutorService createSplitLoaderExecutor(HudiConfig hudiConfig) + { + return newScheduledThreadPool( + hudiConfig.getSplitLoaderParallelism(), + daemonThreadsNamed("hudi-split-loader-%s")); + } + + @Provides + @Singleton + public BiFunction createHiveMetastoreGetter(HudiTransactionManager transactionManager) + { + return (identity, transactionHandle) -> + transactionManager.get(transactionHandle, identity).getMetastore(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPageSource.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPageSource.java new file mode 100644 index 0000000000000..e81887414425b --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPageSource.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.hudi; + +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.reader.HudiTrinoReaderContext; +import io.trino.plugin.hudi.util.HudiAvroSerializer; +import io.trino.plugin.hudi.util.SynthesizedColumnHandler; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.metrics.Metrics; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.table.read.HoodieFileGroupReader; + +import java.io.IOException; +import java.util.List; +import java.util.OptionalLong; +import java.util.concurrent.CompletableFuture; + +import static com.google.common.base.Preconditions.checkState; + +public class HudiPageSource + implements ConnectorPageSource +{ + HoodieFileGroupReader fileGroupReader; + // TODO: Remove pageSource here, Hudi doesn't use this page source to read + ConnectorPageSource pageSource; + HudiTrinoReaderContext readerContext; + PageBuilder pageBuilder; + HudiAvroSerializer avroSerializer; + List columnHandles; + + public HudiPageSource( + ConnectorPageSource pageSource, + HoodieFileGroupReader fileGroupReader, + HudiTrinoReaderContext readerContext, + List columnHandles, + SynthesizedColumnHandler synthesizedColumnHandler) + { + this.pageSource = pageSource; + this.fileGroupReader = fileGroupReader; + this.initFileGroupReader(); + this.readerContext = readerContext; + this.columnHandles = columnHandles; + this.pageBuilder = new PageBuilder(columnHandles.stream().map(HiveColumnHandle::getType).toList()); + this.avroSerializer = new HudiAvroSerializer(columnHandles, synthesizedColumnHandler); + } + + @Override + public long getCompletedBytes() + { + return pageSource.getCompletedBytes(); + } + + @Override + public OptionalLong getCompletedPositions() + { + return pageSource.getCompletedPositions(); + } + + @Override + public long getReadTimeNanos() + { + return pageSource.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + try { + return !fileGroupReader.hasNext(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Page getNextPage() + { + checkState(pageBuilder.isEmpty(), "PageBuilder is not empty at the beginning of a new page"); + try { + while (fileGroupReader.hasNext()) { + avroSerializer.buildRecordInPage(pageBuilder, fileGroupReader.next()); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + + Page newPage = pageBuilder.build(); + pageBuilder.reset(); + return newPage; + } + + @Override + public long getMemoryUsage() + { + return pageSource.getMemoryUsage(); + } + + @Override + public void close() + throws IOException + { + fileGroupReader.close(); + pageSource.close(); + } + + @Override + public CompletableFuture isBlocked() + { + return pageSource.isBlocked(); + } + + @Override + public Metrics getMetrics() + { + return pageSource.getMetrics(); + } + + protected void initFileGroupReader() + { + try { + this.fileGroupReader.initRecordIterators(); + } + catch (IOException e) { + throw new RuntimeException("Failed to initialize file group reader!", e); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java new file mode 100644 index 0000000000000..500934044b371 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -0,0 +1,451 @@ +/* + * 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.hudi; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.airlift.log.Logger; +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.parquet.ParquetCorruptionException; +import io.trino.parquet.ParquetDataSource; +import io.trino.parquet.ParquetDataSourceId; +import io.trino.parquet.ParquetReaderOptions; +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.HiveColumnHandle; +import io.trino.plugin.hive.ReaderColumns; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.plugin.hudi.file.HudiBaseFile; +import io.trino.plugin.hudi.reader.HudiTrinoReaderContext; +import io.trino.plugin.hudi.storage.HudiTrinoStorage; +import io.trino.plugin.hudi.storage.TrinoStorageConfiguration; +import io.trino.plugin.hudi.util.SynthesizedColumnHandler; +import io.trino.spi.TrinoException; +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.EmptyPageSource; +import io.trino.spi.predicate.TupleDomain; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.read.HoodieFileGroupReader; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.storage.StoragePath; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.stream.Collectors; + +import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.trino.parquet.ParquetTypeUtils.getColumnIO; +import static io.trino.parquet.ParquetTypeUtils.getDescriptors; +import static io.trino.parquet.predicate.PredicateUtils.buildPredicate; +import static io.trino.parquet.predicate.PredicateUtils.getFilteredRowGroups; +import static io.trino.plugin.hive.HiveColumnHandle.partitionColumnHandle; +import static io.trino.plugin.hive.HivePageSourceProvider.projectBaseColumns; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.ParquetReaderProvider; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.createDataSource; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.createParquetPageSource; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getParquetMessageType; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getParquetTupleDomain; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_CURSOR_ERROR; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_FILESYSTEM_ERROR; +import static io.trino.plugin.hudi.HudiSessionProperties.getParquetMaxReadBlockRowCount; +import static io.trino.plugin.hudi.HudiSessionProperties.getParquetMaxReadBlockSize; +import static io.trino.plugin.hudi.HudiSessionProperties.getParquetSmallFileThreshold; +import static io.trino.plugin.hudi.HudiSessionProperties.isParquetIgnoreStatistics; +import static io.trino.plugin.hudi.HudiSessionProperties.isParquetUseColumnIndex; +import static io.trino.plugin.hudi.HudiSessionProperties.isParquetVectorizedDecodingEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.shouldUseParquetColumnNames; +import static io.trino.plugin.hudi.HudiSessionProperties.useParquetBloomFilter; +import static io.trino.plugin.hudi.HudiUtil.buildTableMetaClient; +import static io.trino.plugin.hudi.HudiUtil.constructSchema; +import static io.trino.plugin.hudi.HudiUtil.convertToFileSlice; +import static io.trino.plugin.hudi.HudiUtil.prependHudiMetaColumns; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toUnmodifiableList; + +public class HudiPageSourceProvider + implements ConnectorPageSourceProvider +{ + private static final Logger log = Logger.get(HudiPageSourceProvider.class); + private static final int DOMAIN_COMPACTION_THRESHOLD = 1000; + + private final TrinoFileSystemFactory fileSystemFactory; + private final FileFormatDataSourceStats dataSourceStats; + private final ParquetReaderOptions options; + private final DateTimeZone timeZone = DateTimeZone.forID("UTC"); + + @Inject + public HudiPageSourceProvider( + TrinoFileSystemFactory fileSystemFactory, + FileFormatDataSourceStats dataSourceStats, + ParquetReaderConfig parquetReaderConfig) + { + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.dataSourceStats = requireNonNull(dataSourceStats, "dataSourceStats is null"); + this.options = requireNonNull(parquetReaderConfig, "parquetReaderConfig is null").toParquetReaderOptions(); + } + + @Override + public ConnectorPageSource createPageSource( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit connectorSplit, + ConnectorTableHandle connectorTable, + List columns, + DynamicFilter dynamicFilter) + { + HudiTableHandle hudiTableHandle = (HudiTableHandle) connectorTable; + HudiSplit hudiSplit = (HudiSplit) connectorSplit; + Optional hudiBaseFileOpt = hudiSplit.getBaseFile(); + + String dataFilePath = hudiBaseFileOpt.isPresent() + ? hudiBaseFileOpt.get().getPath() + : hudiSplit.getLogFiles().getFirst().getPath(); + // Filter out metadata table splits + // TODO: Move this check into a higher calling stack, such that the split is not created at all + if (dataFilePath.contains(new StoragePath( + ((HudiTableHandle) connectorTable).getBasePath()).toUri().getPath() + "/.hoodie/metadata")) { + return new EmptyPageSource(); + } + + // Handle MERGE_ON_READ tables to be read in read_optimized mode + // IMPORTANT: These tables will have a COPY_ON_WRITE table, see: `HudiTableTypeUtils#fromInputFormat` + // TODO: Move this check into a higher calling stack, such that the split is not created at all + if (hudiTableHandle.getTableType().equals(HoodieTableType.COPY_ON_WRITE) && !hudiSplit.getLogFiles().isEmpty()) { + if (hudiBaseFileOpt.isEmpty()) { + // Handle hasLogFiles=true, hasBaseFile = false + // Ignoring log files without base files, no data required to be read + return new EmptyPageSource(); + } + } + + long start = 0; + long length = 10; + if (hudiBaseFileOpt.isPresent()) { + start = hudiBaseFileOpt.get().getStart(); + length = hudiBaseFileOpt.get().getLength(); + } + + // Enable predicate pushdown for splits containing only base files + boolean isBaseFileOnly = hudiSplit.getLogFiles().isEmpty(); + // Convert columns to HiveColumnHandles + List hiveColumnHandles = getHiveColumns(columns, isBaseFileOnly); + + // Get non-synthesized columns (columns that are available in data file) + List dataColumnHandles = hiveColumnHandles.stream() + .filter(columnHandle -> !columnHandle.isPartitionKey() && !columnHandle.isHidden()) + .collect(Collectors.toList()); + // The `columns` list could be empty when count(*) is issued, + // prepending hoodie meta columns for Hudi split with log files + // to allow a non-empty dataPageSource to be returned + List hudiMetaAndDataColumnHandles = prependHudiMetaColumns(dataColumnHandles); + + TrinoFileSystem fileSystem = fileSystemFactory.create(session); + ConnectorPageSource dataPageSource = createPageSource( + session, + isBaseFileOnly ? dataColumnHandles : hudiMetaAndDataColumnHandles, + hudiSplit, + fileSystem.newInputFile(Location.of(hudiBaseFileOpt.get().getPath()), hudiBaseFileOpt.get().getFileSize()), + dataSourceStats, + options + .withIgnoreStatistics(isParquetIgnoreStatistics(session)) + .withMaxReadBlockSize(getParquetMaxReadBlockSize(session)) + .withMaxReadBlockRowCount(getParquetMaxReadBlockRowCount(session)) + .withSmallFileThreshold(getParquetSmallFileThreshold(session)) + .withUseColumnIndex(isParquetUseColumnIndex(session)) + .withBloomFilter(useParquetBloomFilter(session)) + .withVectorizedDecodingEnabled(isParquetVectorizedDecodingEnabled(session)), + timeZone, dynamicFilter, isBaseFileOnly); + + SynthesizedColumnHandler synthesizedColumnHandler = SynthesizedColumnHandler.create(hudiSplit); + + // Avoid avro serialization if split/filegroup only contains base files + if (isBaseFileOnly) { + ValidationUtils.checkArgument(!hiveColumnHandles.isEmpty(), + "Column handles should always be present for providing Hudi data page source on a base file"); + return new HudiBaseFileOnlyPageSource( + dataPageSource, + hiveColumnHandles, + dataColumnHandles, + synthesizedColumnHandler); + } + + // TODO: Move this into HudiTableHandle + HoodieTableMetaClient metaClient = buildTableMetaClient( + fileSystemFactory.create(session), hudiTableHandle.getSchemaTableName().toString(), hudiTableHandle.getBasePath()); + String latestCommitTime = metaClient.getCommitsTimeline().lastInstant().get().requestedTime(); + Schema dataSchema; + try { + dataSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(latestCommitTime); + } + catch (Throwable e) { + // Unable to find table schema + throw new TrinoException(HUDI_FILESYSTEM_ERROR, e); + } + + HudiTrinoReaderContext readerContext = new HudiTrinoReaderContext( + dataPageSource, + dataColumnHandles, + hudiMetaAndDataColumnHandles, + synthesizedColumnHandler); + + // Construct an Avro schema for log file reader + Schema requestedSchema = constructSchema(dataSchema, hudiMetaAndDataColumnHandles.stream().map(HiveColumnHandle::getName).toList()); + HoodieFileGroupReader fileGroupReader = + new HoodieFileGroupReader<>( + readerContext, + new HudiTrinoStorage(fileSystemFactory.create(session), new TrinoStorageConfiguration()), + hudiTableHandle.getBasePath(), + latestCommitTime, + convertToFileSlice(hudiSplit, hudiTableHandle.getBasePath()), + dataSchema, + requestedSchema, + Option.empty(), + metaClient, + metaClient.getTableConfig().getProps(), + start, + length, + false); + + return new HudiPageSource( + dataPageSource, + fileGroupReader, + readerContext, + hiveColumnHandles, + synthesizedColumnHandler); + } + + static ConnectorPageSource createPageSource( + ConnectorSession session, + List columns, + HudiSplit hudiSplit, + TrinoInputFile inputFile, + FileFormatDataSourceStats dataSourceStats, + ParquetReaderOptions options, + DateTimeZone timeZone, + DynamicFilter dynamicFilter, + boolean enablePredicatePushDown) + { + ParquetDataSource dataSource = null; + boolean useColumnNames = shouldUseParquetColumnNames(session); + HudiBaseFile baseFile = hudiSplit.getBaseFile().get(); + String path = baseFile.getPath(); + long start = baseFile.getStart(); + long length = baseFile.getLength(); + try { + AggregatedMemoryContext memoryContext = newSimpleAggregatedMemoryContext(); + dataSource = createDataSource(inputFile, OptionalLong.of(baseFile.getFileSize()), options, memoryContext, dataSourceStats); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); + MessageType fileSchema = fileMetaData.getSchema(); + + // When not using columnNames, physical indexes are used and there could be cases when the physical index in HiveColumnHandle is different from the fileSchema of the + // parquet files. This could happen when schema evolution happened. In such a case, we will need to remap the column indices in the HiveColumnHandles. + if (!useColumnNames) { + // HiveColumnHandle names are in lower case, case-insensitive + columns = remapColumnIndicesToPhysical(fileSchema, columns, false); + } + + Optional message = getParquetMessageType(columns, useColumnNames, fileSchema); + + MessageType requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + MessageColumnIO messageColumn = getColumnIO(fileSchema, requestedSchema); + + Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); + + TupleDomain parquetTupleDomain = options.isIgnoreStatistics() || !enablePredicatePushDown + ? TupleDomain.all() + : getParquetTupleDomain(descriptorsByPath, getCombinedPredicate(hudiSplit, dynamicFilter), fileSchema, useColumnNames); + + TupleDomainParquetPredicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); + + List rowGroups = getFilteredRowGroups( + start, + length, + dataSource, + parquetMetadata, + ImmutableList.of(parquetTupleDomain), + ImmutableList.of(parquetPredicate), + descriptorsByPath, + timeZone, + DOMAIN_COMPACTION_THRESHOLD, + options); + + Optional readerProjections = projectBaseColumns(columns); + List baseColumns = readerProjections.map(projection -> + projection.get().stream() + .map(HiveColumnHandle.class::cast) + .collect(toUnmodifiableList())) + .orElse(columns); + ParquetDataSourceId dataSourceId = dataSource.getId(); + ParquetDataSource finalDataSource = dataSource; + ParquetReaderProvider parquetReaderProvider = fields -> new ParquetReader( + Optional.ofNullable(fileMetaData.getCreatedBy()), + fields, + rowGroups, + finalDataSource, + timeZone, + memoryContext, + options, + exception -> handleException(dataSourceId, exception), + Optional.of(parquetPredicate), + Optional.empty()); + return createParquetPageSource(baseColumns, fileSchema, messageColumn, useColumnNames, parquetReaderProvider); + } + catch (IOException | RuntimeException e) { + try { + if (dataSource != null) { + dataSource.close(); + } + } + catch (IOException _) { + } + if (e instanceof TrinoException) { + throw (TrinoException) e; + } + if (e instanceof ParquetCorruptionException) { + throw new TrinoException(HUDI_BAD_DATA, e); + } + String message = "Error opening Hudi split %s (offset=%s, length=%s): %s".formatted(path, start, length, e.getMessage()); + throw new TrinoException(HUDI_CANNOT_OPEN_SPLIT, message, e); + } + } + + private static TrinoException handleException(ParquetDataSourceId dataSourceId, Exception exception) + { + if (exception instanceof TrinoException) { + return (TrinoException) exception; + } + if (exception instanceof ParquetCorruptionException) { + return new TrinoException(HUDI_BAD_DATA, exception); + } + return new TrinoException(HUDI_CURSOR_ERROR, format("Failed to read Parquet file: %s", dataSourceId), exception); + } + + /** + * Creates a new list of ColumnHandles where the index associated with each handle corresponds to its physical position within the provided fileSchema (MessageType). + * This is necessary when a downstream component relies on the handle's index for physical data access, and the logical schema order (potentially reflected in the + * original handles) differs from the physical file layout. + * + * @param fileSchema The MessageType representing the physical schema of the Parquet file. + * @param requestedColumns The original list of Trino ColumnHandles as received from the engine. + * @param caseSensitive Whether the lookup between Trino column names (from handles) and Parquet field names (from fileSchema) should be case-sensitive. + * @return A new list of HiveColumnHandle, preserving the original order, but with each handle containing the correct physical index relative to fileSchema. + */ + @VisibleForTesting + public static List remapColumnIndicesToPhysical( + MessageType fileSchema, + List requestedColumns, + boolean caseSensitive) + { + // Create a map from column name to its physical index in the fileSchema. + Map physicalIndexMap = new HashMap<>(); + List fileFields = fileSchema.getFields(); + for (int i = 0; i < fileFields.size(); i++) { + Type field = fileFields.get(i); + String fieldName = field.getName(); + String mapKey = caseSensitive ? fieldName : fieldName.toLowerCase(Locale.getDefault()); + physicalIndexMap.put(mapKey, i); + } + + // Iterate through the columns requested by Trino IN ORDER. + List remappedHandles = new ArrayList<>(requestedColumns.size()); + for (HiveColumnHandle originalHandle : requestedColumns) { + String requestedName = originalHandle.getBaseColumnName(); + + // Determine the key to use for looking up the physical index + String lookupKey = caseSensitive ? requestedName : requestedName.toLowerCase(Locale.getDefault()); + + // Find the physical index from the file schema map constructed from fielSchema + Integer physicalIndex = physicalIndexMap.get(lookupKey); + + HiveColumnHandle remappedHandle = new HiveColumnHandle( + requestedName, + physicalIndex, + originalHandle.getBaseHiveType(), + originalHandle.getType(), + originalHandle.getHiveColumnProjectionInfo(), + originalHandle.getColumnType(), + originalHandle.getComment()); + remappedHandles.add(remappedHandle); + } + + return remappedHandles; + } + + private static TupleDomain getCombinedPredicate(HudiSplit hudiSplit, DynamicFilter dynamicFilter) + { + // Combine static and dynamic predicates + TupleDomain staticPredicate = hudiSplit.getPredicate(); + TupleDomain dynamicPredicate = dynamicFilter.getCurrentPredicate() + .transformKeys(HiveColumnHandle.class::cast); + TupleDomain combinedPredicate = staticPredicate.intersect(dynamicPredicate); + + if (!combinedPredicate.isAll()) { + log.debug("Combined predicate for Parquet read (Split: %s): %s", hudiSplit, combinedPredicate); + } + return combinedPredicate; + } + + private static List getHiveColumns(List columns, + boolean isBaseFileOnly) + { + if (!isBaseFileOnly || !columns.isEmpty()) { + return columns.stream() + .map(HiveColumnHandle.class::cast) + .toList(); + } + + // The `columns` list containing the requested columns to read could be empty + // when count(*) is in the statement; to make sure the page source works properly, + // the synthesized partition column is added in this case. + return Collections.singletonList(partitionColumnHandle()); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPlugin.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPlugin.java new file mode 100644 index 0000000000000..1744105faddb4 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPlugin.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.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; + +public class HudiPlugin + implements Plugin +{ + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new HudiConnectorFactory()); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPredicates.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPredicates.java new file mode 100644 index 0000000000000..16859e8e6dd3e --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiPredicates.java @@ -0,0 +1,68 @@ +/* + * 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.hudi; + +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.TupleDomain; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class HudiPredicates +{ + private final TupleDomain partitionColumnPredicates; + private final TupleDomain regularColumnPredicates; + + public static HudiPredicates from(TupleDomain predicate) + { + Map partitionColumnPredicates = new HashMap<>(); + Map regularColumnPredicates = new HashMap<>(); + + Optional> domains = predicate.getDomains(); + domains.ifPresent(columnHandleDomainMap -> columnHandleDomainMap.forEach((key, value) -> { + HiveColumnHandle columnHandle = (HiveColumnHandle) key; + if (columnHandle.isPartitionKey()) { + partitionColumnPredicates.put(columnHandle, value); + } + else { + regularColumnPredicates.put(columnHandle, value); + } + })); + + return new HudiPredicates( + TupleDomain.withColumnDomains(partitionColumnPredicates), + TupleDomain.withColumnDomains(regularColumnPredicates)); + } + + private HudiPredicates( + TupleDomain partitionColumnPredicates, + TupleDomain regularColumnPredicates) + { + this.partitionColumnPredicates = partitionColumnPredicates; + this.regularColumnPredicates = regularColumnPredicates; + } + + public TupleDomain getPartitionColumnPredicates() + { + return partitionColumnPredicates; + } + + public TupleDomain getRegularColumnPredicates() + { + return regularColumnPredicates; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java new file mode 100644 index 0000000000000..c1414f13ee899 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java @@ -0,0 +1,395 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.session.PropertyMetadata; +import io.trino.spi.type.ArrayType; + +import java.util.Collection; +import java.util.List; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.base.session.PropertyMetadataUtil.dataSizeProperty; +import static io.trino.plugin.base.session.PropertyMetadataUtil.durationProperty; +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.StandardErrorCode.INVALID_SESSION_PROPERTY; +import static io.trino.spi.session.PropertyMetadata.booleanProperty; +import static io.trino.spi.session.PropertyMetadata.doubleProperty; +import static io.trino.spi.session.PropertyMetadata.integerProperty; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; + +public class HudiSessionProperties + implements SessionPropertiesProvider +{ + private static final String COLUMNS_TO_HIDE = "columns_to_hide"; + static final String TABLE_STATISTICS_ENABLED = "table_statistics_enabled"; + static final String METADATA_TABLE_ENABLED = "metadata_enabled"; + private static final String USE_PARQUET_COLUMN_NAMES = "use_parquet_column_names"; + private static final String PARQUET_IGNORE_STATISTICS = "parquet_ignore_statistics"; + private static final String PARQUET_USE_COLUMN_INDEX = "parquet_use_column_index"; + private static final String PARQUET_USE_BLOOM_FILTER = "parquet_use_bloom_filter"; + private static final String PARQUET_MAX_READ_BLOCK_SIZE = "parquet_max_read_block_size"; + private static final String PARQUET_MAX_READ_BLOCK_ROW_COUNT = "parquet_max_read_block_row_count"; + private static final String PARQUET_SMALL_FILE_THRESHOLD = "parquet_small_file_threshold"; + private static final String PARQUET_VECTORIZED_DECODING_ENABLED = "parquet_vectorized_decoding_enabled"; + private static final String SIZE_BASED_SPLIT_WEIGHTS_ENABLED = "size_based_split_weights_enabled"; + private static final String STANDARD_SPLIT_WEIGHT_SIZE = "standard_split_weight_size"; + private static final String MINIMUM_ASSIGNED_SPLIT_WEIGHT = "minimum_assigned_split_weight"; + private static final String TARGET_SPLIT_SIZE = "target_split_size"; + private static final String MAX_SPLITS_PER_SECOND = "max_splits_per_second"; + private static final String MAX_OUTSTANDING_SPLITS = "max_outstanding_splits"; + private static final String SPLIT_GENERATOR_PARALLELISM = "split_generator_parallelism"; + static final String QUERY_PARTITION_FILTER_REQUIRED = "query_partition_filter_required"; + private static final String IGNORE_ABSENT_PARTITIONS = "ignore_absent_partitions"; + static final String DYNAMIC_FILTERING_WAIT_TIMEOUT = "dynamic_filtering_wait_timeout"; + + // Internal configuration for debugging and testing + static final String RECORD_LEVEL_INDEX_ENABLED = "record_level_index_enabled"; + static final String SECONDARY_INDEX_ENABLED = "secondary_index_enabled"; + static final String COLUMN_STATS_INDEX_ENABLED = "column_stats_index_enabled"; + static final String PARTITION_STATS_INDEX_ENABLED = "partition_stats_index_enabled"; + static final String COLUMN_STATS_WAIT_TIMEOUT = "column_stats_wait_timeout"; + static final String RECORD_INDEX_WAIT_TIMEOUT = "record_index_wait_timeout"; + static final String SECONDARY_INDEX_WAIT_TIMEOUT = "secondary_index_wait_timeout"; + + private final List> sessionProperties; + + @Inject + public HudiSessionProperties(HudiConfig hudiConfig, ParquetReaderConfig parquetReaderConfig) + { + sessionProperties = ImmutableList.of( + new PropertyMetadata<>( + COLUMNS_TO_HIDE, + "List of column names that will be hidden", + new ArrayType(VARCHAR), + List.class, + hudiConfig.getColumnsToHide(), + false, + value -> ((Collection) value).stream() + .map(name -> ((String) name).toLowerCase(ENGLISH)) + .collect(toImmutableList()), + value -> value), + booleanProperty( + TABLE_STATISTICS_ENABLED, + "Expose table statistics", + hudiConfig.isTableStatisticsEnabled(), + false), + booleanProperty( + METADATA_TABLE_ENABLED, + "For Hudi tables prefer to fetch the list of files from its metadata table", + hudiConfig.isMetadataEnabled(), + false), + booleanProperty( + USE_PARQUET_COLUMN_NAMES, + "Access parquet columns using names from the file. If disabled, then columns are accessed using index.", + hudiConfig.getUseParquetColumnNames(), + false), + booleanProperty( + PARQUET_IGNORE_STATISTICS, + "Ignore statistics from Parquet to allow querying files with corrupted or incorrect statistics", + parquetReaderConfig.isIgnoreStatistics(), + false), + booleanProperty( + PARQUET_USE_COLUMN_INDEX, + "Use Parquet column index", + hudiConfig.isUseParquetColumnIndex(), + false), + booleanProperty( + PARQUET_USE_BLOOM_FILTER, + "Use Parquet Bloom filters", + parquetReaderConfig.isUseBloomFilter(), + false), + dataSizeProperty( + PARQUET_MAX_READ_BLOCK_SIZE, + "Parquet: Maximum size of a block to read", + parquetReaderConfig.getMaxReadBlockSize(), + false), + integerProperty( + PARQUET_MAX_READ_BLOCK_ROW_COUNT, + "Parquet: Maximum number of rows read in a batch", + parquetReaderConfig.getMaxReadBlockRowCount(), + value -> { + if (value < 128 || value > 65536) { + throw new TrinoException( + INVALID_SESSION_PROPERTY, + format("%s must be between 128 and 65536: %s", PARQUET_MAX_READ_BLOCK_ROW_COUNT, value)); + } + }, + false), + 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), + booleanProperty( + PARQUET_VECTORIZED_DECODING_ENABLED, + "Enable using Java Vector API for faster decoding of parquet files", + parquetReaderConfig.isVectorizedDecodingEnabled(), + false), + booleanProperty( + SIZE_BASED_SPLIT_WEIGHTS_ENABLED, + format("If enabled, size-based splitting ensures that each batch of splits has enough data to process as defined by %s", STANDARD_SPLIT_WEIGHT_SIZE), + hudiConfig.isSizeBasedSplitWeightsEnabled(), + false), + dataSizeProperty( + STANDARD_SPLIT_WEIGHT_SIZE, + "The split size corresponding to the standard weight (1.0) when size-based split weights are enabled", + hudiConfig.getStandardSplitWeightSize(), + false), + doubleProperty( + MINIMUM_ASSIGNED_SPLIT_WEIGHT, + "Minimum assigned split weight when size-based split weights are enabled", + hudiConfig.getMinimumAssignedSplitWeight(), + value -> { + if (!Double.isFinite(value) || value <= 0 || value > 1) { + throw new TrinoException(INVALID_SESSION_PROPERTY, format("%s must be > 0 and <= 1.0: %s", MINIMUM_ASSIGNED_SPLIT_WEIGHT, value)); + } + }, + false), + dataSizeProperty( + TARGET_SPLIT_SIZE, + "The target split size", + hudiConfig.getTargetSplitSize(), + false), + integerProperty( + MAX_SPLITS_PER_SECOND, + "Rate at which splits are enqueued for processing. The queue will throttle if this rate limit is breached.", + hudiConfig.getMaxSplitsPerSecond(), + false), + integerProperty( + MAX_OUTSTANDING_SPLITS, + "Maximum outstanding splits in a batch enqueued for processing", + hudiConfig.getMaxOutstandingSplits(), + false), + integerProperty( + SPLIT_GENERATOR_PARALLELISM, + "Number of threads to generate splits from partitions", + hudiConfig.getSplitGeneratorParallelism(), + false), + booleanProperty( + QUERY_PARTITION_FILTER_REQUIRED, + "Require a filter on at least one partition column", + hudiConfig.isQueryPartitionFilterRequired(), + false), + booleanProperty( + IGNORE_ABSENT_PARTITIONS, + "Ignore absent partitions", + hudiConfig.isIgnoreAbsentPartitions(), + false), + booleanProperty( + RECORD_LEVEL_INDEX_ENABLED, + "Enable record level index for file skipping", + hudiConfig.isRecordLevelIndexEnabled(), + true), + booleanProperty( + SECONDARY_INDEX_ENABLED, + "Enable secondary index for file skipping", + hudiConfig.isSecondaryIndexEnabled(), + true), + booleanProperty( + COLUMN_STATS_INDEX_ENABLED, + "Enable column stats index for file skipping", + hudiConfig.isColumnStatsIndexEnabled(), + true), + booleanProperty( + PARTITION_STATS_INDEX_ENABLED, + "Enable partition stats index for file skipping", + hudiConfig.isPartitionStatsIndexEnabled(), + true), + durationProperty( + COLUMN_STATS_WAIT_TIMEOUT, + "Maximum timeout to wait for loading column stats", + hudiConfig.getColumnStatsWaitTimeout(), + false), + durationProperty( + RECORD_INDEX_WAIT_TIMEOUT, + "Maximum timeout to wait for loading record index", + hudiConfig.getRecordIndexWaitTimeout(), + false), + durationProperty( + SECONDARY_INDEX_WAIT_TIMEOUT, + "Maximum timeout to wait for loading secondary index", + hudiConfig.getSecondaryIndexWaitTimeout(), + false), + durationProperty( + DYNAMIC_FILTERING_WAIT_TIMEOUT, + "Duration to wait for completion of dynamic filters during split generation", + hudiConfig.getDynamicFilteringWaitTimeout(), + false)); + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } + + @SuppressWarnings("unchecked") + public static List getColumnsToHide(ConnectorSession session) + { + return (List) session.getProperty(COLUMNS_TO_HIDE, List.class); + } + + public static boolean isTableStatisticsEnabled(ConnectorSession session) + { + return session.getProperty(TABLE_STATISTICS_ENABLED, Boolean.class); + } + + public static boolean isHudiMetadataTableEnabled(ConnectorSession session) + { + return session.getProperty(METADATA_TABLE_ENABLED, Boolean.class); + } + + public static boolean shouldUseParquetColumnNames(ConnectorSession session) + { + return session.getProperty(USE_PARQUET_COLUMN_NAMES, Boolean.class); + } + + public static boolean isParquetIgnoreStatistics(ConnectorSession session) + { + return session.getProperty(PARQUET_IGNORE_STATISTICS, Boolean.class); + } + + public static boolean isParquetUseColumnIndex(ConnectorSession session) + { + return session.getProperty(PARQUET_USE_COLUMN_INDEX, Boolean.class); + } + + public static boolean useParquetBloomFilter(ConnectorSession session) + { + return session.getProperty(PARQUET_USE_BLOOM_FILTER, Boolean.class); + } + + public static DataSize getParquetMaxReadBlockSize(ConnectorSession session) + { + return session.getProperty(PARQUET_MAX_READ_BLOCK_SIZE, DataSize.class); + } + + public static int getParquetMaxReadBlockRowCount(ConnectorSession session) + { + return session.getProperty(PARQUET_MAX_READ_BLOCK_ROW_COUNT, Integer.class); + } + + public static DataSize getParquetSmallFileThreshold(ConnectorSession session) + { + return session.getProperty(PARQUET_SMALL_FILE_THRESHOLD, DataSize.class); + } + + public static boolean isParquetVectorizedDecodingEnabled(ConnectorSession session) + { + return session.getProperty(PARQUET_VECTORIZED_DECODING_ENABLED, Boolean.class); + } + + public static boolean isSizeBasedSplitWeightsEnabled(ConnectorSession session) + { + return session.getProperty(SIZE_BASED_SPLIT_WEIGHTS_ENABLED, Boolean.class); + } + + public static DataSize getStandardSplitWeightSize(ConnectorSession session) + { + return session.getProperty(STANDARD_SPLIT_WEIGHT_SIZE, DataSize.class); + } + + public static double getMinimumAssignedSplitWeight(ConnectorSession session) + { + return session.getProperty(MINIMUM_ASSIGNED_SPLIT_WEIGHT, Double.class); + } + + public static DataSize getTargetSplitSize(ConnectorSession session) + { + return session.getProperty(TARGET_SPLIT_SIZE, DataSize.class); + } + + public static int getMaxSplitsPerSecond(ConnectorSession session) + { + return session.getProperty(MAX_SPLITS_PER_SECOND, Integer.class); + } + + public static int getMaxOutstandingSplits(ConnectorSession session) + { + return session.getProperty(MAX_OUTSTANDING_SPLITS, Integer.class); + } + + public static int getSplitGeneratorParallelism(ConnectorSession session) + { + return session.getProperty(SPLIT_GENERATOR_PARALLELISM, Integer.class); + } + + public static boolean isQueryPartitionFilterRequired(ConnectorSession session) + { + return session.getProperty(QUERY_PARTITION_FILTER_REQUIRED, Boolean.class); + } + + public static boolean isIgnoreAbsentPartitions(ConnectorSession session) + { + return session.getProperty(IGNORE_ABSENT_PARTITIONS, Boolean.class); + } + + public static boolean isRecordLevelIndexEnabled(ConnectorSession session) + { + return session.getProperty(RECORD_LEVEL_INDEX_ENABLED, Boolean.class); + } + + public static boolean isSecondaryIndexEnabled(ConnectorSession session) + { + return session.getProperty(SECONDARY_INDEX_ENABLED, Boolean.class); + } + + public static boolean isColumnStatsIndexEnabled(ConnectorSession session) + { + return session.getProperty(COLUMN_STATS_INDEX_ENABLED, Boolean.class); + } + + public static boolean isPartitionStatsIndexEnabled(ConnectorSession session) + { + return session.getProperty(PARTITION_STATS_INDEX_ENABLED, Boolean.class); + } + + public static boolean isNoOpIndexEnabled(ConnectorSession session) + { + return !isRecordLevelIndexEnabled(session) && !isSecondaryIndexEnabled(session) && !isColumnStatsIndexEnabled(session); + } + + public static Duration getDynamicFilteringWaitTimeout(ConnectorSession session) + { + return session.getProperty(DYNAMIC_FILTERING_WAIT_TIMEOUT, Duration.class); + } + + public static Duration getColumnStatsWaitTimeout(ConnectorSession session) + { + return session.getProperty(COLUMN_STATS_WAIT_TIMEOUT, Duration.class); + } + + public static Duration getRecordIndexWaitTimeout(ConnectorSession session) + { + return session.getProperty(RECORD_INDEX_WAIT_TIMEOUT, Duration.class); + } + + public static Duration getSecondaryIndexWaitTimeout(ConnectorSession session) + { + return session.getProperty(SECONDARY_INDEX_WAIT_TIMEOUT, Duration.class); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplit.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplit.java new file mode 100644 index 0000000000000..c0c15eada837b --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplit.java @@ -0,0 +1,135 @@ +/* + * 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.hudi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hudi.file.HudiBaseFile; +import io.trino.plugin.hudi.file.HudiLogFile; +import io.trino.spi.SplitWeight; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.predicate.TupleDomain; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static io.airlift.slice.SizeOf.estimatedSizeOf; +import static io.airlift.slice.SizeOf.instanceSize; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public class HudiSplit + implements ConnectorSplit +{ + private static final int INSTANCE_SIZE = toIntExact(instanceSize(HudiSplit.class)); + + private final Optional baseFile; + private final List logFiles; + private final String commitTime; + private final TupleDomain predicate; + private final List partitionKeys; + private final SplitWeight splitWeight; + + @JsonCreator + public HudiSplit( + @JsonProperty("baseFile") HudiBaseFile baseFile, + @JsonProperty("logFiles") List logFiles, + @JsonProperty("commitTime") String commitTime, + @JsonProperty("predicate") TupleDomain predicate, + @JsonProperty("partitionKeys") List partitionKeys, + @JsonProperty("splitWeight") SplitWeight splitWeight) + { + this.baseFile = Optional.ofNullable(baseFile); + this.logFiles = requireNonNull(logFiles, "logFiles is null"); + this.commitTime = requireNonNull(commitTime, "commitTime is null"); + this.predicate = requireNonNull(predicate, "predicate is null"); + this.partitionKeys = ImmutableList.copyOf(requireNonNull(partitionKeys, "partitionKeys is null")); + this.splitWeight = requireNonNull(splitWeight, "splitWeight is null"); + } + + @Override + public Map getSplitInfo() + { + return ImmutableMap.builder() + .put("baseFile", baseFile.toString()) + .put("logFiles", logFiles.toString()) + .put("commitTime", commitTime) + .buildOrThrow(); + } + + @JsonProperty + public Optional getBaseFile() + { + return baseFile; + } + + @JsonProperty + public List getLogFiles() + { + return logFiles; + } + + @JsonProperty + public String getCommitTime() + { + return commitTime; + } + + @JsonProperty + @Override + public SplitWeight getSplitWeight() + { + return splitWeight; + } + + @JsonProperty + public TupleDomain getPredicate() + { + return predicate; + } + + @JsonProperty + public List getPartitionKeys() + { + return partitionKeys; + } + + @Override + public long getRetainedSizeInBytes() + { + return INSTANCE_SIZE + + 10 + + 10 + + estimatedSizeOf(commitTime) + + splitWeight.getRetainedSizeInBytes() + + predicate.getRetainedSizeInBytes(HiveColumnHandle::getRetainedSizeInBytes) + + estimatedSizeOf(partitionKeys, HivePartitionKey::estimatedSizeInBytes); + } + + @Override + public String toString() + { + return toStringHelper(this) + .addValue(baseFile) + .addValue(logFiles) + .addValue(commitTime) + .toString(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java new file mode 100644 index 0000000000000..8f774e3f5f156 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -0,0 +1,140 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.trino.metastore.HiveMetastore; +import io.trino.metastore.Partition; +import io.trino.metastore.StorageFormat; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HiveTransactionHandle; +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 io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.security.ConnectorIdentity; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +import static io.trino.plugin.hive.metastore.MetastoreUtil.computePartitionKeyFilter; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_PARTITION_NOT_FOUND; +import static io.trino.plugin.hudi.HudiSessionProperties.getDynamicFilteringWaitTimeout; +import static io.trino.plugin.hudi.HudiSessionProperties.getMaxOutstandingSplits; +import static io.trino.plugin.hudi.HudiSessionProperties.getMaxSplitsPerSecond; +import static io.trino.plugin.hudi.partition.HiveHudiPartitionInfo.NON_PARTITION; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class HudiSplitManager + implements ConnectorSplitManager +{ + private static final Logger log = Logger.get(HudiSplitManager.class); + private final BiFunction metastoreProvider; + private final ExecutorService executor; + private final ScheduledExecutorService splitLoaderExecutorService; + + @Inject + public HudiSplitManager( + BiFunction metastoreProvider, + @ForHudiSplitManager ExecutorService executor, + @ForHudiSplitSource ScheduledExecutorService splitLoaderExecutorService) + { + this.metastoreProvider = requireNonNull(metastoreProvider, "metastoreProvider is null"); + this.executor = requireNonNull(executor, "executor is null"); + this.splitLoaderExecutorService = requireNonNull(splitLoaderExecutorService, "splitLoaderExecutorService is null"); + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle tableHandle, + DynamicFilter dynamicFilter, + Constraint constraint) + { + HudiTableHandle hudiTableHandle = (HudiTableHandle) tableHandle; + HiveMetastore metastore = metastoreProvider.apply(session.getIdentity(), (HiveTransactionHandle) transaction); + Lazy> lazyAllPartitions = Lazy.lazily(() -> { + HoodieTimer timer = HoodieTimer.start(); + Map allPartitions = getPartitions(metastore, hudiTableHandle); + log.info("Found %s partitions for table %s.%s in %s ms", + allPartitions.size(), hudiTableHandle.getSchemaName(), hudiTableHandle.getTableName(), timer.endTimer()); + return allPartitions; + }); + + HudiSplitSource splitSource = new HudiSplitSource( + session, + hudiTableHandle, + executor, + splitLoaderExecutorService, + getMaxSplitsPerSecond(session), + getMaxOutstandingSplits(session), + lazyAllPartitions, + dynamicFilter, + getDynamicFilteringWaitTimeout(session)); + return new ClassLoaderSafeConnectorSplitSource(splitSource, HudiSplitManager.class.getClassLoader()); + } + + private static Map getPartitions( + HiveMetastore metastore, + HudiTableHandle tableHandle) + { + List partitionColumns = tableHandle.getPartitionColumns(); + if (partitionColumns.isEmpty()) { + return ImmutableMap.of( + NON_PARTITION, Partition.builder() + .setDatabaseName(tableHandle.getSchemaName()) + .setTableName(tableHandle.getTableName()) + .withStorage(storageBuilder -> + storageBuilder.setLocation(tableHandle.getBasePath()) + .setStorageFormat(StorageFormat.NULL_STORAGE_FORMAT)) + .setColumns(ImmutableList.of()) + .setValues(ImmutableList.of()) + .build()); + } + + List partitionNames = metastore.getPartitionNamesByFilter( + tableHandle.getSchemaName(), + tableHandle.getTableName(), + partitionColumns.stream().map(HiveColumnHandle::getName).collect(Collectors.toList()), + computePartitionKeyFilter(partitionColumns, tableHandle.getPartitionPredicates())) + .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())); + Map> partitionsByNames = metastore.getPartitionsByNames(tableHandle.getTable(), partitionNames); + List partitionsNotFound = partitionsByNames.entrySet().stream().filter(e -> e.getValue().isEmpty()).map(Map.Entry::getKey).toList(); + if (!partitionsNotFound.isEmpty()) { + throw new TrinoException(HUDI_PARTITION_NOT_FOUND, format("Cannot find partitions in metastore: %s", partitionsNotFound)); + } + return partitionsByNames + .entrySet().stream() + .filter(e -> e.getValue().isPresent()) + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().get())); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java new file mode 100644 index 0000000000000..f0286aea7e951 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -0,0 +1,239 @@ +/* + * 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.hudi; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.Futures; +import io.airlift.concurrent.BoundedExecutor; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; +import io.trino.metastore.Partition; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.util.AsyncQueue; +import io.trino.plugin.hive.util.HiveUtil; +import io.trino.plugin.hive.util.ThrottledAsyncQueue; +import io.trino.plugin.hudi.query.HudiDirectoryLister; +import io.trino.plugin.hudi.query.HudiSnapshotDirectoryLister; +import io.trino.plugin.hudi.split.HudiBackgroundSplitLoader; +import io.trino.plugin.hudi.split.HudiSplitWeightProvider; +import io.trino.plugin.hudi.split.SizeBasedSplitWeightProvider; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.NullableValue; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.util.Lazy; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.OptionalInt; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.concurrent.MoreFutures.toCompletableFuture; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT; +import static io.trino.plugin.hudi.HudiSessionProperties.getMinimumAssignedSplitWeight; +import static io.trino.plugin.hudi.HudiSessionProperties.getSplitGeneratorParallelism; +import static io.trino.plugin.hudi.HudiSessionProperties.getStandardSplitWeightSize; +import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataTableEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.isSizeBasedSplitWeightsEnabled; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class HudiSplitSource + implements ConnectorSplitSource +{ + private static final ConnectorSplitBatch EMPTY_BATCH = new ConnectorSplitBatch(ImmutableList.of(), false); + private final AsyncQueue queue; + private final ScheduledFuture splitLoaderFuture; + private final AtomicReference trinoException = new AtomicReference<>(); + private final DynamicFilter dynamicFilter; + private final long dynamicFilteringWaitTimeoutMillis; + private final Stopwatch dynamicFilterWaitStopwatch; + + public HudiSplitSource( + ConnectorSession session, + HudiTableHandle tableHandle, + ExecutorService executor, + ScheduledExecutorService splitLoaderExecutorService, + int maxSplitsPerSecond, + int maxOutstandingSplits, + Lazy> lazyPartitions, + DynamicFilter dynamicFilter, + Duration dynamicFilteringWaitTimeoutMillis) + { + boolean enableMetadataTable = isHudiMetadataTableEnabled(session); + Lazy lazyTableMetadata = Lazy.lazily(() -> { + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(enableMetadataTable) + .build(); + HoodieTableMetaClient metaClient = tableHandle.getMetaClient(); + HoodieEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getStorage().getConf()); + return HoodieTableMetadata.create( + engineContext, + tableHandle.getMetaClient().getStorage(), metadataConfig, metaClient.getBasePath().toString(), true); + }); + + HudiDirectoryLister hudiDirectoryLister = new HudiSnapshotDirectoryLister( + session, + tableHandle, + enableMetadataTable, + lazyTableMetadata, + lazyPartitions); + + this.queue = new ThrottledAsyncQueue<>(maxSplitsPerSecond, maxOutstandingSplits, executor); + HudiBackgroundSplitLoader splitLoader = new HudiBackgroundSplitLoader( + session, + tableHandle, + hudiDirectoryLister, + queue, + new BoundedExecutor(executor, getSplitGeneratorParallelism(session)), + createSplitWeightProvider(session), + lazyPartitions, + enableMetadataTable, + lazyTableMetadata, + throwable -> { + trinoException.compareAndSet(null, new TrinoException(HUDI_CANNOT_OPEN_SPLIT, + "Failed to generate splits for " + tableHandle.getSchemaTableName(), throwable)); + queue.finish(); + }); + this.splitLoaderFuture = splitLoaderExecutorService.schedule(splitLoader, 0, TimeUnit.MILLISECONDS); + this.dynamicFilter = requireNonNull(dynamicFilter, "dynamicFilter is null"); + this.dynamicFilteringWaitTimeoutMillis = dynamicFilteringWaitTimeoutMillis.toMillis(); + this.dynamicFilterWaitStopwatch = Stopwatch.createStarted(); + } + + @Override + public CompletableFuture getNextBatch(int maxSize) + { + // If dynamic filtering is enabled and we haven't timed out, wait for the build side to provide the dynamic filter. + long timeLeft = dynamicFilteringWaitTimeoutMillis - dynamicFilterWaitStopwatch.elapsed(MILLISECONDS); + if (dynamicFilter.isAwaitable() && timeLeft > 0) { + // If the filter is not ready, return an empty batch. The query engine will call getNextBatch() again. + // As long as isFinished() is false, effectively polling until the filter is ready or timeout occurs. + return dynamicFilter.isBlocked() + .thenApply(_ -> EMPTY_BATCH) + .completeOnTimeout(EMPTY_BATCH, timeLeft, MILLISECONDS); + } + + TupleDomain dynamicFilterPredicate = + dynamicFilter.getCurrentPredicate().transformKeys(HiveColumnHandle.class::cast); + + if (dynamicFilterPredicate.isNone()) { + close(); + return completedFuture(new ConnectorSplitBatch(ImmutableList.of(), true)); + } + + boolean noMoreSplits = isFinished(); + Throwable throwable = trinoException.get(); + if (throwable != null) { + return CompletableFuture.failedFuture(throwable); + } + + return toCompletableFuture(Futures.transform( + queue.getBatchAsync(maxSize), + splits -> + { + List filteredSplits = splits.stream() + .filter(split -> partitionMatchesPredicate((HudiSplit) split, dynamicFilterPredicate)) + .collect(toImmutableList()); + return new ConnectorSplitBatch(filteredSplits, noMoreSplits); + }, + directExecutor())); + } + + @Override + public void close() + { + queue.finish(); + } + + @Override + public boolean isFinished() + { + return splitLoaderFuture.isDone() && queue.isFinished(); + } + + public static HudiSplitWeightProvider createSplitWeightProvider(ConnectorSession session) + { + if (isSizeBasedSplitWeightsEnabled(session)) { + DataSize standardSplitWeightSize = getStandardSplitWeightSize(session); + double minimumAssignedSplitWeight = getMinimumAssignedSplitWeight(session); + return new SizeBasedSplitWeightProvider(minimumAssignedSplitWeight, standardSplitWeightSize); + } + return HudiSplitWeightProvider.uniformStandardWeightProvider(); + } + + static boolean partitionMatchesPredicate( + HudiSplit split, + TupleDomain dynamicFilterPredicate) + { + if (dynamicFilterPredicate.isNone()) { + return false; + } + + // Pre-process the filter predicate to get a map of relevant partition domains keyed by partition column name + Map> filterPartitionDomains = new HashMap<>(); + if (dynamicFilterPredicate.getDomains().isPresent()) { + for (Map.Entry entry : dynamicFilterPredicate.getDomains().get().entrySet()) { + HiveColumnHandle column = entry.getKey(); + if (column.isPartitionKey()) { + filterPartitionDomains.put(column.getName(), entry); + } + } + } + + // Match each partition key from the split against the pre-processed filter domains + for (HivePartitionKey splitPartitionKey : split.getPartitionKeys()) { + Map.Entry filterInfo = filterPartitionDomains.get(splitPartitionKey.name()); + + if (filterInfo == null) { + // filterInfo is null, the partition key is not constrained by the filter + continue; + } + + HiveColumnHandle filterColumnHandle = filterInfo.getKey(); + Domain filterDomain = filterInfo.getValue(); + + NullableValue value = HiveUtil.getPrefilledColumnValue( + filterColumnHandle, + splitPartitionKey, + null, OptionalInt.empty(), 0, 0, ""); + + // Split does not match this filter condition + if (!filterDomain.includesNullableValue(value.getValue())) { + return false; + } + } + return true; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java new file mode 100644 index 0000000000000..d6a152446ef0a --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableHandle.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.hudi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableSet; +import io.trino.metastore.Table; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.spi.connector.SchemaTableName.schemaTableName; +import static java.util.Objects.requireNonNull; + +public class HudiTableHandle + implements ConnectorTableHandle +{ + private final String schemaName; + private final String tableName; + private final String basePath; + private final HoodieTableType tableType; + private final List partitionColumns; + // Used only for validation when config property hudi.query-partition-filter-required is enabled + private final Set constraintColumns; + private final TupleDomain partitionPredicates; + private final TupleDomain regularPredicates; + // Coordinator-only + private final transient Optional
table; + private final transient Optional> lazyMetaClient; + private final transient Optional> lazyLatestCommitTime; + + @JsonCreator + public HudiTableHandle( + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("basePath") String basePath, + @JsonProperty("tableType") HoodieTableType tableType, + @JsonProperty("partitionColumns") List partitionColumns, + @JsonProperty("partitionPredicates") TupleDomain partitionPredicates, + @JsonProperty("regularPredicates") TupleDomain regularPredicates) + { + this(Optional.empty(), Optional.empty(), schemaName, tableName, basePath, tableType, partitionColumns, ImmutableSet.of(), partitionPredicates, regularPredicates); + } + + public HudiTableHandle( + Optional
table, + Optional> lazyMetaClient, + String schemaName, + String tableName, + String basePath, + HoodieTableType tableType, + List partitionColumns, + Set constraintColumns, + TupleDomain partitionPredicates, + TupleDomain regularPredicates) + { + this.table = requireNonNull(table, "table is null"); + this.lazyMetaClient = requireNonNull(lazyMetaClient, "lazyMetaClient is null"); + this.lazyLatestCommitTime = Optional.of(Lazy.lazily(() -> + getMetaClient().getActiveTimeline() + .getCommitsTimeline() + .filterCompletedInstants() + .lastInstant() + .map(HoodieInstant::requestedTime) + .orElseThrow(() -> new TrinoException(HudiErrorCode.HUDI_NO_VALID_COMMIT, "Table has no valid commits")))); + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.basePath = requireNonNull(basePath, "basePath is null"); + this.tableType = requireNonNull(tableType, "tableType is null"); + this.partitionColumns = requireNonNull(partitionColumns, "partitionColumns is null"); + this.constraintColumns = requireNonNull(constraintColumns, "constraintColumns is null"); + this.partitionPredicates = requireNonNull(partitionPredicates, "partitionPredicates is null"); + this.regularPredicates = requireNonNull(regularPredicates, "regularPredicates is null"); + } + + public Table getTable() + { + checkArgument(table.isPresent(), + "getTable() called on a table handle that has no metastore table object; " + + "this is likely because it is called on the worker."); + return table.get(); + } + + public HoodieTableMetaClient getMetaClient() + { + checkArgument(lazyMetaClient.isPresent(), + "getMetaClient() called on a table handle that has no Hudi meta-client; " + + "this is likely because it is called on the worker."); + return lazyMetaClient.get().get(); + } + + public String getLatestCommitTime() + { + checkArgument(lazyLatestCommitTime.isPresent(), + "getLatestCommitTime() called on a table handle that has no Hudi meta-client; " + + "this is likely because it is called on the worker."); + return lazyLatestCommitTime.get().get(); + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public String getBasePath() + { + return basePath; + } + + @JsonProperty + public HoodieTableType getTableType() + { + return tableType; + } + + @JsonProperty + public TupleDomain getPartitionPredicates() + { + return partitionPredicates; + } + + @JsonProperty + public List getPartitionColumns() + { + return partitionColumns; + } + + // do not serialize constraint columns as they are not needed on workers + @JsonIgnore + public Set getConstraintColumns() + { + return constraintColumns; + } + + @JsonProperty + public TupleDomain getRegularPredicates() + { + return regularPredicates; + } + + public SchemaTableName getSchemaTableName() + { + return schemaTableName(schemaName, tableName); + } + + HudiTableHandle applyPredicates( + Set constraintColumns, + TupleDomain partitionTupleDomain, + TupleDomain regularTupleDomain) + { + return new HudiTableHandle( + table, + lazyMetaClient, + schemaName, + tableName, + basePath, + tableType, + partitionColumns, + constraintColumns, + partitionPredicates.intersect(partitionTupleDomain), + regularPredicates.intersect(regularTupleDomain)); + } + + @Override + public String toString() + { + return getSchemaTableName().toString(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableInfo.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableInfo.java new file mode 100644 index 0000000000000..1a7d6984e16f8 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableInfo.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.hudi; + +import io.trino.spi.connector.SchemaTableName; + +import static java.util.Objects.requireNonNull; + +public record HudiTableInfo(SchemaTableName table, String tableType, String basePath) +{ + public HudiTableInfo + { + requireNonNull(table, "table is null"); + requireNonNull(tableType, "tableType is null"); + requireNonNull(basePath, "basePath is null"); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableName.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableName.java new file mode 100644 index 0000000000000..a16f35e85cebc --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableName.java @@ -0,0 +1,65 @@ +/* + * 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.hudi; + +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.google.common.base.Verify.verify; +import static io.trino.plugin.hudi.TableType.DATA; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public record HudiTableName(String tableName, TableType tableType) +{ + private static final Pattern TABLE_PATTERN = Pattern.compile( + "(?
[^$@]+)(?:\\$(?(?i:timeline)))?"); + + public HudiTableName(String tableName, TableType tableType) + { + this.tableName = requireNonNull(tableName, "tableName is null"); + this.tableType = requireNonNull(tableType, "tableType is null"); + } + + public String tableNameWithType() + { + return tableName + "$" + tableType.name().toLowerCase(ENGLISH); + } + + @Override + public String toString() + { + return tableNameWithType(); + } + + public static Optional from(String name) + { + Matcher match = TABLE_PATTERN.matcher(name); + if (!match.matches()) { + return Optional.empty(); + } + + String table = match.group("table"); + String typeString = match.group("type"); + + TableType type = DATA; + if (typeString != null) { + type = TableType.valueOf(typeString.toUpperCase(ENGLISH)); + verify(type != DATA, "parsedType is unexpectedly DATA"); + } + + return Optional.of(new HudiTableName(table, type)); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java new file mode 100644 index 0000000000000..ddb2c01e8bc22 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java @@ -0,0 +1,70 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.trino.spi.session.PropertyMetadata; +import io.trino.spi.type.ArrayType; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.spi.session.PropertyMetadata.stringProperty; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.util.Locale.ENGLISH; + +public class HudiTableProperties +{ + public static final String LOCATION_PROPERTY = "location"; + public static final String PARTITIONED_BY_PROPERTY = "partitioned_by"; + + private final List> tableProperties; + + @Inject + public HudiTableProperties() + { + tableProperties = ImmutableList.>builder() + .add(stringProperty( + LOCATION_PROPERTY, + "File system location URI for the table", + null, + false)) + .add(new PropertyMetadata<>( + PARTITIONED_BY_PROPERTY, + "Partition columns", + new ArrayType(VARCHAR), + List.class, + ImmutableList.of(), + false, + value -> ((Collection) value).stream() + .map(name -> name.toLowerCase(ENGLISH)) + .collect(toImmutableList()), + value -> value)) + .build(); + } + + public List> getTableProperties() + { + return tableProperties; + } + + public static Optional getTableLocation(Map tableProperties) + { + return Optional.ofNullable((String) tableProperties.get(LOCATION_PROPERTY)); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java new file mode 100644 index 0000000000000..994a4d929359e --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java @@ -0,0 +1,80 @@ +/* + * 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.hudi; + +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.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +public class HudiTransactionManager +{ + private final Map transactions = new ConcurrentHashMap<>(); + private final HudiMetadataFactory metadataFactory; + + @Inject + public HudiTransactionManager(HudiMetadataFactory metadataFactory) + { + this.metadataFactory = requireNonNull(metadataFactory, "metadataFactory is null"); + } + + public HudiMetadata get(ConnectorTransactionHandle transaction, ConnectorIdentity identity) + { + HudiMetadata metadata = transactions.get(transaction).get(identity); + checkArgument(metadata != null, "no such transaction: %s", transaction); + return metadata; + } + + public void commit(ConnectorTransactionHandle transaction) + { + MemoizedMetadata metadata = transactions.remove(transaction); + checkArgument(metadata != null, "no such transaction: %s", transaction); + } + + public void rollback(ConnectorTransactionHandle transaction) + { + MemoizedMetadata transactionalMetadata = transactions.remove(transaction); + checkArgument(transactionalMetadata != null, "no such transaction: %s", transaction); + } + + public void put(ConnectorTransactionHandle transaction) + { + MemoizedMetadata existing = transactions.putIfAbsent(transaction, new MemoizedMetadata()); + checkState(existing == null, "transaction already exists: %s", existing); + } + + private class MemoizedMetadata + { + @GuardedBy("this") + private HudiMetadata metadata; + + public synchronized HudiMetadata get(ConnectorIdentity identity) + { + if (metadata == null) { + try (ThreadContextClassLoader _ = new ThreadContextClassLoader(getClass().getClassLoader())) { + metadata = metadataFactory.create(identity); + } + } + return metadata; + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiUtil.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiUtil.java new file mode 100644 index 0000000000000..05c85efdbe24d --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/HudiUtil.java @@ -0,0 +1,286 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.metastore.HivePartition; +import io.trino.metastore.HiveType; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.HivePartitionManager; +import io.trino.plugin.hive.avro.AvroHiveFileUtils; +import io.trino.plugin.hudi.storage.HudiTrinoStorage; +import io.trino.plugin.hudi.storage.TrinoStorageConfiguration; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.NullableValue; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.VarcharType; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.storage.StoragePath; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static io.trino.plugin.hive.util.HiveUtil.checkCondition; +import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; +import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_FILESYSTEM_ERROR; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_META_CLIENT_ERROR; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_UNSUPPORTED_FILE_FORMAT; +import static org.apache.hudi.common.model.HoodieRecord.HOODIE_META_COLUMNS; + +public final class HudiUtil +{ + private HudiUtil() {} + + public static HoodieFileFormat getHudiFileFormat(String path) + { + String extension = getFileExtension(path); + if (extension.equals(HoodieFileFormat.PARQUET.getFileExtension())) { + return HoodieFileFormat.PARQUET; + } + if (extension.equals(HoodieFileFormat.HOODIE_LOG.getFileExtension())) { + return HoodieFileFormat.HOODIE_LOG; + } + if (extension.equals(HoodieFileFormat.ORC.getFileExtension())) { + return HoodieFileFormat.ORC; + } + if (extension.equals(HoodieFileFormat.HFILE.getFileExtension())) { + return HoodieFileFormat.HFILE; + } + throw new TrinoException(HUDI_UNSUPPORTED_FILE_FORMAT, "Hoodie InputFormat not implemented for base file of type " + extension); + } + + private static String getFileExtension(String fullName) + { + String fileName = Location.of(fullName).fileName(); + int dotIndex = fileName.lastIndexOf('.'); + return dotIndex == -1 ? "" : fileName.substring(dotIndex); + } + + public static boolean hudiMetadataExists(TrinoFileSystem trinoFileSystem, Location baseLocation) + { + try { + Location metaLocation = baseLocation.appendPath(HoodieTableMetaClient.METAFOLDER_NAME); + FileIterator iterator = trinoFileSystem.listFiles(metaLocation); + // If there is at least one file in the .hoodie directory, it's a valid Hudi table + return iterator.hasNext(); + } + catch (IOException e) { + throw new TrinoException(HUDI_FILESYSTEM_ERROR, "Failed to check for Hudi table at location: " + baseLocation, e); + } + } + + public static boolean partitionMatchesPredicates( + SchemaTableName tableName, + String hivePartitionName, + List partitionColumnHandles, + TupleDomain constraintSummary) + { + HivePartition partition = HivePartitionManager.parsePartition( + tableName, hivePartitionName, partitionColumnHandles); + + return partitionMatches(partitionColumnHandles, constraintSummary, partition); + } + + public static boolean partitionMatches(List partitionColumns, TupleDomain constraintSummary, HivePartition partition) + { + if (constraintSummary.isNone()) { + return false; + } + Map domains = constraintSummary.getDomains().orElseGet(ImmutableMap::of); + for (HiveColumnHandle column : partitionColumns) { + NullableValue value = partition.getKeys().get(column); + Domain allowedDomain = domains.get(column); + if (allowedDomain != null && !allowedDomain.includesNullableValue(value.getValue())) { + return false; + } + } + return true; + } + + public static List buildPartitionKeys(List keys, List values) + { + checkCondition(keys.size() == values.size(), HIVE_INVALID_METADATA, + "Expected %s partition key values, but got %s. Keys: %s, Values: %s.", + keys.size(), values.size(), keys, values); + ImmutableList.Builder partitionKeys = ImmutableList.builder(); + for (int i = 0; i < keys.size(); i++) { + String name = keys.get(i).getName(); + String value = values.get(i); + partitionKeys.add(new HivePartitionKey(name, value)); + } + return partitionKeys.build(); + } + + public static HoodieTableMetaClient buildTableMetaClient( + TrinoFileSystem fileSystem, + String tableName, + String basePath) + { + try { + return HoodieTableMetaClient.builder() + .setStorage(new HudiTrinoStorage(fileSystem, new TrinoStorageConfiguration())) + .setBasePath(basePath) + .build(); + } + catch (TableNotFoundException e) { + throw new TrinoException(HUDI_BAD_DATA, + "Location of table %s does not contain Hudi table metadata: %s".formatted(tableName, basePath)); + } + catch (Throwable e) { + throw new TrinoException(HUDI_META_CLIENT_ERROR, + "Unable to load Hudi meta client for table %s (%s)".formatted(tableName, basePath)); + } + } + + public static Schema constructSchema(List columnNames, List columnTypes) + { + // Convert lists into the format expected by the utility class + String columnNamesString = String.join(",", columnNames); + String columnTypesString = columnTypes.stream() + .map(HiveType::getHiveTypeName) + .map(Object::toString) + .collect(Collectors.joining(":")); + + // Create the properties map + Map properties = new HashMap<>(); + properties.put(LIST_COLUMNS, columnNamesString); + properties.put(LIST_COLUMN_TYPES, columnTypesString); + + // Call the public static method to build the schema + try { + // Pass null for the file system as we are not reading from a URL + return AvroHiveFileUtils.determineSchemaOrThrowException(null, properties); + } + catch (IOException e) { + // The IOException is declared on the method, but this path shouldn't throw it + throw new UncheckedIOException("Failed to construct Avro schema", e); + } + } + + public static Schema constructSchema(Schema dataSchema, List columnNames) + { + SchemaBuilder.RecordBuilder schemaBuilder = SchemaBuilder.record("baseRecord"); + SchemaBuilder.FieldAssembler fieldBuilder = schemaBuilder.fields(); + for (String columnName : columnNames) { + Schema originalFieldSchema = dataSchema.getField(columnName).schema(); + Schema typeForNewField; + + // Check if the original field schema is already nullable (i.e., a UNION containing NULL) + if (originalFieldSchema.isNullable()) { + typeForNewField = originalFieldSchema; + } + else { + typeForNewField = Schema.createUnion(Schema.create(Schema.Type.NULL), originalFieldSchema); + } + + fieldBuilder = fieldBuilder + .name(columnName) + .type(typeForNewField) + .withDefault(null); + } + return fieldBuilder.endRecord(); + } + + public static List prependHudiMetaColumns(List dataColumns) + { + //For efficient lookup + Set dataColumnNames = dataColumns.stream() + .map(HiveColumnHandle::getName) + .collect(Collectors.toSet()); + + // If all Hudi meta columns are already present, return the original list + if (dataColumnNames.containsAll(HOODIE_META_COLUMNS)) { + return dataColumns; + } + + // Identify only the meta columns that are missing from dataColumns to avoid duplicates + List missingMetaColumns = HOODIE_META_COLUMNS.stream() + .filter(metaColumn -> !dataColumnNames.contains(metaColumn)) + .toList(); + + List columns = new ArrayList<>(); + + // Create and prepend the new HiveColumnHandles for the missing meta columns + columns.addAll(IntStream.range(0, missingMetaColumns.size()) + .boxed() + .map(i -> new HiveColumnHandle( + missingMetaColumns.get(i), + i, + HiveType.HIVE_STRING, + VarcharType.VARCHAR, + Optional.empty(), + HiveColumnHandle.ColumnType.REGULAR, + Optional.empty())) + .toList()); + + // Add all the original data columns after the new meta columns + columns.addAll(dataColumns); + + return columns; + } + + public static FileSlice convertToFileSlice(HudiSplit split, String basePath) + { + String dataFilePath = split.getBaseFile().isPresent() + ? split.getBaseFile().get().getPath() + : split.getLogFiles().getFirst().getPath(); + String fileId = FSUtils.getFileIdFromFileName(new StoragePath(dataFilePath).getName()); + HoodieBaseFile baseFile = split.getBaseFile().isPresent() + ? new HoodieBaseFile(dataFilePath, fileId, split.getCommitTime(), null) + : null; + + return new FileSlice( + new HoodieFileGroupId(FSUtils.getRelativePartitionPath(new StoragePath(basePath), new StoragePath(dataFilePath)), fileId), + split.getCommitTime(), + baseFile, + split.getLogFiles().stream().map(lf -> new HoodieLogFile(lf.getPath())).toList()); + } + + public static HoodieTableFileSystemView getFileSystemView( + HoodieTableMetadata tableMetadata, + HoodieTableMetaClient metaClient) + { + return new HoodieTableFileSystemView( + tableMetadata, metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/TableType.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/TableType.java new file mode 100644 index 0000000000000..2a8238c77f87b --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/TableType.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.hudi; + +public enum TableType +{ + DATA, + TIMELINE +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/TimelineTable.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/TimelineTable.java new file mode 100644 index 0000000000000..1024d87a0b5c2 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/TimelineTable.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.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.metastore.Table; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.InMemoryRecordSet; +import io.trino.spi.connector.RecordCursor; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SystemTable; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.Type; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; + +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.hudi.HudiUtil.buildTableMetaClient; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.util.Objects.requireNonNull; + +public class TimelineTable + implements SystemTable +{ + private final ConnectorTableMetadata tableMetadata; + private final List types; + private final TrinoFileSystem fileSystem; + private final String location; + + public TimelineTable(TrinoFileSystem fileSystem, SchemaTableName tableName, Table hudiTable) + { + this.tableMetadata = new ConnectorTableMetadata(requireNonNull(tableName, "tableName is null"), + ImmutableList.builder() + .add(new ColumnMetadata("timestamp", VARCHAR)) + .add(new ColumnMetadata("action", VARCHAR)) + .add(new ColumnMetadata("state", VARCHAR)) + .build()); + this.types = tableMetadata.getColumns().stream().map(ColumnMetadata::getType).collect(toImmutableList()); + this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); + this.location = requireNonNull(hudiTable.getStorage().getLocation(), "location is null"); + } + + @Override + public Distribution getDistribution() + { + return Distribution.SINGLE_COORDINATOR; + } + + @Override + public ConnectorTableMetadata getTableMetadata() + { + return tableMetadata; + } + + @Override + public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, ConnectorSession session, TupleDomain constraint) + { + HoodieTableMetaClient metaClient = buildTableMetaClient(fileSystem, tableMetadata.getTable().toString(), location); + Iterable> records = () -> metaClient.getCommitsTimeline().getInstants().stream() + .map(this::getRecord).iterator(); + return new InMemoryRecordSet(types, records).cursor(); + } + + private List getRecord(HoodieInstant hudiInstant) + { + List columns = new ArrayList<>(); + columns.add(hudiInstant.requestedTime()); + columns.add(hudiInstant.getAction()); + columns.add(hudiInstant.getState().toString()); + checkArgument(columns.size() == types.size(), "Expected %s types in row, but got %s values", types.size(), columns.size()); + return columns; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiBaseFile.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiBaseFile.java new file mode 100644 index 0000000000000..f21a9b56bd011 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiBaseFile.java @@ -0,0 +1,113 @@ +/* + * 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.hudi.file; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hudi.common.model.HoodieBaseFile; + +import static com.google.common.base.Preconditions.checkArgument; + +public class HudiBaseFile + implements HudiFile +{ + private final String path; + private final String fileName; + private final long fileSize; + private final long modificationTime; + private final long start; + private final long length; + + public static HudiBaseFile of(HoodieBaseFile baseFile) + { + return of(baseFile, 0, baseFile.getFileSize()); + } + + public static HudiBaseFile of(HoodieBaseFile baseFile, long start, long length) + { + return new HudiBaseFile(baseFile, start, length); + } + + @JsonCreator + public HudiBaseFile(@JsonProperty("path") String path, + @JsonProperty("fileName") String fileName, + @JsonProperty("fileSize") long fileSize, + @JsonProperty("modificationTime") long modificationTime, + @JsonProperty("start") long start, + @JsonProperty("length") long length) + { + this.path = path; + this.fileName = fileName; + this.fileSize = fileSize; + this.modificationTime = modificationTime; + this.start = start; + this.length = length; + } + + private HudiBaseFile(HoodieBaseFile baseFile, long start, long length) + { + checkArgument(baseFile != null, "baseFile is null"); + checkArgument(start >= 0, "start must be positive"); + checkArgument(length >= 0, "length must be positive"); + checkArgument(start + length <= baseFile.getFileSize(), "fileSize must be at least start + length"); + this.path = baseFile.getPath(); + this.fileName = baseFile.getFileName(); + this.fileSize = baseFile.getFileSize(); + this.modificationTime = baseFile.getPathInfo().getModificationTime(); + this.start = start; + this.length = length; + } + + @JsonProperty + @Override + public String getPath() + { + return path; + } + + @JsonProperty + @Override + public String getFileName() + { + return fileName; + } + + @JsonProperty + @Override + public long getFileSize() + { + return fileSize; + } + + @JsonProperty + @Override + public long getModificationTime() + { + return modificationTime; + } + + @JsonProperty + @Override + public long getStart() + { + return start; + } + + @JsonProperty + @Override + public long getLength() + { + return length; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiFile.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiFile.java new file mode 100644 index 0000000000000..00e2ac66f9c7f --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiFile.java @@ -0,0 +1,29 @@ +/* + * 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.hudi.file; + +public interface HudiFile +{ + String getPath(); + + String getFileName(); + + long getFileSize(); + + long getModificationTime(); + + long getStart(); + + long getLength(); +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiLogFile.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiLogFile.java new file mode 100644 index 0000000000000..0ed071f233809 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/file/HudiLogFile.java @@ -0,0 +1,113 @@ +/* + * 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.hudi.file; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hudi.common.model.HoodieLogFile; + +import static com.google.common.base.Preconditions.checkArgument; + +public class HudiLogFile + implements HudiFile +{ + private final String path; + private final String fileName; + private final long fileSize; + private final long modificationTime; + private final long start; + private final long length; + + public static HudiLogFile of(HoodieLogFile logFile) + { + return of(logFile, 0, logFile.getFileSize()); + } + + public static HudiLogFile of(HoodieLogFile logFile, long start, long length) + { + return new HudiLogFile(logFile, start, length); + } + + @JsonCreator + public HudiLogFile(@JsonProperty("path") String path, + @JsonProperty("fileName") String fileName, + @JsonProperty("fileSize") long fileSize, + @JsonProperty("modificationTime") long modificationTime, + @JsonProperty("start") long start, + @JsonProperty("length") long length) + { + this.path = path; + this.fileName = fileName; + this.fileSize = fileSize; + this.modificationTime = modificationTime; + this.start = start; + this.length = length; + } + + private HudiLogFile(HoodieLogFile logFile, long start, long length) + { + checkArgument(logFile != null, "logFile is null"); + checkArgument(start >= 0, "start must be positive"); + checkArgument(length >= 0, "length must be positive"); + checkArgument(start + length <= logFile.getFileSize(), "fileSize must be at least start + length"); + this.path = logFile.getPath().toString(); + this.fileName = logFile.getFileName(); + this.fileSize = logFile.getFileSize(); + this.modificationTime = logFile.getPathInfo().getModificationTime(); + this.start = start; + this.length = length; + } + + @JsonProperty + @Override + public String getPath() + { + return path; + } + + @JsonProperty + @Override + public String getFileName() + { + return fileName; + } + + @JsonProperty + @Override + public long getFileSize() + { + return fileSize; + } + + @JsonProperty + @Override + public long getModificationTime() + { + return modificationTime; + } + + @JsonProperty + @Override + public long getStart() + { + return start; + } + + @JsonProperty + @Override + public long getLength() + { + return length; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/HudiTrinoFileReaderFactory.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/HudiTrinoFileReaderFactory.java new file mode 100644 index 0000000000000..7e8bb967ffc72 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/HudiTrinoFileReaderFactory.java @@ -0,0 +1,73 @@ +/* + * 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.hudi.io; + +import org.apache.avro.Schema; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.io.storage.HoodieAvroBootstrapFileReader; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.io.storage.HoodieNativeAvroHFileReader; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; + +import java.io.IOException; + +public class HudiTrinoFileReaderFactory + extends HoodieFileReaderFactory +{ + public HudiTrinoFileReaderFactory(HoodieStorage storage) + { + super(storage); + } + + @Override + protected HoodieFileReader newParquetFileReader(StoragePath path) + { + throw new UnsupportedOperationException("HudiTrinoFileReaderFactory does not support Parquet file reader"); + } + + @Override + protected HoodieFileReader newHFileFileReader(HoodieConfig hoodieConfig, + StoragePath path, + Option schemaOption) + throws IOException + { + return new HoodieNativeAvroHFileReader(storage, path, schemaOption); + } + + @Override + protected HoodieFileReader newHFileFileReader(HoodieConfig hoodieConfig, + StoragePath path, + HoodieStorage storage, + byte[] content, + Option schemaOption) + throws IOException + { + return new HoodieNativeAvroHFileReader(this.storage, content, schemaOption); + } + + @Override + protected HoodieFileReader newOrcFileReader(StoragePath path) + { + throw new UnsupportedOperationException("HudiTrinoFileReaderFactory does not support ORC file reader"); + } + + @Override + public HoodieFileReader newBootstrapFileReader(HoodieFileReader skeletonFileReader, HoodieFileReader dataFileReader, Option partitionFields, Object[] partitionValues) + { + return new HoodieAvroBootstrapFileReader(skeletonFileReader, dataFileReader, partitionFields, partitionValues); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/HudiTrinoIOFactory.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/HudiTrinoIOFactory.java new file mode 100644 index 0000000000000..8044e92b58ec7 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/HudiTrinoIOFactory.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.hudi.io; + +import org.apache.hudi.common.fs.ConsistencyGuard; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.FileFormatUtils; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.io.storage.HoodieFileWriterFactory; +import org.apache.hudi.io.storage.HoodieIOFactory; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; + +public class HudiTrinoIOFactory + extends HoodieIOFactory +{ + public HudiTrinoIOFactory(HoodieStorage storage) + { + super(storage); + } + + @Override + public HoodieFileReaderFactory getReaderFactory(HoodieRecord.HoodieRecordType recordType) + { + return new HudiTrinoFileReaderFactory(storage); + } + + @Override + public HoodieFileWriterFactory getWriterFactory(HoodieRecord.HoodieRecordType recordType) + { + throw new UnsupportedOperationException("HudiTrinoIOFactory does not support writers."); + } + + @Override + public FileFormatUtils getFileFormatUtils(HoodieFileFormat fileFormat) + { + throw new UnsupportedOperationException("FileFormatUtils not supported in HudiTrinoIOFactory"); + } + + @Override + public HoodieStorage getStorage(StoragePath storagePath) + { + return storage; + } + + @Override + public HoodieStorage getStorage(StoragePath path, boolean enableRetry, long maxRetryIntervalMs, int maxRetryNumbers, long initialRetryIntervalMs, String retryExceptions, ConsistencyGuard consistencyGuard) + { + return storage; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/InlineSeekableDataInputStream.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/InlineSeekableDataInputStream.java new file mode 100644 index 0000000000000..5b1c322e0a39b --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/InlineSeekableDataInputStream.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.hudi.io; + +import io.trino.filesystem.TrinoInputStream; + +import java.io.IOException; + +public class InlineSeekableDataInputStream + extends TrinoSeekableDataInputStream +{ + private final long startOffset; + private final long length; + + public InlineSeekableDataInputStream(TrinoInputStream stream, long startOffset, long length) + { + super(stream); + this.startOffset = startOffset; + this.length = length; + } + + @Override + public long getPos() + throws IOException + { + return super.getPos() - startOffset; + } + + @Override + public void seek(long pos) + throws IOException + { + if (pos > length) { + throw new IOException(String.format( + "Attempting to seek past inline content: position to seek to is %s but the length is %s", + pos, length)); + } + super.seek(startOffset + pos); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/TrinoSeekableDataInputStream.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/TrinoSeekableDataInputStream.java new file mode 100644 index 0000000000000..babf44eba755e --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/io/TrinoSeekableDataInputStream.java @@ -0,0 +1,45 @@ +/* + * 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.hudi.io; + +import io.trino.filesystem.TrinoInputStream; +import org.apache.hudi.io.SeekableDataInputStream; + +import java.io.IOException; + +public class TrinoSeekableDataInputStream + extends SeekableDataInputStream +{ + private final TrinoInputStream stream; + + public TrinoSeekableDataInputStream(TrinoInputStream stream) + { + super(stream); + this.stream = stream; + } + + @Override + public long getPos() + throws IOException + { + return stream.getPosition(); + } + + @Override + public void seek(long pos) + throws IOException + { + stream.seek(pos); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java new file mode 100644 index 0000000000000..f7e6fb1fa81b1 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java @@ -0,0 +1,113 @@ +/* + * 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.hudi.partition; + +import io.trino.filesystem.Location; +import io.trino.metastore.Partition; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.TupleDomain; + +import java.util.List; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static io.trino.plugin.hudi.HudiUtil.buildPartitionKeys; +import static io.trino.plugin.hudi.HudiUtil.partitionMatchesPredicates; + +public class HiveHudiPartitionInfo + implements HudiPartitionInfo +{ + public static final String NON_PARTITION = ""; + + private final SchemaTableName schemaTableName; + private final List partitionColumnHandles; + private final TupleDomain constraintSummary; + private final String hivePartitionName; + private final String relativePartitionPath; + private final List hivePartitionKeys; + + public HiveHudiPartitionInfo( + SchemaTableName schemaTableName, + Location tableLocation, + String hivePartitionName, + Partition partition, + List partitionColumnHandles, + TupleDomain constraintSummary) + { + this.schemaTableName = schemaTableName; + this.partitionColumnHandles = partitionColumnHandles; + this.constraintSummary = constraintSummary; + this.hivePartitionName = hivePartitionName; + this.relativePartitionPath = getRelativePartitionPath( + tableLocation, + Location.of(partition.getStorage().getLocation())); + this.hivePartitionKeys = buildPartitionKeys( + partitionColumnHandles, partition.getValues()); + } + + @Override + public String getRelativePartitionPath() + { + return relativePartitionPath; + } + + @Override + public List getHivePartitionKeys() + { + return hivePartitionKeys; + } + + @Override + public boolean doesMatchPredicates() + { + if (hivePartitionName.equals(NON_PARTITION)) { + return true; + } + return partitionMatchesPredicates(schemaTableName, hivePartitionName, partitionColumnHandles, constraintSummary); + } + + public String getHivePartitionName() + { + return hivePartitionName; + } + + private static String getRelativePartitionPath(Location baseLocation, Location fullPartitionLocation) + { + String basePath = baseLocation.path(); + String fullPartitionPath = fullPartitionLocation.path(); + + if (!fullPartitionPath.startsWith(basePath)) { + throw new IllegalArgumentException("Partition location does not belong to base-location"); + } + + String baseLocationParent = baseLocation.parentDirectory().path(); + String baseLocationName = baseLocation.fileName(); + int partitionStartIndex = fullPartitionPath.indexOf( + baseLocationName, + baseLocationParent == null ? 0 : baseLocationParent.length()); + // Partition-Path could be empty for non-partitioned tables + boolean isNonPartitionedTable = partitionStartIndex + baseLocationName.length() == fullPartitionPath.length(); + return isNonPartitionedTable ? NON_PARTITION : fullPartitionPath.substring(partitionStartIndex + baseLocationName.length() + 1); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("hivePartitionName", hivePartitionName) + .add("hivePartitionKeys", hivePartitionKeys) + .toString(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfo.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfo.java new file mode 100644 index 0000000000000..1b607f1accf2b --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfo.java @@ -0,0 +1,27 @@ +/* + * 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.hudi.partition; + +import io.trino.plugin.hive.HivePartitionKey; + +import java.util.List; + +public interface HudiPartitionInfo +{ + String getRelativePartitionPath(); + + List getHivePartitionKeys(); + + boolean doesMatchPredicates(); +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfoLoader.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfoLoader.java new file mode 100644 index 0000000000000..874297b9b692a --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfoLoader.java @@ -0,0 +1,90 @@ +/* + * 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.hudi.partition; + +import io.airlift.concurrent.MoreFutures; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.util.AsyncQueue; +import io.trino.plugin.hudi.query.HudiDirectoryLister; +import io.trino.plugin.hudi.split.HudiSplitFactory; +import io.trino.spi.connector.ConnectorSplit; +import org.apache.hudi.common.model.FileSlice; + +import java.util.Deque; +import java.util.List; +import java.util.Optional; + +import static io.trino.plugin.hudi.partition.HiveHudiPartitionInfo.NON_PARTITION; + +public class HudiPartitionInfoLoader + implements Runnable +{ + private final HudiDirectoryLister hudiDirectoryLister; + private final HudiSplitFactory hudiSplitFactory; + private final AsyncQueue asyncQueue; + private final Deque partitionQueue; + private final String commitTime; + private final boolean useIndex; + + private boolean isRunning; + + public HudiPartitionInfoLoader( + HudiDirectoryLister hudiDirectoryLister, + String commitTime, + HudiSplitFactory hudiSplitFactory, + AsyncQueue asyncQueue, + Deque partitionQueue, + boolean useIndex) + { + this.hudiDirectoryLister = hudiDirectoryLister; + this.commitTime = commitTime; + this.hudiSplitFactory = hudiSplitFactory; + this.asyncQueue = asyncQueue; + this.partitionQueue = partitionQueue; + this.isRunning = true; + this.useIndex = useIndex; + } + + @Override + public void run() + { + while (isRunning || !partitionQueue.isEmpty()) { + String partitionName = partitionQueue.poll(); + + if (partitionName != null) { + generateSplitsFromPartition(partitionName); + } + } + } + + private void generateSplitsFromPartition(String partitionName) + { + Optional partitionInfo = hudiDirectoryLister.getPartitionInfo(partitionName); + partitionInfo.ifPresent(hudiPartitionInfo -> { + if (hudiPartitionInfo.doesMatchPredicates() || partitionName.equals(NON_PARTITION)) { + List partitionKeys = hudiPartitionInfo.getHivePartitionKeys(); + List partitionFileSlices = hudiDirectoryLister.listStatus(hudiPartitionInfo, useIndex); + partitionFileSlices.stream() + .flatMap(slice -> hudiSplitFactory.createSplits(partitionKeys, slice, commitTime).stream()) + .map(asyncQueue::offer) + .forEachOrdered(MoreFutures::getFutureValue); + } + }); + } + + public void stopRunning() + { + this.isRunning = false; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/HudiDirectoryLister.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/HudiDirectoryLister.java new file mode 100644 index 0000000000000..fb8f1903358eb --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/HudiDirectoryLister.java @@ -0,0 +1,29 @@ +/* + * 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.hudi.query; + +import io.trino.plugin.hudi.partition.HudiPartitionInfo; +import org.apache.hudi.common.model.FileSlice; + +import java.io.Closeable; +import java.util.List; +import java.util.Optional; + +public interface HudiDirectoryLister + extends Closeable +{ + List listStatus(HudiPartitionInfo partitionInfo, boolean useIndex); + + Optional getPartitionInfo(String partition); +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/HudiSnapshotDirectoryLister.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/HudiSnapshotDirectoryLister.java new file mode 100644 index 0000000000000..dfd7ef7d3c392 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/HudiSnapshotDirectoryLister.java @@ -0,0 +1,123 @@ +/* + * 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.hudi.query; + +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.trino.filesystem.Location; +import io.trino.metastore.Partition; +import io.trino.plugin.hudi.HudiTableHandle; +import io.trino.plugin.hudi.partition.HiveHudiPartitionInfo; +import io.trino.plugin.hudi.partition.HudiPartitionInfo; +import io.trino.plugin.hudi.query.index.HudiIndexSupport; +import io.trino.plugin.hudi.query.index.IndexSupportFactory; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.hudi.HudiUtil.getFileSystemView; + +public class HudiSnapshotDirectoryLister + implements HudiDirectoryLister +{ + private static final Logger log = Logger.get(HudiSnapshotDirectoryLister.class); + private final HudiTableHandle tableHandle; + private final Lazy lazyFileSystemView; + private final Lazy> lazyAllPartitionInfoMap; + private final Optional indexSupportOpt; + + public HudiSnapshotDirectoryLister( + ConnectorSession session, + HudiTableHandle tableHandle, + boolean enableMetadataTable, + Lazy lazyTableMetadata, + Lazy> lazyAllPartitions) + { + this.tableHandle = tableHandle; + SchemaTableName schemaTableName = tableHandle.getSchemaTableName(); + this.lazyFileSystemView = Lazy.lazily(() -> { + HoodieTimer timer = HoodieTimer.start(); + HoodieTableMetaClient metaClient = tableHandle.getMetaClient(); + HoodieTableFileSystemView fileSystemView = getFileSystemView(lazyTableMetadata.get(), metaClient); + if (enableMetadataTable) { + fileSystemView.loadAllPartitions(); + } + log.info("Created file system view of table %s in %s ms", schemaTableName, timer.endTimer()); + return fileSystemView; + }); + + this.lazyAllPartitionInfoMap = Lazy.lazily(() -> lazyAllPartitions.get().entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + e -> new HiveHudiPartitionInfo( + schemaTableName, + Location.of(tableHandle.getBasePath()), + e.getKey(), + e.getValue(), + tableHandle.getPartitionColumns(), + tableHandle.getPartitionPredicates())))); + Lazy lazyMetaClient = Lazy.lazily(tableHandle::getMetaClient); + this.indexSupportOpt = enableMetadataTable ? + IndexSupportFactory.createIndexSupport(schemaTableName, lazyMetaClient, lazyTableMetadata, tableHandle.getRegularPredicates(), session) : Optional.empty(); + } + + @Override + public List listStatus(HudiPartitionInfo partitionInfo, boolean useIndex) + { + HoodieTimer timer = HoodieTimer.start(); + Stream slices = lazyFileSystemView.get().getLatestFileSlicesBeforeOrOn( + partitionInfo.getRelativePartitionPath(), + tableHandle.getLatestCommitTime(), + false); + + if (!useIndex) { + return slices.collect(toImmutableList()); + } + + ImmutableList collect = slices + .filter(slice -> indexSupportOpt + .map(indexSupport -> !indexSupport.shouldSkipFileSlice(slice)) + .orElse(true)) + .collect(toImmutableList()); + log.info("Listed partition [%s] on table %s.%s in %s ms", + partitionInfo, tableHandle.getSchemaName(), tableHandle.getTableName(), timer.endTimer()); + return collect; + } + + @Override + public Optional getPartitionInfo(String partition) + { + return Optional.ofNullable(lazyAllPartitionInfoMap.get().get(partition)); + } + + @Override + public void close() + { + if (!lazyFileSystemView.get().isClosed()) { + lazyFileSystemView.get().close(); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiBaseIndexSupport.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiBaseIndexSupport.java new file mode 100644 index 0000000000000..044908f9c49f4 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiBaseIndexSupport.java @@ -0,0 +1,66 @@ +/* + * 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.hudi.query.index; + +import io.airlift.log.Logger; +import io.trino.spi.connector.SchemaTableName; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public abstract class HudiBaseIndexSupport + implements HudiIndexSupport +{ + private final Logger log; + protected final SchemaTableName schemaTableName; + protected final Lazy lazyMetaClient; + + public HudiBaseIndexSupport(Logger log, SchemaTableName schemaTableName, Lazy lazyMetaClient) + { + this.log = requireNonNull(log, "log is null"); + this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); + this.lazyMetaClient = requireNonNull(lazyMetaClient, "metaClient is null"); + } + + public void printDebugMessage(Map> candidateFileSlices, Map> inputFileSlices, long lookupDurationMs) + { + if (log.isDebugEnabled()) { + int candidateFileSize = candidateFileSlices.values().stream().mapToInt(List::size).sum(); + int totalFiles = inputFileSlices.values().stream().mapToInt(List::size).sum(); + double skippingPercent = totalFiles == 0 ? 0.0d : (totalFiles - candidateFileSize) / (totalFiles * 1.0d); + + log.info("Total files: %s; files after data skipping: %s; skipping percent %s; time taken: %s ms; table name: %s", + totalFiles, + candidateFileSize, + skippingPercent, + lookupDurationMs, + schemaTableName); + } + } + + protected Map getAllIndexDefinitions() + { + if (lazyMetaClient.get().getIndexMetadata().isEmpty()) { + return Map.of(); + } + + return lazyMetaClient.get().getIndexMetadata().get().getIndexDefinitions(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiColumnStatsIndexSupport.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiColumnStatsIndexSupport.java new file mode 100644 index 0000000000000..72e46693ee928 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiColumnStatsIndexSupport.java @@ -0,0 +1,319 @@ +/* + * 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.hudi.query.index; + +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.airlift.units.Duration; +import io.trino.parquet.predicate.TupleDomainParquetPredicate; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.util.TupleDomainUtils; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.ValueSet; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.hash.ColumnIndexID; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static io.trino.parquet.predicate.PredicateUtils.isStatisticsOverflow; +import static io.trino.plugin.hudi.HudiSessionProperties.getColumnStatsWaitTimeout; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DateType.DATE; +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.TinyintType.TINYINT; +import static java.lang.Float.floatToRawIntBits; + +public class HudiColumnStatsIndexSupport + extends HudiBaseIndexSupport +{ + private static final Logger log = Logger.get(HudiColumnStatsIndexSupport.class); + private final CompletableFuture>>> statsByFileNameFuture; + protected final TupleDomain regularColumnPredicates; + private final List regularColumns; + private final Duration columnStatsWaitTimeout; + private final long futureStartTimeMs; + + public HudiColumnStatsIndexSupport(ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + this(log, session, schemaTableName, lazyMetaClient, lazyTableMetadata, regularColumnPredicates); + } + + public HudiColumnStatsIndexSupport(Logger log, ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + super(log, schemaTableName, lazyMetaClient); + this.columnStatsWaitTimeout = getColumnStatsWaitTimeout(session); + this.regularColumnPredicates = regularColumnPredicates.transformKeys(HiveColumnHandle::getName); + this.regularColumns = this.regularColumnPredicates + .getDomains().get().entrySet().stream().map(Map.Entry::getKey).collect(Collectors.toList()); + if (regularColumnPredicates.isAll() || !regularColumnPredicates.getDomains().isPresent()) { + this.statsByFileNameFuture = CompletableFuture.completedFuture(Optional.empty()); + } + else { + // Get filter columns + List encodedTargetColumnNames = regularColumns + .stream() + .map(col -> new ColumnIndexID(col).asBase64EncodedString()).collect(Collectors.toList()); + statsByFileNameFuture = CompletableFuture.supplyAsync(() -> { + HoodieTimer timer = HoodieTimer.start(); + if (!lazyMetaClient.get().getTableConfig().getMetadataPartitions() + .contains(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS)) { + return Optional.empty(); + } + + Map> statsByFileName = + lazyTableMetadata.get().getRecordsByKeyPrefixes(encodedTargetColumnNames, + HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS, true) + .collectAsList() + .stream() + .filter(f -> f.getData().getColumnStatMetadata().isPresent()) + .map(f -> f.getData().getColumnStatMetadata().get()) + .collect(Collectors.groupingBy( + HoodieMetadataColumnStats::getFileName, + Collectors.toMap( + HoodieMetadataColumnStats::getColumnName, + Function.identity()))); + log.debug("Column stats lookup took %s ms and identified %d relevant file IDs.", timer.endTimer(), statsByFileName.size()); + + return Optional.of(statsByFileName); + }); + } + this.futureStartTimeMs = System.currentTimeMillis(); + } + + @Override + public boolean shouldSkipFileSlice(FileSlice slice) + { + try { + if (statsByFileNameFuture.isDone()) { + Optional>> statsOpt = statsByFileNameFuture.get(); + return statsOpt + .map(stats -> shouldSkipFileSlice(slice, stats, regularColumnPredicates, regularColumns)) + .orElse(false); + } + + long elapsedMs = System.currentTimeMillis() - futureStartTimeMs; + if (elapsedMs > columnStatsWaitTimeout.toMillis()) { + // Took too long; skip decision + return false; + } + + // If still within the timeout window, wait up to the remaining time + long remainingMs = Math.max(0, columnStatsWaitTimeout.toMillis() - elapsedMs); + Optional>> statsOpt = + statsByFileNameFuture.get(remainingMs, TimeUnit.MILLISECONDS); + + return statsOpt + .map(stats -> shouldSkipFileSlice(slice, stats, regularColumnPredicates, regularColumns)) + .orElse(false); + } + catch (TimeoutException | InterruptedException | ExecutionException e) { + return false; + } + } + + @Override + public boolean canApply(TupleDomain tupleDomain) + { + boolean isIndexSupported = isIndexSupportAvailable(); + // indexDefinition is only available after table version EIGHT + // For tables that have versions < EIGHT, column stats index is available as long as partition in metadata is available + if (!isIndexSupported || lazyMetaClient.get().getTableConfig().getTableVersion().lesserThan(HoodieTableVersion.EIGHT)) { + log.debug("Column Stats Index partition is not enabled in metadata."); + return isIndexSupported; + } + + Map indexDefinitions = getAllIndexDefinitions(); + HoodieIndexDefinition colStatsDefinition = indexDefinitions.get(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS); + if (colStatsDefinition == null || colStatsDefinition.getSourceFields() == null || colStatsDefinition.getSourceFields().isEmpty()) { + log.warn("Column stats index definition is missing or has no source fields defined"); + return false; + } + + // Optimization applied: Only consider applicable if predicates reference indexed columns + List sourceFields = colStatsDefinition.getSourceFields(); + boolean applicable = TupleDomainUtils.areSomeFieldsReferenced(tupleDomain, sourceFields); + + if (applicable) { + log.debug("Column Stats Index is available and applicable (predicates reference indexed columns)."); + } + else { + log.debug("Column Stats Index is available, but predicates do not reference any indexed columns."); + } + return applicable; + } + + public boolean isIndexSupportAvailable() + { + return lazyMetaClient.get().getTableConfig().getMetadataPartitions() + .contains(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS); + } + + // TODO: Move helper functions below to TupleDomain/DomainUtils + private static boolean shouldSkipFileSlice( + FileSlice fileSlice, + Map> statsByFileName, + TupleDomain regularColumnPredicates, + List regularColumns) + { + String fileSliceName = fileSlice.getBaseFile().map(BaseFile::getFileName).orElse(""); + // If no stats exist for this specific file, we cannot prune it. + if (!statsByFileName.containsKey(fileSliceName)) { + return false; + } + Map stats = statsByFileName.get(fileSliceName); + return !evaluateStatisticPredicate(regularColumnPredicates, stats, regularColumns); + } + + protected static boolean evaluateStatisticPredicate( + TupleDomain regularColumnPredicates, + Map stats, + List regularColumns) + { + if (regularColumnPredicates.isNone() || !regularColumnPredicates.getDomains().isPresent()) { + return true; + } + for (String regularColumn : regularColumns) { + Domain columnPredicate = regularColumnPredicates.getDomains().get().get(regularColumn); + Optional currentColumnStats = Optional.ofNullable(stats.get(regularColumn)); + if (currentColumnStats.isEmpty()) { + // No stats for column + } + else { + Domain domain = getDomain(regularColumn, columnPredicate.getType(), currentColumnStats.get()); + if (columnPredicate.intersect(domain).isNone()) { + return false; + } + } + } + return true; + } + + private static Domain getDomain(String colName, Type type, HoodieMetadataColumnStats statistics) + { + if (statistics == null) { + return Domain.all(type); + } + boolean hasNullValue = statistics.getNullCount() != 0L; + boolean hasNonNullValue = statistics.getValueCount() - statistics.getNullCount() > 0; + if (!hasNonNullValue || statistics.getMaxValue() == null || statistics.getMinValue() == null) { + return Domain.create(ValueSet.all(type), hasNullValue); + } + if (!(statistics.getMinValue() instanceof GenericRecord) || + !(statistics.getMaxValue() instanceof GenericRecord)) { + return Domain.all(type); + } + return getDomain(colName, type, ((GenericRecord) statistics.getMinValue()).get(0), + ((GenericRecord) statistics.getMaxValue()).get(0), hasNullValue); + } + + /** + * Get a domain for the ranges defined by each pair of elements from {@code minimums} and {@code maximums}. + * Both arrays must have the same length. + */ + private static Domain getDomain(String colName, Type type, Object minimum, Object maximum, boolean hasNullValue) + { + try { + if (type.equals(BOOLEAN)) { + boolean hasTrueValue = (boolean) minimum || (boolean) maximum; + boolean hasFalseValue = !(boolean) minimum || !(boolean) maximum; + if (hasTrueValue && hasFalseValue) { + return Domain.all(type); + } + if (hasTrueValue) { + return Domain.create(ValueSet.of(type, true), hasNullValue); + } + if (hasFalseValue) { + return Domain.create(ValueSet.of(type, false), hasNullValue); + } + // No other case, since all null case is handled earlier. + } + + if ((type.equals(BIGINT) || type.equals(TINYINT) || type.equals(SMALLINT) + || type.equals(INTEGER) || type.equals(DATE))) { + long minValue = TupleDomainParquetPredicate.asLong(minimum); + long maxValue = TupleDomainParquetPredicate.asLong(maximum); + if (isStatisticsOverflow(type, minValue, maxValue)) { + return Domain.create(ValueSet.all(type), hasNullValue); + } + return ofMinMax(type, minValue, maxValue, hasNullValue); + } + + if (type.equals(REAL)) { + Float minValue = (Float) minimum; + Float maxValue = (Float) maximum; + if (minValue.isNaN() || maxValue.isNaN()) { + return Domain.create(ValueSet.all(type), hasNullValue); + } + return ofMinMax(type, (long) floatToRawIntBits(minValue), (long) floatToRawIntBits(maxValue), hasNullValue); + } + + if (type.equals(DOUBLE)) { + Double minValue = (Double) minimum; + Double maxValue = (Double) maximum; + if (minValue.isNaN() || maxValue.isNaN()) { + return Domain.create(ValueSet.all(type), hasNullValue); + } + return ofMinMax(type, minValue, maxValue, hasNullValue); + } + + if (type.equals(VarcharType.VARCHAR)) { + Slice min = Slices.utf8Slice((String) minimum); + Slice max = Slices.utf8Slice((String) maximum); + return ofMinMax(type, min, max, hasNullValue); + } + return Domain.create(ValueSet.all(type), hasNullValue); + } + catch (Exception e) { + log.warn("failed to create Domain for column: %s which type is: %s", colName, type.toString()); + return Domain.create(ValueSet.all(type), hasNullValue); + } + } + + private static Domain ofMinMax(Type type, Object min, Object max, boolean hasNullValue) + { + Range range = Range.range(type, min, true, max, true); + ValueSet vs = ValueSet.ofRanges(ImmutableList.of(range)); + return Domain.create(vs, hasNullValue); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiIndexSupport.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiIndexSupport.java new file mode 100644 index 0000000000000..de33640f48904 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiIndexSupport.java @@ -0,0 +1,27 @@ +/* + * 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.hudi.query.index; + +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.model.FileSlice; + +public interface HudiIndexSupport +{ + boolean canApply(TupleDomain tupleDomain); + + default boolean shouldSkipFileSlice(FileSlice slice) + { + return false; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiNoOpIndexSupport.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiNoOpIndexSupport.java new file mode 100644 index 0000000000000..e79eb342812b8 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiNoOpIndexSupport.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.hudi.query.index; + +import io.airlift.log.Logger; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.util.Lazy; + +/** + * Noop index support to ensure that MDT enabled split generation is entered. + */ +public class HudiNoOpIndexSupport + extends HudiBaseIndexSupport +{ + private static final Logger log = Logger.get(HudiNoOpIndexSupport.class); + + public HudiNoOpIndexSupport(SchemaTableName schemaTableName, Lazy lazyMetaClient) + { + super(log, schemaTableName, lazyMetaClient); + } + + @Override + public boolean canApply(TupleDomain tupleDomain) + { + return true; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiPartitionStatsIndexSupport.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiPartitionStatsIndexSupport.java new file mode 100644 index 0000000000000..c2e98820fb6fe --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiPartitionStatsIndexSupport.java @@ -0,0 +1,146 @@ +/* + * 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.hudi.query.index; + +import io.airlift.log.Logger; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.util.TupleDomainUtils; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.hash.ColumnIndexID; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.util.Lazy; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static io.trino.plugin.hudi.util.TupleDomainUtils.hasSimpleNullCheck; + +public class HudiPartitionStatsIndexSupport + extends HudiColumnStatsIndexSupport +{ + private static final Logger log = Logger.get(HudiColumnStatsIndexSupport.class); + private final Lazy lazyMetadataTable; + + public HudiPartitionStatsIndexSupport(ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + super(log, session, schemaTableName, lazyMetaClient, lazyTableMetadata, regularColumnPredicates); + this.lazyMetadataTable = lazyTableMetadata; + } + + public Optional> prunePartitions( + List allPartitions) + { + HoodieTimer timer = HoodieTimer.start(); + + // Filter out predicates containing simple null checks (`IS NULL` or `IS NOT NULL`) + TupleDomain filteredRegularPredicates = regularColumnPredicates.filter((_, domain) -> !hasSimpleNullCheck(domain)); + + // Sanity check, if no regular domains, return immediately + if (filteredRegularPredicates.getDomains().isEmpty()) { + timer.endTimer(); + return Optional.empty(); + } + + List regularColumns = new ArrayList<>(filteredRegularPredicates.getDomains().get().keySet()); + + // Get columns to filter on + List encodedTargetColumnNames = regularColumns + .stream() + .map(col -> new ColumnIndexID(col).asBase64EncodedString()).toList(); + + // Map of partition stats keyed by partition name + Map> statsByPartitionName = lazyMetadataTable.get().getRecordsByKeyPrefixes( + encodedTargetColumnNames, + HoodieTableMetadataUtil.PARTITION_NAME_PARTITION_STATS, true) + .collectAsList() + .stream() + .filter(f -> f.getData().getColumnStatMetadata().isPresent()) + .map(f -> f.getData().getColumnStatMetadata().get()) + .collect(Collectors.groupingBy( + HoodieMetadataColumnStats::getFileName, + Collectors.toMap( + HoodieMetadataColumnStats::getColumnName, + Function.identity()))); + + // For each partition, determine if it should be kept based on stats availability and predicate evaluation + List prunedPartitions = allPartitions.stream() + .filter(partition -> { + // Check if stats exist for this partition + Map partitionStats = statsByPartitionName.get(partition); + if (partitionStats == null) { + // Partition has no stats in the index, keep it + return true; + } + else { + // Partition has stats, evaluate the predicate against them + // Keep the partition only if the predicate evaluates to true + // Important: If some columns in encodedTargetColumnNames is not available in partition stats, partition will not be pruned iff all available predicate + // evaluates to true. Since we cannot determine if the predicate will evaluate to true or not on the missing stat, adopt conservative measure to true, + // i.e. to not prune + return evaluateStatisticPredicate(filteredRegularPredicates, partitionStats, regularColumns); + } + }) + .collect(Collectors.toList()); + + log.info("Took %s ms to prune partitions using Partition Stats Index for table %s", timer.endTimer(), schemaTableName); + return Optional.of(prunedPartitions); + } + + @Override + public boolean isIndexSupportAvailable() + { + return lazyMetaClient.get().getTableConfig().getMetadataPartitions() + .contains(HoodieTableMetadataUtil.PARTITION_NAME_PARTITION_STATS); + } + + @Override + public boolean canApply(TupleDomain tupleDomain) + { + // Important: has the same implementation as col stats superclass, only difference is that log messages are different + if (!isIndexSupportAvailable()) { + log.debug("Partition Stats Index partition is not enabled in metadata table."); + return false; + } + + Map indexDefinitions = getAllIndexDefinitions(); + HoodieIndexDefinition partitionsStatsIndex = indexDefinitions.get(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS); + if (partitionsStatsIndex == null || partitionsStatsIndex.getSourceFields() == null || partitionsStatsIndex.getSourceFields().isEmpty()) { + log.warn("Partition stats index definition is missing or has no source fields defined"); + return false; + } + + // Optimization applied: Only consider applicable if predicates reference indexed columns + List sourceFields = partitionsStatsIndex.getSourceFields(); + boolean applicable = TupleDomainUtils.areSomeFieldsReferenced(tupleDomain, sourceFields); + + if (applicable) { + log.debug("Partition Stats Index is available and applicable (predicates reference indexed columns)."); + } + else { + log.debug("Partition Stats Index is available, but predicates do not reference any indexed columns."); + } + return applicable; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiRecordLevelIndexSupport.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiRecordLevelIndexSupport.java new file mode 100644 index 0000000000000..3c5e33f4e6a01 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiRecordLevelIndexSupport.java @@ -0,0 +1,327 @@ +/* + * 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.hudi.query.index; + +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.airlift.units.Duration; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.util.TupleDomainUtils; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieRecordGlobalLocation; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.util.Lazy; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; +import static io.trino.plugin.hudi.HudiSessionProperties.getRecordIndexWaitTimeout; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class HudiRecordLevelIndexSupport + extends HudiBaseIndexSupport +{ + private static final Logger log = Logger.get(HudiRecordLevelIndexSupport.class); + + public static final String DEFAULT_COLUMN_VALUE_SEPARATOR = ":"; + public static final String DEFAULT_RECORD_KEY_PARTS_SEPARATOR = ","; + private final CompletableFuture>> relevantFileIdsFuture; + private final Duration recordIndexWaitTimeout; + private final long futureStartTimeMs; + + public HudiRecordLevelIndexSupport(ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + super(log, schemaTableName, lazyMetaClient); + this.recordIndexWaitTimeout = getRecordIndexWaitTimeout(session); + if (regularColumnPredicates.isAll()) { + log.debug("Predicates cover all data, skipping record level index lookup."); + this.relevantFileIdsFuture = CompletableFuture.completedFuture(Optional.empty()); + } + else { + this.relevantFileIdsFuture = CompletableFuture.supplyAsync(() -> { + HoodieTimer timer = HoodieTimer.start(); + Option recordKeyFieldsOpt = lazyMetaClient.get().getTableConfig().getRecordKeyFields(); + if (recordKeyFieldsOpt.isEmpty() || recordKeyFieldsOpt.get().length == 0) { + // Should not happen since canApply checks for this, include for safety + throw new TrinoException(HUDI_BAD_DATA, "Record key fields must be defined to use Record Level Index."); + } + List recordKeyFields = Arrays.asList(recordKeyFieldsOpt.get()); + + TupleDomain regularPredicatesTransformed = regularColumnPredicates.transformKeys(HiveColumnHandle::getName); + // Only extract the predicates relevant to the record key fields + TupleDomain filteredDomains = extractPredicatesForColumns(regularPredicatesTransformed, recordKeyFields); + + // Construct the actual record keys based on the filtered predicates using Hudi's encoding scheme + List recordKeys = constructRecordKeys(filteredDomains, recordKeyFields); + + if (recordKeys.isEmpty()) { + // If key construction fails (e.g., incompatible predicates not caught by canApply, or placeholder issue) + log.warn("Took %s ms, but could not construct record keys from predicates. Skipping record index pruning for table %s", + timer.endTimer(), schemaTableName); + return Optional.empty(); + } + log.debug(String.format("Constructed %d record keys for index lookup.", recordKeys.size())); + + // Perform index lookup in metadataTable + // TODO: document here what this map is keyed by + Map recordIndex = lazyTableMetadata.get().readRecordIndex(recordKeys); + if (recordIndex.isEmpty()) { + log.debug("Record level index lookup took %s ms but returned no locations for the given keys %s for table %s", + timer.endTimer(), recordKeys, schemaTableName); + // Return all original fileSlices + return Optional.empty(); + } + + // Collect fileIds for pruning + Set relevantFiles = recordIndex.values().stream() + .map(HoodieRecordGlobalLocation::getFileId) + .collect(Collectors.toSet()); + log.debug("Record level index lookup took %s ms and identified %d relevant file IDs.", timer.endTimer(), relevantFiles.size()); + + return Optional.of(relevantFiles); + }); + } + this.futureStartTimeMs = System.currentTimeMillis(); + } + + @Override + public boolean shouldSkipFileSlice(FileSlice slice) + { + try { + if (relevantFileIdsFuture.isDone()) { + Optional> relevantFileIds = relevantFileIdsFuture.get(); + return relevantFileIds.map(fileIds -> !fileIds.contains(slice.getFileId())).orElse(false); + } + + long elapsedMs = System.currentTimeMillis() - futureStartTimeMs; + if (elapsedMs > recordIndexWaitTimeout.toMillis()) { + // Took too long; skip decision + return false; + } + + long remainingMs = Math.max(0, recordIndexWaitTimeout.toMillis() - elapsedMs); + Optional> relevantFileIds = relevantFileIdsFuture.get(remainingMs, MILLISECONDS); + return relevantFileIds.map(fileIds -> !fileIds.contains(slice.getFileId())).orElse(false); + } + catch (TimeoutException | InterruptedException | ExecutionException e) { + return false; + } + } + + /** + * Checks if the Record Level Index is available and the query predicates + * reference all record key fields with compatible (IN/EQUAL) constraints. + */ + @Override + public boolean canApply(TupleDomain tupleDomain) + { + if (!isIndexSupportAvailable()) { + log.debug("Record Level Index partition is not enabled in metadata."); + return false; + } + + Option recordKeyFieldsOpt = lazyMetaClient.get().getTableConfig().getRecordKeyFields(); + if (recordKeyFieldsOpt.isEmpty() || recordKeyFieldsOpt.get().length == 0) { + log.debug("Record key fields are not defined in table config."); + return false; + } + List recordKeyFields = Arrays.asList(recordKeyFieldsOpt.get()); + + // Ensure that predicates reference all record key fields and use IN/EQUAL + boolean applicable = TupleDomainUtils.areAllFieldsReferenced(tupleDomain, recordKeyFields) + && TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, recordKeyFields); + + if (!applicable) { + log.debug("Predicates do not reference all record key fields or use non-compatible (non IN/EQUAL) constraints."); + } + else { + log.debug("Record Level Index is available and applicable based on predicates."); + } + return applicable; + } + + private boolean isIndexSupportAvailable() + { + return lazyMetaClient.get().getTableConfig().getMetadataPartitions() + .contains(HoodieTableMetadataUtil.PARTITION_NAME_RECORD_INDEX); + } + + /** + * Extracts predicates from a TupleDomain that match a given set of columns. + * Preserves all complex predicate properties including multi-value domains, + * range-based predicates, and nullability. + * + * @param tupleDomain The source TupleDomain containing all predicates + * @param columnFields The set of columns for which to extract predicates + * @return A new TupleDomain containing only the predicates for the specified columns + */ + public static TupleDomain extractPredicatesForColumns(TupleDomain tupleDomain, List columnFields) + { + if (tupleDomain.isNone()) { + return TupleDomain.none(); + } + + if (tupleDomain.isAll()) { + return TupleDomain.all(); + } + + // Extract the domains matching the specified columns + Map allDomains = tupleDomain.getDomains().get(); + Map filteredDomains = allDomains.entrySet().stream().filter(entry -> columnFields.contains(entry.getKey())) // Ensure key is in the column set + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + // If no domains matched, but we had some columns to extract, return ALL + if (filteredDomains.isEmpty() && !columnFields.isEmpty() && !allDomains.isEmpty()) { + return TupleDomain.all(); + } + + return TupleDomain.withColumnDomains(filteredDomains); + } + + /** + * Constructs a record key from TupleDomain based on whether it's a complex key or not. + *

+ * Construction of record keys will only be handled for domains generated from EQUALITY or IN predicates. + *

+ * An empty list of record keys will be generated if the following conditions are not met: + *

    + *
  1. recordKeysFields is empty
  2. + *
  3. recordKeyDomains isAll
  4. + *
  5. For the case of complex key, domains are not applied to all recordKeysFields
  6. + *
  7. For the case of complex key, domains are applied to all recordKeyFields, but one of the domain is NOT + * generated from an equality or IN predicate
  8. + *
+ *

+ * Note: This function is O(m^n) where m is the average size of value literals and n is the number of record keys. + *

+ * Optimization 1: If MDT enabled functions allows for streams to be passed in, we can implement an iterator to be more memory efficient. + *

+ * Optimization 2: We should also consider limiting the number of recordKeys generated, if it is estimated to be more than a limit, RLI should just be skipped + * as it may just be faster to read out all data and filer accordingly. + * + * @param recordKeyDomains The filtered TupleDomain containing column handles and values + * @param recordKeyFields List of column names that represent the record keys + * @return List of string values representing the record key(s) + */ + public static List constructRecordKeys(TupleDomain recordKeyDomains, List recordKeyFields) + { + // TODO: Move this to TupleDomainUtils + // If no recordKeys or no recordKeyDomains, return empty list + if (recordKeyFields == null || recordKeyFields.isEmpty() || recordKeyDomains.isAll()) { + return Collections.emptyList(); + } + + // All recordKeys must have a domain else, return empty list (applicable to complexKeys) + // If a one of the recordKey in the set of complexKeys does not have a domain, we are unable to construct + // a complete complexKey + if (!recordKeyDomains.getDomains().get().keySet().containsAll(recordKeyFields)) { + return Collections.emptyList(); + } + + // Extract the domain mappings from the tuple domain + Map domains = recordKeyDomains.getDomains().get(); + + // Case 1: Not a complex key (single record key) + if (recordKeyFields.size() == 1) { + String recordKey = recordKeyFields.getFirst(); + + // Extract value for this key + Domain domain = domains.get(recordKey); + return extractStringValues(domain); + } + // Case 2: Complex/Composite key (multiple record keys) + else { + // Create a queue to manage the Cartesian product generation + Queue results = new LinkedList<>(); + + // For each key in the complex key + for (String recordKeyField : recordKeyFields) { + // Extract value for this key + Domain domain = domains.get(recordKeyField); + List values = extractStringValues(domain); + // First iteration: initialize the queue + if (results.isEmpty()) { + values.forEach(v -> results.offer(recordKeyField + DEFAULT_COLUMN_VALUE_SEPARATOR + v)); + } + else { + int size = results.size(); + for (int j = 0; j < size; j++) { + String currentEntry = results.poll(); + + // Generate new combinations by appending keyParts to existing keyParts + for (String v : values) { + String newKeyPart = recordKeyField + DEFAULT_COLUMN_VALUE_SEPARATOR + v; + String newEntry = currentEntry + DEFAULT_RECORD_KEY_PARTS_SEPARATOR + newKeyPart; + results.offer(newEntry); + } + } + } + } + return results.stream().toList(); + } + } + + /** + * Extract string values from a domain, handle EQUAL and IN domains only. + * Note: Actual implementation depends on your Domain class structure. + */ + private static List extractStringValues(Domain domain) + { + List values = new ArrayList<>(); + + if (domain.isSingleValue()) { + // Handle EQUAL condition (single value domain) + Object value = domain.getSingleValue(); + values.add(convertToString(value)); + } + else if (domain.getValues().isDiscreteSet()) { + // Handle IN condition (set of discrete values) + for (Object value : domain.getValues().getDiscreteSet()) { + values.add(convertToString(value)); + } + } + return values; + } + + private static String convertToString(Object value) + { + if (value instanceof Slice) { + return ((Slice) value).toStringUtf8(); + } + else { + return value.toString(); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiSecondaryIndexSupport.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiSecondaryIndexSupport.java new file mode 100644 index 0000000000000..d078c9e6e9f6e --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/HudiSecondaryIndexSupport.java @@ -0,0 +1,232 @@ +/* + * 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.hudi.query.index; + +import io.airlift.log.Logger; +import io.airlift.units.Duration; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.util.TupleDomainUtils; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecordGlobalLocation; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +import static io.trino.plugin.hudi.HudiSessionProperties.getSecondaryIndexWaitTimeout; +import static io.trino.plugin.hudi.query.index.HudiRecordLevelIndexSupport.constructRecordKeys; +import static io.trino.plugin.hudi.query.index.HudiRecordLevelIndexSupport.extractPredicatesForColumns; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class HudiSecondaryIndexSupport + extends HudiBaseIndexSupport +{ + private static final Logger log = Logger.get(HudiSecondaryIndexSupport.class); + private final CompletableFuture>> relevantFileIdsFuture; + private final Duration secondaryIndexWaitTimeout; + private final long futureStartTimeMs; + + public HudiSecondaryIndexSupport(ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + super(log, schemaTableName, lazyMetaClient); + this.secondaryIndexWaitTimeout = getSecondaryIndexWaitTimeout(session); + TupleDomain regularPredicatesTransformed = regularColumnPredicates.transformKeys(HiveColumnHandle::getName); + this.relevantFileIdsFuture = CompletableFuture.supplyAsync(() -> { + HoodieTimer timer = HoodieTimer.start(); + if (regularColumnPredicates.isAll() || lazyMetaClient.get().getIndexMetadata().isEmpty()) { + log.debug("Predicates cover all data, skipping secondary index lookup."); + return Optional.empty(); + } + + Optional> firstApplicableIndex = findFirstApplicableSecondaryIndex(regularPredicatesTransformed); + if (firstApplicableIndex.isEmpty()) { + log.debug("Took %s ms but no secondary index definition found matching the query's referenced columns for table %s", + timer.endTimer(), schemaTableName); + return Optional.empty(); + } + + Map.Entry applicableIndexEntry = firstApplicableIndex.get(); + String indexName = applicableIndexEntry.getKey(); + // `indexedColumns` should only contain one element as secondary indices only support one column + List indexedColumns = applicableIndexEntry.getValue().getSourceFields(); + log.debug(String.format("Using secondary index '%s' on columns %s for pruning.", indexName, indexedColumns)); + TupleDomain indexPredicates = extractPredicatesForColumns(regularPredicatesTransformed, indexedColumns); + + List secondaryKeys = constructRecordKeys(indexPredicates, indexedColumns); + if (secondaryKeys.isEmpty()) { + log.warn("Took %s ms, but could not construct secondary keys for index '%s' from predicates. Skipping pruning for table %s", + timer.endTimer(), indexName, schemaTableName); + return Optional.empty(); + } + log.debug(String.format("Constructed %d secondary keys for index lookup.", secondaryKeys.size())); + + // Perform index lookup in metadataTable + // TODO: document here what this map is keyed by + Map recordKeyLocationsMap = lazyTableMetadata.get().readSecondaryIndex(secondaryKeys, indexName); + if (recordKeyLocationsMap.isEmpty()) { + log.debug("Took %s ms, but secondary index lookup returned no locations for the given keys for table %s", timer.endTimer(), schemaTableName); + // Return all original fileSlices + return Optional.empty(); + } + + // Collect fileIds for pruning + Set relevantFileIds = recordKeyLocationsMap.values().stream() + .map(HoodieRecordGlobalLocation::getFileId) + .collect(Collectors.toSet()); + log.debug(String.format("Secondary index lookup identified %d relevant file IDs.", relevantFileIds.size())); + + return Optional.of(relevantFileIds); + }); + + this.futureStartTimeMs = System.currentTimeMillis(); + } + + @Override + public boolean shouldSkipFileSlice(FileSlice slice) + { + try { + if (relevantFileIdsFuture.isDone()) { + Optional> relevantFileIds = relevantFileIdsFuture.get(); + return relevantFileIds.map(fileIds -> !fileIds.contains(slice.getFileId())).orElse(false); + } + + long elapsedMs = System.currentTimeMillis() - futureStartTimeMs; + if (elapsedMs > secondaryIndexWaitTimeout.toMillis()) { + // Took too long; skip decision + return false; + } + + long remainingMs = Math.max(0, secondaryIndexWaitTimeout.toMillis() - elapsedMs); + Optional> relevantFileIds = relevantFileIdsFuture.get(remainingMs, MILLISECONDS); + return relevantFileIds.map(fileIds -> !fileIds.contains(slice.getFileId())).orElse(false); + } + catch (TimeoutException | InterruptedException | ExecutionException e) { + return false; + } + } + + /** + * Determines whether secondary index (SI) should be used based on the given tuple domain and index definitions. + *

+ * This method first filters out the secondary index definitions from the provided map of index definitions. + * It then checks if there are any secondary indices defined. If no secondary indices are found, it returns {@code false}. + *

+ * For each secondary index definition, the method verifies two conditions: + *

    + *
  1. All fields referenced in the tuple domain must be part of the source fields of the secondary index.
  2. + *
  3. The predicates on these fields must be either of type IN or EQUAL.
  4. + *
+ *

+ * If at least one secondary index definition meets these conditions, the method returns {@code true}; otherwise, + * it returns {@code false}. + * + * @param tupleDomain the domain representing the constraints on the columns + * HoodieIndexDefinition object + * @return {@code true} if at least one secondary index can be used based on the given tuple domain; otherwise, + * {@code false} + */ + @Override + public boolean canApply(TupleDomain tupleDomain) + { + if (!isIndexSupportAvailable()) { + log.debug("Secondary Index partition is not enabled in metadata."); + return false; + } + + Map secondaryIndexDefinitions = getApplicableIndexDefinitions(tupleDomain, true); + if (secondaryIndexDefinitions.isEmpty()) { + log.debug("No applicable secondary index definitions found."); + return false; + } + + boolean atLeastOneIndexUsable = secondaryIndexDefinitions.values().stream() + .anyMatch(indexDef -> { + List sourceFields = indexDef.getSourceFields(); + // Predicates referencing columns with secondary index needs to be IN or EQUAL only + boolean usable = TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFields) + && TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields); + if (log.isDebugEnabled() && usable) { + log.debug(String.format("Secondary index '%s' on fields '%s' is usable for the query.", indexDef.getIndexName(), sourceFields)); + } + return usable; + }); + if (!atLeastOneIndexUsable) { + log.debug("Although secondary indexes exist, none match the required fields and predicate types (IN/EQUAL) for the query."); + } + return atLeastOneIndexUsable; + } + + private boolean isIndexSupportAvailable() + { + // Filter out definitions that are secondary indices + Map secondaryIndexDefinitions = getAllIndexDefinitions() + .entrySet().stream() + .filter(e -> e.getKey().contains(HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX)) + .collect(Collectors.toMap(e -> e.getValue().getIndexName(), + Map.Entry::getValue)); + return !secondaryIndexDefinitions.isEmpty(); + } + + private Map getApplicableIndexDefinitions(TupleDomain tupleDomain, boolean checkPredicateCompatibility) + { + Map allDefinitions = getAllIndexDefinitions(); + if (allDefinitions.isEmpty()) { + return Map.of(); + } + // Filter out definitions that are secondary indices + return allDefinitions.entrySet().stream() + .filter(entry -> entry.getKey().contains(HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX)) + .filter(entry -> { + if (!checkPredicateCompatibility) { + return true; + } + // Perform additional compatibility checks + List sourceFields = entry.getValue().getSourceFields(); + return TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFields) + && TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private Optional> findFirstApplicableSecondaryIndex(TupleDomain queryPredicates) + { + // Predicate checks would have already been done, skip predicate checks here + Map secondaryIndexDefinitions = getApplicableIndexDefinitions(queryPredicates, false); + if (queryPredicates.getDomains().isEmpty()) { + return Optional.empty(); + } + List queryReferencedColumns = List.copyOf(queryPredicates.getDomains().get().keySet()); + return secondaryIndexDefinitions.entrySet().stream() + .filter(entry -> { + List sourceFields = entry.getValue().getSourceFields(); + // Only filter for sourceFields that match the predicates + return !sourceFields.isEmpty() && queryReferencedColumns.contains(sourceFields.getFirst()); + }) + .findFirst(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/IndexSupportFactory.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/IndexSupportFactory.java new file mode 100644 index 0000000000000..ab78e7662eb2b --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/query/index/IndexSupportFactory.java @@ -0,0 +1,148 @@ +/* + * 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.hudi.query.index; + +import io.airlift.log.Logger; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Optional; +import java.util.function.Supplier; + +import static io.trino.plugin.hudi.HudiSessionProperties.isColumnStatsIndexEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.isNoOpIndexEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.isPartitionStatsIndexEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.isRecordLevelIndexEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.isSecondaryIndexEnabled; +import static java.util.Objects.requireNonNull; + +/** + * Factory to create the appropriate HudiIndexSupport strategy based on: + * 1. Query predicates + * 2. Available table indexes + * 3. Configuration flags + */ +public class IndexSupportFactory +{ + private static final Logger log = Logger.get(IndexSupportFactory.class); + + private IndexSupportFactory() {} + + /** + * Creates the most suitable HudiIndexSupport strategy, considering configuration. + * Uses Supplier-based lazy instantiation combined with config checks. + * + * @param schemaTableName The table schema and name + * @param lazyMetaClient The Hudi table metadata client that is lazily instantiated. + * @param tupleDomain The query predicates. + * @param session Session containing session properties, which is required to control index behaviours for testing/debugging + * @return An Optional containing the chosen HudiIndexSupport strategy, or empty if none are applicable or enabled. + */ + public static Optional createIndexSupport( + SchemaTableName schemaTableName, + Lazy lazyMetaClient, + Lazy lazyTableMetadata, + TupleDomain tupleDomain, + ConnectorSession session) + { + // Define strategies as Suppliers paired with their config (isEnabled) flag + // IMPORTANT: Order of strategy here determines which index implementation is preferred first + List strategyProviders = List.of( + new StrategyProvider( + () -> isRecordLevelIndexEnabled(session), + () -> new HudiRecordLevelIndexSupport(session, schemaTableName, lazyMetaClient, lazyTableMetadata, tupleDomain)), + new StrategyProvider( + () -> isSecondaryIndexEnabled(session), + () -> new HudiSecondaryIndexSupport(session, schemaTableName, lazyMetaClient, lazyTableMetadata, tupleDomain)), + new StrategyProvider( + () -> isColumnStatsIndexEnabled(session), + () -> new HudiColumnStatsIndexSupport(session, schemaTableName, lazyMetaClient, lazyTableMetadata, tupleDomain)), + new StrategyProvider( + () -> isNoOpIndexEnabled(session), + () -> new HudiNoOpIndexSupport(schemaTableName, lazyMetaClient))); + + TupleDomain transformedTupleDomain = tupleDomain.transformKeys(HiveColumnHandle::getName); + for (StrategyProvider provider : strategyProviders) { + // Check if the strategy is enabled via config before instantiating + if (provider.isEnabled()) { + HudiIndexSupport strategy = provider.getStrategy(); + String strategyName = strategy.getClass().getSimpleName(); // Get name for logging + + // Check if the instantiated strategy is applicable + if (strategy.canApply(transformedTupleDomain)) { + log.debug(String.format("Selected %s strategy (Enabled & Applicable).", strategyName)); + return Optional.of(strategy); + } + else { + log.debug(String.format("%s is enabled but not applicable for this query.", strategyName)); + // Strategy object becomes eligible for GC here, acceptable penalty as the object is lightweight + } + } + else { + log.debug(String.format("Strategy associated with supplier %s is disabled by configuration.", provider.supplier.get().getClass().getSimpleName())); + } + } + + log.debug("No suitable and enabled index support strategy found to be applicable."); + return Optional.empty(); + } + + public static Optional createPartitionStatsIndexSupport( + SchemaTableName schemaTableName, + Lazy lazyMetaClient, + Lazy lazyTableMetadata, + TupleDomain tupleDomain, + ConnectorSession session) + { + StrategyProvider partitionStatsStrategy = new StrategyProvider( + () -> isPartitionStatsIndexEnabled(session), () -> new HudiPartitionStatsIndexSupport(session, schemaTableName, lazyMetaClient, lazyTableMetadata, tupleDomain)); + + TupleDomain transformedTupleDomain = tupleDomain.transformKeys(HiveColumnHandle::getName); + if (partitionStatsStrategy.isEnabled() && partitionStatsStrategy.getStrategy().canApply(transformedTupleDomain)) { + return Optional.of((HudiPartitionStatsIndexSupport) partitionStatsStrategy.getStrategy()); + } + return Optional.empty(); + } + + /** + * Helper class to pair the configuration check with the strategy supplier to allow for lazy initialization. + */ + private static class StrategyProvider + { + private final Supplier isEnabled; + private final Supplier supplier; + + StrategyProvider(Supplier isEnabled, Supplier supplier) + { + this.isEnabled = requireNonNull(isEnabled); + this.supplier = requireNonNull(supplier); + } + + boolean isEnabled() + { + return isEnabled.get(); + } + + HudiIndexSupport getStrategy() + { + return supplier.get(); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/reader/HudiTrinoReaderContext.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/reader/HudiTrinoReaderContext.java new file mode 100644 index 0000000000000..36c5342920f4e --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/reader/HudiTrinoReaderContext.java @@ -0,0 +1,227 @@ +/* + * 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.hudi.reader; + +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.util.HudiAvroSerializer; +import io.trino.plugin.hudi.util.SynthesizedColumnHandler; +import io.trino.spi.Page; +import io.trino.spi.connector.ConnectorPageSource; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericData.Record; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.config.RecordMergeMode; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; +import org.apache.hudi.common.model.HoodieEmptyRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.read.BufferedRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.UnaryOperator; + +public class HudiTrinoReaderContext + extends HoodieReaderContext +{ + ConnectorPageSource pageSource; + private final HudiAvroSerializer avroSerializer; + Map colToPosMap; + List dataHandles; + List columnHandles; + + public HudiTrinoReaderContext( + ConnectorPageSource pageSource, + List dataHandles, + List columnHandles, + SynthesizedColumnHandler synthesizedColumnHandler) + { + this.pageSource = pageSource; + this.avroSerializer = new HudiAvroSerializer(columnHandles, synthesizedColumnHandler); + this.dataHandles = dataHandles; + this.columnHandles = columnHandles; + this.colToPosMap = new HashMap<>(); + for (int i = 0; i < columnHandles.size(); i++) { + HiveColumnHandle handle = columnHandles.get(i); + colToPosMap.put(handle.getBaseColumnName(), i); + } + } + + @Override + public ClosableIterator getFileRecordIterator( + StoragePath storagePath, + long start, + long length, + Schema dataSchema, + Schema requiredSchema, + HoodieStorage storage) + { + return new ClosableIterator<>() + { + private Page currentPage; + private int currentPosition; + + @Override + public void close() + { + try { + pageSource.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean hasNext() + { + // If all records in the current page are consume, try to get next page + if (currentPage == null || currentPosition >= currentPage.getPositionCount()) { + if (pageSource.isFinished()) { + return false; + } + + // Get next page and reset currentPosition + currentPage = pageSource.getNextPage(); + currentPosition = 0; + + // If no more pages are available + return currentPage != null; + } + + return true; + } + + @Override + public IndexedRecord next() + { + if (!hasNext()) { + // TODO: This can probably be removed or ignored, added this as a sanity check + throw new RuntimeException("No more records in the iterator"); + } + + IndexedRecord record = avroSerializer.serialize(currentPage, currentPosition); + currentPosition++; + return record; + } + }; + } + + @Override + public IndexedRecord convertAvroRecord(IndexedRecord record) + { + return record; + } + + @Override + public GenericRecord convertToAvroRecord(IndexedRecord record, Schema schema) + { + GenericRecord ret = new GenericData.Record(schema); + for (Schema.Field field : schema.getFields()) { + ret.put(field.name(), record.get(field.pos())); + } + return ret; + } + + @Override + public Option getRecordMerger(RecordMergeMode mergeMode, String mergeStrategyId, String mergeImplClasses) + { + return Option.of(HoodieAvroRecordMerger.INSTANCE); + } + + @Override + public Object getValue(IndexedRecord record, Schema schema, String fieldName) + { + if (colToPosMap.containsKey(fieldName)) { + return record.get(colToPosMap.get(fieldName)); + } + else { + // record doesn't have the queried field, return null + return null; + } + } + + @Override + public IndexedRecord seal(IndexedRecord record) + { + // TODO: this can rely on colToPos map directly instead of schema + Schema schema = record.getSchema(); + IndexedRecord newRecord = new Record(schema); + List fields = schema.getFields(); + for (Schema.Field field : fields) { + int pos = schema.getField(field.name()).pos(); + newRecord.put(pos, record.get(pos)); + } + return newRecord; + } + + @Override + public IndexedRecord toBinaryRow(Schema schema, IndexedRecord record) + { + return record; + } + + @Override + public ClosableIterator mergeBootstrapReaders( + ClosableIterator closableIterator, Schema schema, + ClosableIterator closableIterator1, Schema schema1) + { + return null; + } + + @Override + public UnaryOperator projectRecord( + Schema from, + Schema to, + Map renamedColumns) + { + List toFields = to.getFields(); + int[] projection = new int[toFields.size()]; + for (int i = 0; i < projection.length; i++) { + projection[i] = from.getField(toFields.get(i).name()).pos(); + } + + return fromRecord -> { + IndexedRecord toRecord = new Record(to); + for (int i = 0; i < projection.length; i++) { + toRecord.put(i, fromRecord.get(projection[i])); + } + return toRecord; + }; + } + + @Override + public HoodieRecord constructHoodieRecord( + BufferedRecord bufferedRecord) + { + if (bufferedRecord.isDelete()) { + return new HoodieEmptyRecord<>( + new HoodieKey(bufferedRecord.getRecordKey(), null), + HoodieRecord.HoodieRecordType.AVRO); + } + + return new HoodieAvroIndexedRecord(bufferedRecord.getRecord()); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/reader/HudiTrinoRecord.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/reader/HudiTrinoRecord.java new file mode 100644 index 0000000000000..7fa0f71399edc --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/reader/HudiTrinoRecord.java @@ -0,0 +1,183 @@ +/* + * 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.hudi.reader; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.MetadataValues; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.BaseKeyGenerator; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + +public class HudiTrinoRecord + extends HoodieRecord +{ + public HudiTrinoRecord() + { + } + + @Override + public HoodieRecord newInstance() + { + return null; + } + + @Override + public HoodieRecord newInstance(HoodieKey hoodieKey, HoodieOperation hoodieOperation) + { + return null; + } + + @Override + public HoodieRecord newInstance(HoodieKey hoodieKey) + { + return null; + } + + @Override + public Comparable doGetOrderingValue(Schema schema, Properties properties) + { + return null; + } + + @Override + public HoodieRecordType getRecordType() + { + return null; + } + + @Override + public String getRecordKey(Schema schema, Option option) + { + return ""; + } + + @Override + public String getRecordKey(Schema schema, String s) + { + return ""; + } + + @Override + protected void writeRecordPayload(IndexedRecord page, Kryo kryo, Output output) + { + } + + @Override + protected IndexedRecord readRecordPayload(Kryo kryo, Input input) + { + return null; + } + + @Override + public Object[] getColumnValues(Schema schema, String[] strings, boolean b) + { + return new Object[0]; + } + + @Override + public HoodieRecord joinWith(HoodieRecord hoodieRecord, Schema schema) + { + return null; + } + + @Override + public HoodieRecord prependMetaFields(Schema schema, Schema schema1, + MetadataValues metadataValues, Properties properties) + { + return null; + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema schema, Properties properties, + Schema schema1, Map map) + { + return null; + } + + @Override + public boolean isDelete(Schema schema, Properties properties) + throws IOException + { + return false; + } + + @Override + public boolean shouldIgnore(Schema schema, Properties properties) + throws IOException + { + return false; + } + + @Override + public HoodieRecord copy() + { + return null; + } + + @Override + public Option> getMetadata() + { + return null; + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithParams(Schema schema, Properties properties, + Option> option, Boolean aBoolean, Option option1, + Boolean aBoolean1, Option option2) + throws IOException + { + return null; + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Schema schema, Properties properties, + Option option) + { + return null; + } + + @Override + public HoodieRecord truncateRecordKey(Schema schema, Properties properties, String s) + throws IOException + { + return null; + } + + @Override + public Option toIndexedRecord(Schema schema, Properties properties) + throws IOException + { + return null; + } + + @Override + public ByteArrayOutputStream getAvroBytes(Schema schema, Properties properties) + throws IOException + { + return null; + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiBackgroundSplitLoader.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiBackgroundSplitLoader.java new file mode 100644 index 0000000000000..56c89ea92634c --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiBackgroundSplitLoader.java @@ -0,0 +1,145 @@ +/* + * 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.hudi.split; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.log.Logger; +import io.trino.metastore.Partition; +import io.trino.plugin.hive.util.AsyncQueue; +import io.trino.plugin.hudi.HudiTableHandle; +import io.trino.plugin.hudi.partition.HudiPartitionInfoLoader; +import io.trino.plugin.hudi.query.HudiDirectoryLister; +import io.trino.plugin.hudi.query.index.HudiPartitionStatsIndexSupport; +import io.trino.plugin.hudi.query.index.IndexSupportFactory; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.SchemaTableName; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.util.Lazy; + +import java.util.ArrayList; +import java.util.Deque; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.function.Consumer; + +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.concurrent.MoreFutures.addExceptionCallback; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT; +import static io.trino.plugin.hudi.HudiSessionProperties.getSplitGeneratorParallelism; +import static io.trino.plugin.hudi.HudiSessionProperties.getTargetSplitSize; +import static java.util.Objects.requireNonNull; + +public class HudiBackgroundSplitLoader + implements Runnable +{ + private static final Logger log = Logger.get(HudiBackgroundSplitLoader.class); + private final HudiTableHandle tableHandle; + private final HudiDirectoryLister hudiDirectoryLister; + private final AsyncQueue asyncQueue; + private final Executor splitGeneratorExecutor; + private final int splitGeneratorNumThreads; + private final HudiSplitFactory hudiSplitFactory; + private final Lazy> lazyPartitions; + private final Consumer errorListener; + private final boolean enableMetadataTable; + private final Optional partitionIndexSupportOpt; + + public HudiBackgroundSplitLoader( + ConnectorSession session, + HudiTableHandle tableHandle, + HudiDirectoryLister hudiDirectoryLister, + AsyncQueue asyncQueue, + Executor splitGeneratorExecutor, + HudiSplitWeightProvider hudiSplitWeightProvider, + Lazy> lazyPartitionMap, + boolean enableMetadataTable, + Lazy lazyTableMetadata, + Consumer errorListener) + { + this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); + this.hudiDirectoryLister = requireNonNull(hudiDirectoryLister, "hudiDirectoryLister is null"); + this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); + this.splitGeneratorExecutor = requireNonNull(splitGeneratorExecutor, "splitGeneratorExecutorService is null"); + this.splitGeneratorNumThreads = getSplitGeneratorParallelism(session); + this.hudiSplitFactory = new HudiSplitFactory(tableHandle, hudiSplitWeightProvider, getTargetSplitSize(session)); + this.lazyPartitions = Lazy.lazily(() -> requireNonNull(lazyPartitionMap, "partitions is null").get().keySet().stream().toList()); + this.enableMetadataTable = enableMetadataTable; + this.errorListener = requireNonNull(errorListener, "errorListener is null"); + SchemaTableName schemaTableName = tableHandle.getSchemaTableName(); + this.partitionIndexSupportOpt = enableMetadataTable ? + IndexSupportFactory.createPartitionStatsIndexSupport(schemaTableName, Lazy.lazily(tableHandle::getMetaClient), lazyTableMetadata, tableHandle.getRegularPredicates(), session) : Optional.empty(); + } + + @Override + public void run() + { + // Wrap entire logic so that ANY error will be thrown out and not cause program to get stuc + try { + if (enableMetadataTable) { + generateSplits(true); + return; + } + + // Fallback to partition pruning generator + generateSplits(false); + } + catch (Exception e) { + errorListener.accept(e); + } + } + + private void generateSplits(boolean useIndex) + { + // Attempt to apply partition pruning using partition stats index + Optional> effectivePartitionsOpt = useIndex && partitionIndexSupportOpt.isPresent() ? + partitionIndexSupportOpt.get().prunePartitions(lazyPartitions.get()) : Optional.empty(); + + Deque partitionQueue = new ConcurrentLinkedDeque<>(effectivePartitionsOpt.orElse(lazyPartitions.get())); + List splitGenerators = new ArrayList<>(); + List> futures = new ArrayList<>(); + + for (int i = 0; i < splitGeneratorNumThreads; i++) { + HudiPartitionInfoLoader generator = new HudiPartitionInfoLoader(hudiDirectoryLister, tableHandle.getLatestCommitTime(), hudiSplitFactory, + asyncQueue, partitionQueue, useIndex); + splitGenerators.add(generator); + ListenableFuture future = Futures.submit(generator, splitGeneratorExecutor); + addExceptionCallback(future, errorListener); + futures.add(future); + } + + // Signal all generators to stop once partition queue is drained + splitGenerators.forEach(HudiPartitionInfoLoader::stopRunning); + + log.info("Wait for partition pruning split generation to finish on table %s.%s", tableHandle.getSchemaName(), tableHandle.getTableName()); + try { + Futures.whenAllComplete(futures) + .run(asyncQueue::finish, directExecutor()) + .get(); + log.info("Partition pruning split generation finished on table %s.%s", tableHandle.getSchemaName(), tableHandle.getTableName()); + } + catch (InterruptedException | ExecutionException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new TrinoException(HUDI_CANNOT_OPEN_SPLIT, "Error generating Hudi split", e); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiSplitFactory.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiSplitFactory.java new file mode 100644 index 0000000000000..872f56cb352d9 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiSplitFactory.java @@ -0,0 +1,128 @@ +/* + * 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.hudi.split; + +import com.google.common.collect.ImmutableList; +import io.airlift.units.DataSize; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hudi.HudiSplit; +import io.trino.plugin.hudi.HudiTableHandle; +import io.trino.plugin.hudi.file.HudiBaseFile; +import io.trino.plugin.hudi.file.HudiLogFile; +import io.trino.spi.TrinoException; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.util.Option; + +import java.util.Collections; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_FILESYSTEM_ERROR; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class HudiSplitFactory +{ + private static final double SPLIT_SLOP = 1.1; // 10% slop/overflow allowed in bytes per split while generating splits + + private final HudiTableHandle hudiTableHandle; + private final HudiSplitWeightProvider hudiSplitWeightProvider; + private final DataSize targetSplitSize; + + public HudiSplitFactory( + HudiTableHandle hudiTableHandle, + HudiSplitWeightProvider hudiSplitWeightProvider, + DataSize targetSplitSize) + { + this.hudiTableHandle = requireNonNull(hudiTableHandle, "hudiTableHandle is null"); + this.hudiSplitWeightProvider = requireNonNull(hudiSplitWeightProvider, "hudiSplitWeightProvider is null"); + this.targetSplitSize = requireNonNull(targetSplitSize, "targetSplitSize is null"); + } + + public List createSplits(List partitionKeys, FileSlice fileSlice, String commitTime) + { + return createHudiSplits(hudiTableHandle, partitionKeys, fileSlice, commitTime, hudiSplitWeightProvider, targetSplitSize); + } + + public static List createHudiSplits( + HudiTableHandle hudiTableHandle, + List partitionKeys, + FileSlice fileSlice, + String commitTime, + HudiSplitWeightProvider hudiSplitWeightProvider, + DataSize targetSplitSize) + { + if (fileSlice.isEmpty()) { + throw new TrinoException(HUDI_FILESYSTEM_ERROR, format("Not a valid file slice: %s", fileSlice.toString())); + } + + // Handle MERGE_ON_READ tables to be read in read_optimized mode + // IMPORTANT: These tables will have a COPY_ON_WRITE table type due to how `HudiTableTypeUtils#fromInputFormat` + if (fileSlice.getLogFiles().findAny().isEmpty() + || hudiTableHandle.getTableType().equals(HoodieTableType.COPY_ON_WRITE)) { + // Base file only + checkArgument(fileSlice.getBaseFile().isPresent(), + "Hudi base file must exist if there is no log file in the file slice"); + HoodieBaseFile baseFile = fileSlice.getBaseFile().get(); + long fileSize = baseFile.getFileSize(); + + if (fileSize == 0) { + return ImmutableList.of(new HudiSplit( + HudiBaseFile.of(baseFile), + Collections.emptyList(), + commitTime, + hudiTableHandle.getRegularPredicates(), + partitionKeys, + hudiSplitWeightProvider.calculateSplitWeight(fileSize))); + } + + ImmutableList.Builder splits = ImmutableList.builder(); + long targetSplitSizeInBytes = Math.max(targetSplitSize.toBytes(), baseFile.getPathInfo().getBlockSize()); + + long bytesRemaining = fileSize; + while (((double) bytesRemaining) / targetSplitSizeInBytes > SPLIT_SLOP) { + splits.add(new HudiSplit( + HudiBaseFile.of(baseFile, fileSize - bytesRemaining, targetSplitSizeInBytes), + Collections.emptyList(), + commitTime, + hudiTableHandle.getRegularPredicates(), + partitionKeys, + hudiSplitWeightProvider.calculateSplitWeight(targetSplitSizeInBytes))); + bytesRemaining -= targetSplitSizeInBytes; + } + if (bytesRemaining > 0) { + splits.add(new HudiSplit( + HudiBaseFile.of(baseFile, fileSize - bytesRemaining, bytesRemaining), + Collections.emptyList(), + commitTime, + hudiTableHandle.getRegularPredicates(), + partitionKeys, + hudiSplitWeightProvider.calculateSplitWeight(bytesRemaining))); + } + return splits.build(); + } + + // Base and log files + Option baseFileOption = fileSlice.getBaseFile(); + return ImmutableList.of(new HudiSplit( + baseFileOption.isPresent() ? HudiBaseFile.of(baseFileOption.get()) : null, + fileSlice.getLogFiles().map(HudiLogFile::of).toList(), + commitTime, + hudiTableHandle.getRegularPredicates(), + partitionKeys, + hudiSplitWeightProvider.calculateSplitWeight(fileSlice.getTotalFileSize()))); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiSplitWeightProvider.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiSplitWeightProvider.java new file mode 100644 index 0000000000000..dd90dd3563401 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/HudiSplitWeightProvider.java @@ -0,0 +1,26 @@ +/* + * 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.hudi.split; + +import io.trino.spi.SplitWeight; + +public interface HudiSplitWeightProvider +{ + SplitWeight calculateSplitWeight(long splitSizeInBytes); + + static HudiSplitWeightProvider uniformStandardWeightProvider() + { + return splitSizeInBytes -> SplitWeight.standard(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/SizeBasedSplitWeightProvider.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/SizeBasedSplitWeightProvider.java new file mode 100644 index 0000000000000..e94e2d73c373f --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/split/SizeBasedSplitWeightProvider.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.hudi.split; + +import io.airlift.units.DataSize; +import io.trino.spi.SplitWeight; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.Math.clamp; +import static java.util.Objects.requireNonNull; + +public class SizeBasedSplitWeightProvider + implements HudiSplitWeightProvider +{ + private final double minimumWeight; + private final double standardSplitSizeInBytes; + + public SizeBasedSplitWeightProvider(double minimumWeight, DataSize standardSplitSize) + { + checkArgument( + Double.isFinite(minimumWeight) && minimumWeight > 0 && minimumWeight <= 1, + "minimumWeight must be > 0 and <= 1, found: %s", minimumWeight); + this.minimumWeight = minimumWeight; + long standardSplitSizeInBytesLong = requireNonNull(standardSplitSize, "standardSplitSize is null").toBytes(); + checkArgument(standardSplitSizeInBytesLong > 0, "standardSplitSize must be > 0, found: %s", standardSplitSize); + this.standardSplitSizeInBytes = (double) standardSplitSizeInBytesLong; + } + + @Override + public SplitWeight calculateSplitWeight(long splitSizeInBytes) + { + double computedWeight = splitSizeInBytes / standardSplitSizeInBytes; + // Clamp the value between the minimum weight and 1.0 (standard weight) + return SplitWeight.fromProportion(clamp(computedWeight, minimumWeight, 1.0)); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/ForHudiTableStatistics.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/ForHudiTableStatistics.java new file mode 100644 index 0000000000000..f324887a3e7c3 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/ForHudiTableStatistics.java @@ -0,0 +1,29 @@ +/* + * 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.hudi.stats; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@BindingAnnotation +public @interface ForHudiTableStatistics {} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/HudiTableStatistics.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/HudiTableStatistics.java new file mode 100644 index 0000000000000..04526521b6ae2 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/HudiTableStatistics.java @@ -0,0 +1,23 @@ +/* + * 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.hudi.stats; + +import io.trino.spi.statistics.TableStatistics; +import org.apache.hudi.common.table.timeline.HoodieInstant; + +public record HudiTableStatistics( + HoodieInstant latestCommit, + TableStatistics tableStatistics) +{ +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/TableMetadataReader.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/TableMetadataReader.java new file mode 100644 index 0000000000000..e87122de07953 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/TableMetadataReader.java @@ -0,0 +1,114 @@ +/* + * 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.hudi.stats; + +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.common.util.hash.ColumnIndexID; +import org.apache.hudi.common.util.hash.FileIndexID; +import org.apache.hudi.common.util.hash.PartitionIndexID; +import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieMetadataMetrics; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.storage.HoodieStorage; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Reads metadata efficiently from a Hudi metadata table. + */ +public class TableMetadataReader + extends HoodieBackedTableMetadata +{ + TableMetadataReader(HoodieEngineContext engineContext, HoodieStorage storage, + HoodieMetadataConfig metadataConfig, String datasetBasePath, boolean reuse) + { + super(engineContext, storage, metadataConfig, datasetBasePath, reuse); + } + + /** + * Retrieves column statistics for the specified partition and file names. + * + * @param partitionNameFileNameList a list of partition and file name pairs for which column statistics are retrieved + * @param columnNames a list of column names for which statistics are needed + * @return a map from column name to their corresponding {@link HoodieColumnRangeMetadata} + * @throws HoodieMetadataException if an error occurs while fetching the column statistics + */ + Map getColumnStats(List> partitionNameFileNameList, List columnNames) + throws HoodieMetadataException + { + return computeFileToColumnStatsMap(computeColumnStatsLookupKeys(partitionNameFileNameList, columnNames)); + } + + /** + * @param partitionNameFileNameList a list of partition and file name pairs for which column stats need to be retrieved + * @param columnNames list of column names for which stats are needed + * @return a list of column stats keys to look up in the metadata table col_stats partition. + */ + private List computeColumnStatsLookupKeys( + final List> partitionNameFileNameList, + final List columnNames) + { + return columnNames.stream() + .flatMap(columnName -> partitionNameFileNameList.stream() + .map(partitionNameFileNamePair -> HoodieMetadataPayload.getColumnStatsIndexKey( + new PartitionIndexID(HoodieTableMetadataUtil.getColumnStatsIndexPartitionIdentifier(partitionNameFileNamePair.getLeft())), + new FileIndexID(partitionNameFileNamePair.getRight()), + new ColumnIndexID(columnName)))) + .toList(); + } + + /** + * @param columnStatsLookupKeys a map from column stats key to partition and file name pair + * @return a map from column name to merged HoodieMetadataColumnStats + */ + private Map computeFileToColumnStatsMap(List columnStatsLookupKeys) + { + HoodieTimer timer = HoodieTimer.start(); + Map> hoodieRecords = + getRecordsByKeys(columnStatsLookupKeys, MetadataPartitionType.COLUMN_STATS.getPartitionPath()); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_COLUMN_STATS_METADATA_STR, timer.endTimer())); + return hoodieRecords.values().stream() + .collect(Collectors.groupingBy( + r -> r.getData().getColumnStatMetadata().get().getColumnName(), + Collectors.mapping(r -> r.getData().getColumnStatMetadata().get(), Collectors.toList()))) + .entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + e -> { + long valueCount = 0L; + long nullCount = 0L; + long totalSize = 0L; + long totalUncompressedSize = 0L; + for (HoodieMetadataColumnStats stats : e.getValue()) { + valueCount += stats.getValueCount(); + nullCount += stats.getNullCount(); + totalSize += stats.getTotalSize(); + totalUncompressedSize += stats.getTotalUncompressedSize(); + } + return HoodieColumnRangeMetadata.create( + "", e.getKey(), null, null, nullCount, valueCount, totalSize, totalUncompressedSize); + })); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/TableStatisticsReader.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/TableStatisticsReader.java new file mode 100644 index 0000000000000..30e03622f5775 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/stats/TableStatisticsReader.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.hudi.stats; + +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hudi.storage.TrinoStorageConfiguration; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.statistics.ColumnStatistics; +import io.trino.spi.statistics.Estimate; +import io.trino.spi.statistics.TableStatistics; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.collection.Pair; + +import java.util.List; +import java.util.Map; + +import static io.trino.plugin.hudi.HudiUtil.getFileSystemView; + +/** + * Reads table statistics of a Hudi table from the metadata table files and column stats partitions. + */ +public class TableStatisticsReader +{ + private static final Logger log = Logger.get(TableStatisticsReader.class); + private final HoodieTableMetaClient metaClient; + private final TableMetadataReader tableMetadata; + private final HoodieTableFileSystemView fileSystemView; + + private TableStatisticsReader(HoodieTableMetaClient metaClient) + { + this.metaClient = metaClient; + HoodieEngineContext engineContext = new HoodieLocalEngineContext(new TrinoStorageConfiguration()); + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().enable(true).build(); + this.tableMetadata = new TableMetadataReader( + engineContext, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString(), true); + this.fileSystemView = getFileSystemView(tableMetadata, metaClient); + } + + public static TableStatisticsReader create(HoodieTableMetaClient metaClient) + { + return new TableStatisticsReader(metaClient); + } + + /** + * Retrieves table statistics of a Hudi table based on the latest commit and specified columns. + * + * @param latestCommit the most recent commit at which to retrieve the statistics + * @param columnHandles a list of {@link HiveColumnHandle} representing the columns for which statistics are needed + * @return {@link TableStatistics} object containing the statistics of the specified columns, or empty statistics if unable to retrieve + */ + public TableStatistics getTableStatistics(HoodieInstant latestCommit, + List columnHandles) + { + List columnNames = columnHandles.stream() + .map(HiveColumnHandle::getName) + .toList(); + Map columnStatsMap = getColumnStats(latestCommit, tableMetadata, fileSystemView, columnNames); + if (columnStatsMap.isEmpty()) { + log.warn("Unable to get column stats from metadata table for table, returning empty table statistics: %s", + metaClient.getBasePath()); + return TableStatistics.empty(); + } + long rowCount = columnStatsMap.values().stream() + .map(e -> e.getNullCount() + e.getValueCount()) + .max(Long::compare) + .get(); + ImmutableMap.Builder columnHandleBuilder = ImmutableMap.builder(); + for (HiveColumnHandle columnHandle : columnHandles) { + HoodieColumnRangeMetadata columnStats = columnStatsMap.get(columnHandle.getName()); + if (columnStats == null) { + log.warn("Unable to get column stats for column %s in table %s", + columnHandle.getName(), metaClient.getBasePath()); + continue; + } + ColumnStatistics.Builder columnStatisticsBuilder = new ColumnStatistics.Builder(); + long totalCount = columnStats.getNullCount() + columnStats.getValueCount(); + columnStatisticsBuilder.setNullsFraction(Estimate.of( + columnStats.getNullCount() / (double) totalCount)); + columnStatisticsBuilder.setDataSize(Estimate.of(columnStats.getTotalUncompressedSize() / (double) totalCount)); + columnHandleBuilder.put(columnHandle, columnStatisticsBuilder.build()); + } + return new TableStatistics(Estimate.of(rowCount), columnHandleBuilder.buildOrThrow()); + } + + private static Map getColumnStats( + HoodieInstant latestCommit, + TableMetadataReader tableMetadata, + HoodieTableFileSystemView fileSystemView, + List columnNames) + { + fileSystemView.loadAllPartitions(); + List> filePaths = fileSystemView.getAllLatestBaseFilesBeforeOrOn(latestCommit.requestedTime()) + .entrySet() + .stream().flatMap(entry -> entry.getValue() + .map(baseFile -> Pair.of(entry.getKey(), baseFile.getFileName()))) + .toList(); + return tableMetadata.getColumnStats(filePaths, columnNames); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/HudiTrinoInlineStorage.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/HudiTrinoInlineStorage.java new file mode 100644 index 0000000000000..26e1eae088a0b --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/HudiTrinoInlineStorage.java @@ -0,0 +1,242 @@ +/* + * 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.hudi.storage; + +import io.trino.filesystem.TrinoInputStream; +import io.trino.plugin.hudi.io.InlineSeekableDataInputStream; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.io.SeekableDataInputStream; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathFilter; +import org.apache.hudi.storage.StoragePathInfo; +import org.apache.hudi.storage.inline.InLineFSUtils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.URI; +import java.util.Collections; +import java.util.List; + +public class HudiTrinoInlineStorage + extends HoodieStorage +{ + private final HudiTrinoStorage storage; + + public HudiTrinoInlineStorage(HudiTrinoStorage storage) + { + super(storage.getConf()); + this.storage = storage; + } + + @Override + public HoodieStorage newInstance(StoragePath path, StorageConfiguration storageConf) + { + if (InLineFSUtils.SCHEME.equals(path.toUri().getScheme())) { + return this; + } + return storage; + } + + @Override + public String getScheme() + { + return InLineFSUtils.SCHEME; + } + + @Override + public int getDefaultBlockSize(StoragePath path) + { + return storage.getDefaultBlockSize(getFilePathFromInlinePath(path)); + } + + @Override + public int getDefaultBufferSize() + { + return storage.getDefaultBufferSize(); + } + + @Override + public short getDefaultReplication(StoragePath path) + { + return storage.getDefaultReplication(getFilePathFromInlinePath(path)); + } + + @Override + public URI getUri() + { + return URI.create(getScheme()); + } + + @Override + public OutputStream create(StoragePath path, boolean b) + throws IOException + { + throw new HoodieNotSupportedException("Hudi inline storage supports reads only."); + } + + @Override + public OutputStream create(StoragePath path, boolean b, Integer integer, Short aShort, Long aLong) + throws IOException + { + throw new HoodieNotSupportedException("Hudi inline storage supports reads only."); + } + + @Override + public InputStream open(StoragePath path) + throws IOException + { + return openSeekable(path, getDefaultBufferSize(), false); + } + + @Override + public SeekableDataInputStream openSeekable(StoragePath path, int bufferSize, boolean wrapStream) + throws IOException + { + return new InlineSeekableDataInputStream( + (TrinoInputStream) storage.open(getFilePathFromInlinePath(path)), + InLineFSUtils.startOffset(path), + InLineFSUtils.length(path)); + } + + @Override + public OutputStream append(StoragePath path) + throws IOException + { + throw new HoodieNotSupportedException("Hudi inline storage supports reads only."); + } + + @Override + public boolean exists(StoragePath path) + throws IOException + { + return storage.exists(getFilePathFromInlinePath(path)); + } + + @Override + public StoragePathInfo getPathInfo(StoragePath path) + throws IOException + { + StoragePathInfo pathInfo = storage.getPathInfo(getFilePathFromInlinePath(path)); + return new StoragePathInfo( + path, InLineFSUtils.length(path), pathInfo.isDirectory(), + pathInfo.getBlockReplication(), pathInfo.getBlockSize(), + pathInfo.getModificationTime()); + } + + @Override + public boolean createDirectory(StoragePath path) + throws IOException + { + throw new HoodieNotSupportedException("Hudi inline storage supports reads only."); + } + + @Override + public List listDirectEntries(StoragePath path) + throws IOException + { + // This is supposed to be called on a file path only + StoragePathInfo pathInfo = getPathInfo(path); + ValidationUtils.checkArgument(pathInfo.isFile(), + "HudiTrinoInlineStorage#listDirectEntries should only be called on a file path"); + return Collections.singletonList(pathInfo); + } + + @Override + public List listFiles(StoragePath path) + throws IOException + { + throw new HoodieNotSupportedException("This API should not be called by Hudi inline storage."); + } + + @Override + public List listDirectEntries(StoragePath path, StoragePathFilter storagePathFilter) + throws IOException + { + throw new HoodieNotSupportedException("This API should not be called by Hudi inline storage."); + } + + @Override + public List globEntries(StoragePath path, StoragePathFilter storagePathFilter) + throws IOException + { + throw new HoodieNotSupportedException("This API should not be called by Hudi inline storage."); + } + + @Override + public boolean rename(StoragePath oldPath, StoragePath newPath) + throws IOException + { + throw new HoodieNotSupportedException("Hudi inline storage supports reads only."); + } + + @Override + public boolean deleteDirectory(StoragePath path) + throws IOException + { + throw new HoodieNotSupportedException("Hudi inline storage supports reads only."); + } + + @Override + public boolean deleteFile(StoragePath path) + throws IOException + { + throw new HoodieNotSupportedException("Hudi inline storage supports reads only."); + } + + @Override + public void setModificationTime(StoragePath path, long modificationTime) + throws IOException + { + throw new HoodieNotSupportedException("Hudi inline storage supports reads only."); + } + + @Override + public Object getFileSystem() + { + return storage.getFileSystem(); + } + + @Override + public HoodieStorage getRawStorage() + { + return this; + } + + @Override + public void close() + throws IOException + { + } + + private static StoragePath getFilePathFromInlinePath(StoragePath inlinePath) + { + assertInlineFSPath(inlinePath); + String outerFileScheme = inlinePath.getParent().getName(); + StoragePath basePath = inlinePath.getParent().getParent(); + ValidationUtils.checkArgument(basePath.toString().contains(":"), "Invalid InLineFS path: " + inlinePath); + String pathExceptScheme = basePath.toString().substring(basePath.toString().indexOf(":") + 1); + String fullPath = outerFileScheme + ":" + (outerFileScheme.equals("file") || outerFileScheme.equals("local") ? "/" : "") + pathExceptScheme; + return new StoragePath(fullPath); + } + + private static void assertInlineFSPath(StoragePath inlinePath) + { + String scheme = inlinePath.toUri().getScheme(); + ValidationUtils.checkArgument(InLineFSUtils.SCHEME.equals(scheme)); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/HudiTrinoStorage.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/HudiTrinoStorage.java new file mode 100644 index 0000000000000..48c5409c10d83 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/HudiTrinoStorage.java @@ -0,0 +1,287 @@ +/* + * 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.hudi.storage; + +import io.airlift.units.DataSize; +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.plugin.hudi.io.TrinoSeekableDataInputStream; +import org.apache.hudi.io.SeekableDataInputStream; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathFilter; +import org.apache.hudi.storage.StoragePathInfo; +import org.apache.hudi.storage.inline.InLineFSUtils; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static io.airlift.units.DataSize.Unit.MEGABYTE; + +public class HudiTrinoStorage + extends HoodieStorage +{ + private static final int DEFAULT_BLOCK_SIZE = (int) DataSize.of(32, MEGABYTE).toBytes(); + private static final int DEFAULT_BUFFER_SIZE = 4096; + private static final int DEFAULT_REPLICATION = 1; + + private final TrinoFileSystem fileSystem; + + public HudiTrinoStorage(TrinoFileSystem fileSystem, TrinoStorageConfiguration storageConf) + { + super(storageConf); + this.fileSystem = fileSystem; + } + + public static Location convertToLocation(StoragePath path) + { + return Location.of(path.toString()); + } + + public static StoragePath convertToPath(Location location) + { + return new StoragePath(location.toString()); + } + + public static StoragePathInfo convertToPathInfo(FileEntry fileEntry) + { + return new StoragePathInfo( + convertToPath(fileEntry.location()), + fileEntry.length(), + false, + (short) 0, + 0, + fileEntry.lastModified().toEpochMilli()); + } + + @Override + public HoodieStorage newInstance(StoragePath path, StorageConfiguration storageConf) + { + if (InLineFSUtils.SCHEME.equals(path.toUri().getScheme())) { + return new HudiTrinoInlineStorage(this); + } + return this; + } + + @Override + public String getScheme() + { + return "file"; + } + + @Override + public int getDefaultBlockSize(StoragePath path) + { + return DEFAULT_BLOCK_SIZE; + } + + @Override + public int getDefaultBufferSize() + { + return DEFAULT_BUFFER_SIZE; + } + + @Override + public short getDefaultReplication(StoragePath path) + { + return DEFAULT_REPLICATION; + } + + @Override + public URI getUri() + { + return URI.create(getScheme()); + } + + @Override + public OutputStream create(StoragePath path, boolean overwrite) + throws IOException + { + return fileSystem.newOutputFile(convertToLocation(path)).create(); + } + + @Override + public OutputStream create(StoragePath path, boolean overwrite, Integer bufferSize, Short replication, Long sizeThreshold) + throws IOException + { + return create(path, overwrite); + } + + @Override + public InputStream open(StoragePath path) + throws IOException + { + return fileSystem.newInputFile(convertToLocation(path)).newStream(); + } + + @Override + public SeekableDataInputStream openSeekable(StoragePath path, int bufferSize, boolean wrapStream) + throws IOException + { + return new TrinoSeekableDataInputStream( + fileSystem.newInputFile(convertToLocation(path)).newStream()); + } + + @Override + public OutputStream append(StoragePath path) + throws IOException + { + throw new UnsupportedOperationException("HudiTrinoStorage does not support append operation."); + } + + @Override + public boolean exists(StoragePath path) + throws IOException + { + return fileSystem.newInputFile(convertToLocation(path)).exists(); + } + + @Override + public StoragePathInfo getPathInfo(StoragePath path) + throws IOException + { + Location location = convertToLocation(path); + Optional result = fileSystem.directoryExists(location); + if (result.isPresent() && result.get()) { + return new StoragePathInfo(path, 0, true, (short) 0, 0, 0); + } + TrinoInputFile inputFile = fileSystem.newInputFile(location); + if (!inputFile.exists()) { + throw new FileNotFoundException("Path " + path + " does not exist"); + } + return new StoragePathInfo(path, inputFile.length(), false, (short) 0, 0, inputFile.lastModified().toEpochMilli()); + } + + @Override + public boolean createDirectory(StoragePath path) + throws IOException + { + fileSystem.createDirectory(convertToLocation(path)); + return true; + } + + @Override + public List listDirectEntries(StoragePath path) + throws IOException + { + // TrinoFileSystem#listFiles lists recursively, we need to limit the result to only the direct children + Location location = convertToLocation(path); + FileIterator fileIterator = fileSystem.listFiles(location); + List fileList = new ArrayList<>(); + while (fileIterator.hasNext()) { + FileEntry entry = fileIterator.next(); + if (entry.location().parentDirectory().path().equals(location.path())) { + fileList.add(convertToPathInfo(entry)); + } + } + return fileList; + } + + @Override + public List listFiles(StoragePath path) + throws IOException + { + FileIterator fileIterator = fileSystem.listFiles(convertToLocation(path)); + List fileList = new ArrayList<>(); + while (fileIterator.hasNext()) { + fileList.add(convertToPathInfo(fileIterator.next())); + } + return fileList; + } + + @Override + public List listDirectEntries(StoragePath path, StoragePathFilter filter) + throws IOException + { + // TrinoFileSystem#listFiles lists recursively, we need to limit the result to only the direct children + Location location = convertToLocation(path); + FileIterator fileIterator = fileSystem.listFiles(location); + List fileList = new ArrayList<>(); + while (fileIterator.hasNext()) { + FileEntry entry = fileIterator.next(); + if (filter.accept(new StoragePath(entry.location().toString())) + && entry.location().parentDirectory().path().equals(location.path())) { + fileList.add(convertToPathInfo(entry)); + } + } + return fileList; + } + + @Override + public List globEntries(StoragePath pathPattern, StoragePathFilter filter) + throws IOException + { + throw new UnsupportedOperationException("HudiTrinoStorage does not support globEntries operation."); + } + + @Override + public boolean rename(StoragePath oldPath, StoragePath newPath) + throws IOException + { + fileSystem.renameFile(convertToLocation(oldPath), convertToLocation(newPath)); + return true; + } + + @Override + public boolean deleteDirectory(StoragePath path) + throws IOException + { + fileSystem.deleteDirectory(convertToLocation(path)); + return true; + } + + @Override + public boolean deleteFile(StoragePath path) + throws IOException + { + fileSystem.deleteFile(convertToLocation(path)); + return true; + } + + @Override + public void setModificationTime(StoragePath path, long modificationTime) + throws IOException + { + Location sameLocation = convertToLocation(path); + fileSystem.renameFile(sameLocation, sameLocation); + } + + @Override + public Object getFileSystem() + { + return fileSystem; + } + + @Override + public HoodieStorage getRawStorage() + { + return this; + } + + @Override + public void close() + throws IOException + { + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/TrinoStorageConfiguration.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/TrinoStorageConfiguration.java new file mode 100644 index 0000000000000..48939ae84c5a9 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/storage/TrinoStorageConfiguration.java @@ -0,0 +1,84 @@ +/* + * 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.hudi.storage; + +import io.trino.plugin.hudi.io.HudiTrinoIOFactory; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.storage.StorageConfiguration; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.hudi.common.config.HoodieStorageConfig.HOODIE_IO_FACTORY_CLASS; +import static org.apache.hudi.common.config.HoodieStorageConfig.HOODIE_STORAGE_CLASS; + +public class TrinoStorageConfiguration + extends StorageConfiguration +{ + private final Map configMap; + + public TrinoStorageConfiguration() + { + this(getDefaultConfigs()); + } + + public TrinoStorageConfiguration(Map configMap) + { + this.configMap = configMap; + } + + public static Map getDefaultConfigs() + { + Map configMap = new HashMap<>(); + configMap.put(HOODIE_IO_FACTORY_CLASS.key(), HudiTrinoIOFactory.class.getName()); + configMap.put(HOODIE_STORAGE_CLASS.key(), HudiTrinoStorage.class.getName()); + return configMap; + } + + @Override + public StorageConfiguration newInstance() + { + return new TrinoStorageConfiguration(new HashMap<>(configMap)); + } + + @Override + public Object unwrap() + { + return configMap; + } + + @Override + public Object unwrapCopy() + { + return new HashMap<>(configMap); + } + + @Override + public void set(String key, String value) + { + configMap.put(key, value); + } + + @Override + public Option getString(String key) + { + return Option.ofNullable(configMap.get(key)); + } + + @Override + public StorageConfiguration getInline() + { + return newInstance(); + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/HudiAvroSerializer.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/HudiAvroSerializer.java new file mode 100644 index 0000000000000..91b38f593e19e --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/HudiAvroSerializer.java @@ -0,0 +1,504 @@ +/* + * 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.hudi.util; + +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.trino.plugin.hive.HiveColumnHandle; +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.BlockBuilder; +import io.trino.spi.block.Fixed12BlockBuilder; +import io.trino.spi.block.MapBlockBuilder; +import io.trino.spi.block.RowBlockBuilder; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DateType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Decimals; +import io.trino.spi.type.Int128; +import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.LongTimestampWithTimeZone; +import io.trino.spi.type.MapType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.SqlDate; +import io.trino.spi.type.SqlDecimal; +import io.trino.spi.type.SqlTimestamp; +import io.trino.spi.type.SqlVarbinary; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarbinaryType; +import io.trino.spi.type.VarcharType; +import org.apache.avro.Conversions; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.util.Utf8; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.DateTimeException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.List; +import java.util.Map; + +import static com.google.common.base.Verify.verify; +import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.plugin.hudi.HudiUtil.constructSchema; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.trino.spi.StandardErrorCode.NUMERIC_VALUE_OUT_OF_RANGE; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.Decimals.encodeShortScaledValue; +import static io.trino.spi.type.Decimals.writeBigDecimal; +import static io.trino.spi.type.Decimals.writeShortDecimal; +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.TimestampWithTimeZoneType.TIMESTAMP_TZ_MICROS; +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_SECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MICROSECOND; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_MICROSECOND; +import static io.trino.spi.type.TinyintType.TINYINT; +import static java.lang.Integer.parseInt; +import static java.lang.Math.floorDiv; +import static java.lang.Math.floorMod; +import static java.lang.Math.toIntExact; +import static java.lang.String.format; +import static java.time.ZoneOffset.UTC; + +public class HudiAvroSerializer +{ + private static final int[] NANO_FACTOR = { + -1, // 0, no need to multiply + 100_000_000, // 1 digit after the dot + 10_000_000, // 2 digits after the dot + 1_000_000, // 3 digits after the dot + 100_000, // 4 digits after the dot + 10_000, // 5 digits after the dot + 1000, // 6 digits after the dot + 100, // 7 digits after the dot + 10, // 8 digits after the dot + 1, // 9 digits after the dot + }; + + private static final AvroDecimalConverter DECIMAL_CONVERTER = new AvroDecimalConverter(); + private final SynthesizedColumnHandler synthesizedColumnHandler; + + private final List columnHandles; + private final List columnTypes; + private final Schema schema; + + public HudiAvroSerializer(List columnHandles, SynthesizedColumnHandler synthesizedColumnHandler) + { + this.columnHandles = columnHandles; + this.columnTypes = columnHandles.stream().map(HiveColumnHandle::getType).toList(); + // Fetches projected schema + this.schema = constructSchema(columnHandles.stream().filter(ch -> !ch.isHidden()).map(HiveColumnHandle::getName).toList(), + columnHandles.stream().filter(ch -> !ch.isHidden()).map(HiveColumnHandle::getHiveType).toList()); + this.synthesizedColumnHandler = synthesizedColumnHandler; + } + + public IndexedRecord serialize(Page sourcePage, int position) + { + IndexedRecord record = new GenericData.Record(schema); + for (int i = 0; i < columnTypes.size(); i++) { + Object value = getValue(sourcePage, i, position); + record.put(i, value); + } + return record; + } + + public Object getValue(Page sourcePage, int channel, int position) + { + return columnTypes.get(channel).getObjectValue(null, sourcePage.getBlock(channel), position); + } + + public void buildRecordInPage(PageBuilder pageBuilder, IndexedRecord record) + { + pageBuilder.declarePosition(); + int blockSeq = 0; + for (int channel = 0; channel < columnTypes.size(); channel++, blockSeq++) { + BlockBuilder output = pageBuilder.getBlockBuilder(blockSeq); + HiveColumnHandle columnHandle = columnHandles.get(channel); + if (synthesizedColumnHandler.isSynthesizedColumn(columnHandle)) { + synthesizedColumnHandler.getColumnStrategy(columnHandle).appendToBlock(output, columnTypes.get(channel)); + } + else { + // Record may not be projected, get index from it + int fieldPosInSchema = record.getSchema().getField(columnHandle.getName()).pos(); + appendTo(columnTypes.get(channel), record.get(fieldPosInSchema), output); + } + } + } + + public static void appendTo(Type type, Object value, BlockBuilder output) + { + if (value == null) { + output.appendNull(); + return; + } + + Class javaType = type.getJavaType(); + try { + if (javaType == boolean.class) { + type.writeBoolean(output, (Boolean) value); + } + else if (javaType == long.class) { + if (type.equals(BIGINT)) { + type.writeLong(output, ((Number) value).longValue()); + } + else if (type.equals(INTEGER)) { + type.writeLong(output, ((Number) value).intValue()); + } + else if (type.equals(SMALLINT)) { + type.writeLong(output, ((Number) value).shortValue()); + } + else if (type.equals(TINYINT)) { + type.writeLong(output, ((Number) value).byteValue()); + } + else if (type.equals(REAL)) { + if (value instanceof Number) { + // Directly get the float value from the Number + // This preserves the fractional part + float floatValue = ((Number) value).floatValue(); + + // Get the IEEE 754 single-precision 32-bit representation of this float + int intBits = Float.floatToRawIntBits(floatValue); + + // The writeLong method expects these int bits, passed as a long + // NOTE: Java handles the widening conversion from int to long + type.writeLong(output, intBits); + } + else { + // Handle cases where 'value' is not a Number + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s | value type: %s", javaType.getSimpleName(), type, value.getClass().getName())); + } + } + else if (type instanceof DecimalType decimalType) { + if (value instanceof SqlDecimal sqlDecimal) { + if (decimalType.isShort()) { + writeShortDecimal(output, sqlDecimal.toBigDecimal().unscaledValue().longValue()); + } + else { + writeBigDecimal(decimalType, output, sqlDecimal.toBigDecimal()); + } + } + else if (value instanceof GenericData.Fixed fixed) { + verify(decimalType.isShort(), "The type should be short decimal"); + BigDecimal decimal = DECIMAL_CONVERTER.convert(decimalType.getPrecision(), decimalType.getScale(), fixed.bytes()); + type.writeLong(output, encodeShortScaledValue(decimal, decimalType.getScale())); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s | value type: %s", javaType.getSimpleName(), type, value.getClass().getName())); + } + } + else if (type.equals(DATE)) { + if (value instanceof SqlDate sqlDate) { + type.writeLong(output, sqlDate.getDays()); + } + else if (value instanceof Integer days) { + ((DateType) type).writeInt(output, days); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s | value type: %s", javaType.getSimpleName(), type, value.getClass().getName())); + } + } + else if (type.equals(TIMESTAMP_MICROS)) { + type.writeLong(output, toTrinoTimestamp(((Utf8) value).toString())); + } + else if (type.equals(TIME_MICROS)) { + type.writeLong(output, (long) value * PICOSECONDS_PER_MICROSECOND); + } + 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.getJavaType() == Int128.class) { + writeObject(output, type, value); + } + else if (javaType == Slice.class) { + writeSlice(output, type, value); + } + else if (javaType == LongTimestamp.class) { + if (value instanceof SqlTimestamp sqlTimestamp) { + // value is read from parquet + // From tests, sqlTimestamp is a UTC epoch that is converted from ZoneId#systemDefault() + // IMPORTANT: Even when session's zoneId != ZoneId#systemDefault(), ZoneId#systemDefault() is used calculate/produce the false UTC. + // The current sqlTimestamp is calculated as such: + // 1. The true UTC timestamp that is stored in file is assumed to be in the local timezone + // 2. Trino will them attempt to convert this to a false UTC by subtracting the timezone's offset (factoring offset rules like DST) + // Hence, to calculate the true UTC, we will just have to reverse the steps + + // Reconstruct the original local wall time from sqlTimestamp's fields + long microsFromSqlTs = sqlTimestamp.getEpochMicros(); + // picosFromSqlTs is defined as "picoseconds within the microsecond" (0 to 999,999) + int picosFromSqlTs = sqlTimestamp.getPicosOfMicros(); + long secondsComponent = microsFromSqlTs / 1_000_000L; + // Storing nanos component separately from seconds component, hence the modulo to remove secondsComponent + int nanosComponent = (int) ((microsFromSqlTs % 1_000_000L) * 1000L + picosFromSqlTs / 1000L); + LocalDateTime originalLocalWallTime = LocalDateTime.ofEpochSecond(secondsComponent, nanosComponent, ZoneOffset.UTC); + + // Determine the ZoneId in which originalLocalWallTime was observed + ZoneId assumedOriginalZoneId = ZoneId.systemDefault(); + + // Convert to true UTC by interpreting the local wall time in its original zone + // This correctly handles DST for that zone at that specific historical date/time. + ZonedDateTime zdtInOriginalZone; + try { + zdtInOriginalZone = originalLocalWallTime.atZone(assumedOriginalZoneId); + } + catch (DateTimeException e) { + // Handle cases where the local time is invalid in the zone (e.g., during DST "spring forward" gap) or ambiguous (during DST "fall back" overlap). + // For now, rethrow or log, as robustly handling this requires a defined policy. + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Cannot uniquely or validly map local time %s to zone %s: %s", + originalLocalWallTime, assumedOriginalZoneId, e.getMessage()), e); + } + Instant trueUtcInstant = zdtInOriginalZone.toInstant(); + + // Extract true UTC epoch micros and picos + long trueUtcEpochSeconds = trueUtcInstant.getEpochSecond(); + long trueUtcEpochMicrosContributionFromSeconds; + try { + trueUtcEpochMicrosContributionFromSeconds = Math.multiplyExact(trueUtcEpochSeconds, 1_000_000L); + } + catch (ArithmeticException e) { + // Multiplication could overflow if epochSeconds is approximately more than 292,271 years from epoch + throw new TrinoException(NUMERIC_VALUE_OUT_OF_RANGE, + "Timestamp " + trueUtcInstant + " is too far in the past or future to be represented as microseconds in a long.", e); + } + + long trueUtcEpochMicrosContributionFromNanos = trueUtcInstant.getNano() / 1000L; + long trueUtcEpochMicros; + + try { + trueUtcEpochMicros = Math.addExact(trueUtcEpochMicrosContributionFromSeconds, trueUtcEpochMicrosContributionFromNanos); + } + catch (ArithmeticException e) { + // Addition could also theoretically overflow if epochMicrosContributionFromSeconds is: + // 1. Very close to Long.MAX_VALUE and trueUtcEpochMicrosContributionFromNanos is positive + // 2. Very close to Long.MIN_VALUE and trueUtcEpochMicrosContributionFromNanos is negative + throw new TrinoException(NUMERIC_VALUE_OUT_OF_RANGE, + "Timestamp " + trueUtcInstant + " results in microsecond representation overflow after adding nanosecond component.", e); + } + + int truePicosOfMicros = (trueUtcInstant.getNano() % 1000) * 1000; + ((Fixed12BlockBuilder) output).writeFixed12(trueUtcEpochMicros, truePicosOfMicros); + } + else if (value instanceof Long epochMicros) { + // value is read from log + // epochMicros is in micros, no nanos or picos component + ((Fixed12BlockBuilder) output).writeFixed12(epochMicros, 0); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s | value type: %s", javaType.getSimpleName(), type, value.getClass().getName())); + } + } + else if (javaType == LongTimestampWithTimeZone.class) { + verify(type.equals(TIMESTAMP_TZ_MICROS)); + long epochMicros = (long) value; + int picosOfMillis = toIntExact(floorMod(epochMicros, MICROSECONDS_PER_MILLISECOND)) * PICOSECONDS_PER_MICROSECOND; + type.writeObject(output, fromEpochMillisAndFraction(floorDiv(epochMicros, MICROSECONDS_PER_MILLISECOND), picosOfMillis, UTC_KEY)); + } + else if (type instanceof ArrayType arrayType) { + writeArray((ArrayBlockBuilder) output, (List) value, arrayType); + } + else if (type instanceof RowType rowType) { + if (value instanceof List list) { + // value is read from parquet + writeRow((RowBlockBuilder) output, rowType, list); + } + else if (value instanceof GenericRecord record) { + // value is read from log + writeRow((RowBlockBuilder) output, rowType, record); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s | value type: %s", javaType.getSimpleName(), type, value.getClass().getName())); + } + } + else if (type instanceof MapType mapType) { + writeMap((MapBlockBuilder) output, mapType, (Map) value); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, format("Unhandled type for %s: %s", javaType.getSimpleName(), type)); + } + } + catch (ClassCastException cce) { + throw new TrinoException(GENERIC_INTERNAL_ERROR, format("ClassCastException for type %s: %s with error %s", javaType.getSimpleName(), type, cce)); + } + } + + public static LocalDateTime toLocalDateTime(String datetime) + { + int dotPosition = datetime.indexOf('.'); + if (dotPosition == -1) { + // no sub-second element + return LocalDateTime.from(DateTimeFormatter.ISO_LOCAL_DATE_TIME.parse(datetime)); + } + LocalDateTime result = LocalDateTime.from(DateTimeFormatter.ISO_LOCAL_DATE_TIME.parse(datetime.substring(0, dotPosition))); + // has sub-second element, so convert to nanosecond + String nanosStr = datetime.substring(dotPosition + 1); + int nanoOfSecond = parseInt(nanosStr) * NANO_FACTOR[nanosStr.length()]; + return result.withNano(nanoOfSecond); + } + + public static long toTrinoTimestamp(String datetime) + { + Instant instant = toLocalDateTime(datetime).toInstant(UTC); + return (instant.getEpochSecond() * MICROSECONDS_PER_SECOND) + (instant.getNano() / NANOSECONDS_PER_MICROSECOND); + } + + private static void writeSlice(BlockBuilder output, Type type, Object value) + { + if (type instanceof VarcharType) { + if (value instanceof Utf8) { + type.writeSlice(output, utf8Slice(((Utf8) value).toString())); + } + else if (value instanceof String) { + type.writeSlice(output, utf8Slice((String) value)); + } + else { + type.writeSlice(output, utf8Slice(value.toString())); + } + } + else if (type instanceof VarbinaryType) { + if (value instanceof ByteBuffer) { + type.writeSlice(output, Slices.wrappedHeapBuffer((ByteBuffer) value)); + } + else if (value instanceof SqlVarbinary sqlVarbinary) { + type.writeSlice(output, Slices.wrappedBuffer(sqlVarbinary.getBytes())); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s | value type: %s", type.getJavaType().getSimpleName(), type, value.getClass().getName())); + } + } + else if (type instanceof CharType) { + String stringValue; + if (value instanceof Utf8) { + stringValue = ((Utf8) value).toString(); + } + else if (value instanceof String) { + stringValue = (String) value; + } + else { + // Fallback: convert any other object to its string representation + stringValue = value.toString(); + } + // IMPORTANT: Char types may be padded with trailing "space" characters to make up for length if the contents are lesser than defined length. + // Need to trim out trailing spaces as Slice representing Char should not have trailing spaces + type.writeSlice(output, utf8Slice(stringValue.trim())); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s | value type: %s", type.getJavaType().getSimpleName(), type, value.getClass().getName())); + } + } + + private static void writeObject(BlockBuilder output, Type type, Object value) + { + if (type instanceof DecimalType decimalType) { + BigDecimal valueAsBigDecimal; + if (value instanceof SqlDecimal sqlDecimal) { + valueAsBigDecimal = sqlDecimal.toBigDecimal(); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Unhandled type for %s: %s | value type: %s", type.getJavaType().getSimpleName(), type, value.getClass().getName())); + } + + Object trinoNativeDecimalValue = Decimals.encodeScaledValue(valueAsBigDecimal, decimalType.getScale()); + type.writeObject(output, trinoNativeDecimalValue); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, "Unhandled type for Object: " + type.getTypeSignature()); + } + } + + private static void writeArray(ArrayBlockBuilder output, List value, ArrayType arrayType) + { + Type elementType = arrayType.getElementType(); + output.buildEntry(elementBuilder -> { + for (Object element : value) { + appendTo(elementType, element, elementBuilder); + } + }); + } + + private static void writeRow(RowBlockBuilder output, RowType rowType, GenericRecord record) + { + List fields = rowType.getFields(); + output.buildEntry(fieldBuilders -> { + for (int index = 0; index < fields.size(); index++) { + RowType.Field field = fields.get(index); + appendTo(field.getType(), record.get(field.getName().orElse("field" + index)), fieldBuilders.get(index)); + } + }); + } + + private static void writeRow(RowBlockBuilder output, RowType rowType, List list) + { + List fields = rowType.getFields(); + output.buildEntry(fieldBuilders -> { + for (int index = 0; index < fields.size(); index++) { + RowType.Field field = fields.get(index); + appendTo(field.getType(), list.get(index), fieldBuilders.get(index)); + } + }); + } + + private static void writeMap(MapBlockBuilder output, MapType mapType, Map value) + { + Type keyType = mapType.getKeyType(); + Type valueType = mapType.getValueType(); + output.buildEntry((keyBuilder, valueBuilder) -> { + for (Map.Entry entry : value.entrySet()) { + appendTo(keyType, entry.getKey(), keyBuilder); + appendTo(valueType, entry.getValue(), valueBuilder); + } + }); + } + + static class AvroDecimalConverter + { + private static final Conversions.DecimalConversion AVRO_DECIMAL_CONVERSION = new Conversions.DecimalConversion(); + + BigDecimal convert(int precision, int scale, byte[] bytes) + { + Schema schema = new Schema.Parser().parse(format("{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":%d,\"scale\":%d}", precision, scale)); + return AVRO_DECIMAL_CONVERSION.fromBytes(ByteBuffer.wrap(bytes), schema, schema.getLogicalType()); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/HudiTableTypeUtils.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/HudiTableTypeUtils.java new file mode 100644 index 0000000000000..c26dd4ca98a84 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/HudiTableTypeUtils.java @@ -0,0 +1,44 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hudi.util; + +import io.trino.plugin.hudi.HudiErrorCode; +import io.trino.spi.TrinoException; +import org.apache.hudi.common.model.HoodieTableType; + +public class HudiTableTypeUtils +{ + private static final String HUDI_PARQUET_INPUT_FORMAT = "org.apache.hudi.hadoop.HoodieParquetInputFormat"; + private static final String HUDI_PARQUET_REALTIME_INPUT_FORMAT = "org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat"; + private static final String HUDI_INPUT_FORMAT = "com.uber.hoodie.hadoop.HoodieInputFormat"; + private static final String HUDI_REALTIME_INPUT_FORMAT = "com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat"; + + private HudiTableTypeUtils() + { + } + + public static HoodieTableType fromInputFormat(String inputFormat) + { + switch (inputFormat) { + case HUDI_PARQUET_INPUT_FORMAT: + case HUDI_INPUT_FORMAT: + return HoodieTableType.COPY_ON_WRITE; + case HUDI_PARQUET_REALTIME_INPUT_FORMAT: + case HUDI_REALTIME_INPUT_FORMAT: + return HoodieTableType.MERGE_ON_READ; + default: + throw new TrinoException(HudiErrorCode.HUDI_UNSUPPORTED_TABLE_TYPE, "Table has an unsupported input format: " + inputFormat); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/SynthesizedColumnHandler.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/SynthesizedColumnHandler.java new file mode 100644 index 0000000000000..5d55adb5577c5 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/SynthesizedColumnHandler.java @@ -0,0 +1,320 @@ +/* + * 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.hudi.util; + +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hudi.HudiSplit; +import io.trino.plugin.hudi.file.HudiFile; +import io.trino.spi.TrinoException; +import io.trino.spi.block.Block; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.RunLengthEncodedBlock; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.BooleanType; +import io.trino.spi.type.DateType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.IntegerType; +import io.trino.spi.type.SqlDecimal; +import io.trino.spi.type.TimestampWithTimeZoneType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; + +import java.math.BigDecimal; +import java.time.LocalDate; +import java.time.format.DateTimeParseException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.metastore.Partitions.makePartName; +import static io.trino.plugin.hive.HiveColumnHandle.FILE_MODIFIED_TIME_COLUMN_NAME; +import static io.trino.plugin.hive.HiveColumnHandle.FILE_SIZE_COLUMN_NAME; +import static io.trino.plugin.hive.HiveColumnHandle.PARTITION_COLUMN_NAME; +import static io.trino.plugin.hive.HiveColumnHandle.PATH_COLUMN_NAME; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; +import static io.trino.spi.type.Decimals.writeBigDecimal; +import static io.trino.spi.type.Decimals.writeShortDecimal; +import static io.trino.spi.type.TimeZoneKey.UTC_KEY; +import static java.lang.Math.toIntExact; +import static java.lang.String.format; + +/** + * Handles synthesized (virtual) columns in Hudi tables, such as partition columns and metadata (not hudi metadata) + * columns. + */ +public class SynthesizedColumnHandler +{ + private final Map strategies; + private final SplitMetadata splitMetadata; + + public static SynthesizedColumnHandler create(HudiSplit hudiSplit) + { + return new SynthesizedColumnHandler(hudiSplit); + } + + /** + * Constructs a SynthesizedColumnHandler with the given partition keys. + */ + public SynthesizedColumnHandler(HudiSplit hudiSplit) + { + this.splitMetadata = SplitMetadata.of(hudiSplit); + ImmutableMap.Builder builder = ImmutableMap.builder(); + initSynthesizedColStrategies(builder); + initPartitionKeyStrategies(builder, hudiSplit); + strategies = builder.buildOrThrow(); + } + + /** + * Initializes strategies for synthesized columns. + */ + private void initSynthesizedColStrategies(ImmutableMap.Builder builder) + { + builder.put(PARTITION_COLUMN_NAME, (blockBuilder, _) -> + VarcharType.VARCHAR.writeSlice(blockBuilder, + utf8Slice(toPartitionName(splitMetadata.getPartitionKeyVals())))); + + builder.put(PATH_COLUMN_NAME, (blockBuilder, _) -> + VarcharType.VARCHAR.writeSlice(blockBuilder, utf8Slice(splitMetadata.getFilePath()))); + + builder.put(FILE_SIZE_COLUMN_NAME, (blockBuilder, _) -> + BigintType.BIGINT.writeLong(blockBuilder, splitMetadata.getFileSize())); + + builder.put(FILE_MODIFIED_TIME_COLUMN_NAME, (blockBuilder, _) -> { + long packedTimestamp = packDateTimeWithZone( + splitMetadata.getFileModificationTime(), UTC_KEY); + TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS.writeLong(blockBuilder, packedTimestamp); + }); + } + + /** + * Initializes strategies for partition columns. + */ + private void initPartitionKeyStrategies(ImmutableMap.Builder builder, + HudiSplit hudiSplit) + { + // Type is ignored here as input partitionKey.value() is always passed as a String type + for (HivePartitionKey partitionKey : hudiSplit.getPartitionKeys()) { + builder.put(partitionKey.name(), (blockBuilder, targetType) -> + appendPartitionKey(targetType, partitionKey.value(), blockBuilder)); + } + } + + /** + * Checks if a column is a synthesized column. + * + * @param columnName The column name. + * @return True if the column is synthesized, false otherwise. + */ + public boolean isSynthesizedColumn(String columnName) + { + return strategies.containsKey(columnName); + } + + /** + * Checks if a Hive column handle represents a synthesized column. + * + * @param columnHandle The Hive column handle. + * @return True if the column is synthesized, false otherwise. + */ + public boolean isSynthesizedColumn(HiveColumnHandle columnHandle) + { + return isSynthesizedColumn(columnHandle.getName()); + } + + /** + * Retrieves the strategy for a given synthesized column. + * + * @param columnHandle The Hive column handle. + * @return The corresponding column strategy, or null if not found. + */ + public SynthesizedColumnStrategy getColumnStrategy(HiveColumnHandle columnHandle) + { + return strategies.get(columnHandle.getName()); + } + + /** + * Retrieves the count of synthesized column strategies currently present. + * + * @return The number of synthesized column strategies. + */ + public int getSynthesizedColumnCount() + { + return strategies.size(); + } + + /** + * Converts partition key-value pairs into a partition name string. + * + * @param partitionKeyVals Map of partition key-value pairs. + * @return Partition name string. + */ + private static String toPartitionName(Map partitionKeyVals) + { + return makePartName(List.copyOf(partitionKeyVals.keySet()), List.copyOf(partitionKeyVals.values())); + } + + /** + * Creates a {@link Block} for the given synthesized column, typically a {@link RunLengthEncodedBlock} as the synthesized value is constant for all positions within a split. + * + * @param columnHandle The handle of the synthesized column to create a block for. + * @param positionCount The number of positions (rows) the resulting block should represent. + * @return A {@link Block} containing the synthesized values. + */ + public Block createRleSynthesizedBlock(HiveColumnHandle columnHandle, int positionCount) + { + Type columnType = columnHandle.getType(); + + if (positionCount == 0) { + return columnType.createBlockBuilder(null, 0).build(); + } + + SynthesizedColumnStrategy strategy = getColumnStrategy(columnHandle); + + // Because this builder will only hold the single constant value + int expectedEntriesForValueBlock = 1; + BlockBuilder valueBuilder = columnType.createBlockBuilder(null, expectedEntriesForValueBlock); + + if (strategy == null) { + valueBuilder.appendNull(); + } + else { + // Apply the strategy to write the single value into the builder + strategy.appendToBlock(valueBuilder, columnType); + } + Block valueBlock = valueBuilder.build(); + + return RunLengthEncodedBlock.create(valueBlock, positionCount); + } + + /** + * Represents metadata about split being processed. + * Splits are assumed to be in the same partition. + */ + public static class SplitMetadata + { + private final Map partitionKeyVals; + private final String filePath; + private final long fileSize; + private final long modifiedTime; + + /** + * Creates SplitMetadata from a Hudi split and partition key list. + */ + public static SplitMetadata of(HudiSplit hudiSplit) + { + return new SplitMetadata(hudiSplit); + } + + public SplitMetadata(HudiSplit hudiSplit) + { + this.partitionKeyVals = hudiSplit.getPartitionKeys().stream() + .collect(Collectors.toMap(HivePartitionKey::name, HivePartitionKey::value)); + // Parquet files will be prioritised over log files + HudiFile hudiFile = hudiSplit.getBaseFile().isPresent() + ? hudiSplit.getBaseFile().get() + : hudiSplit.getLogFiles().getFirst(); + this.filePath = hudiFile.getPath(); + this.fileSize = hudiFile.getFileSize(); + this.modifiedTime = hudiFile.getModificationTime(); + } + + public Map getPartitionKeyVals() + { + return partitionKeyVals; + } + + public String getFilePath() + { + return filePath; + } + + public long getFileSize() + { + return fileSize; + } + + public long getFileModificationTime() + { + return modifiedTime; + } + } + + /** + * Helper function to prefill BlockBuilders with values from PartitionKeys which are in the String type. + * This function handles the casting of String type the actual column type. + */ + private static void appendPartitionKey(Type targetType, String value, BlockBuilder blockBuilder) + { + if (value == null) { + blockBuilder.appendNull(); + return; + } + + if (targetType instanceof VarcharType varcharType) { + varcharType.writeSlice(blockBuilder, utf8Slice(value)); + } + else if (targetType instanceof IntegerType integerType) { + integerType.writeInt(blockBuilder, Integer.parseInt(value)); + } + else if (targetType instanceof BigintType bigintType) { + bigintType.writeLong(blockBuilder, Long.parseLong(value)); + } + else if (targetType instanceof BooleanType booleanType) { + booleanType.writeBoolean(blockBuilder, Boolean.parseBoolean(value)); + } + else if (targetType instanceof DecimalType decimalType) { + SqlDecimal sqlDecimal = SqlDecimal.decimal(value, decimalType); + BigDecimal bigDecimal = sqlDecimal.toBigDecimal(); + + if (decimalType.isShort()) { + // For short decimals, get the unscaled long value + // SqlDecimal.toBigDecimal() is used for consistency with the original SqlDecimal path + // The unscaled value of a Trino short decimal (precision <= 18) fits in a long + writeShortDecimal(blockBuilder, bigDecimal.unscaledValue().longValue()); + } + else { + // For long decimals, use the BigDecimal representation obtained from SqlDecimal. + writeBigDecimal(decimalType, blockBuilder, bigDecimal); + } + } + else if (targetType instanceof DateType dateType) { + try { + // Parse the date string using "YYYY-MM-DD" format + LocalDate localDate = LocalDate.parse(value); + // Convert LocalDate to days since epoch where LocalDate#toEpochDay() returns a long + int daysSinceEpoch = toIntExact(localDate.toEpochDay()); + dateType.writeInt(blockBuilder, daysSinceEpoch); + } + catch (DateTimeParseException e) { + // Handle parsing error + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Invalid date string format for DATE type: '%s'. Expected format like YYYY-MM-DD. Details: %s", + value, e.getMessage()), e); + } + catch (ArithmeticException e) { + // Handle potential overflow if toEpochDay() result is outside int range + throw new TrinoException(GENERIC_INTERNAL_ERROR, + format("Date string '%s' results in a day count out of integer range for DATE type. Details: %s", + value, e.getMessage()), e); + } + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, format("Unknown target type '%s' for column '%s'", targetType, value)); + } + } +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/SynthesizedColumnStrategy.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/SynthesizedColumnStrategy.java new file mode 100644 index 0000000000000..916e6cb83e4fd --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/SynthesizedColumnStrategy.java @@ -0,0 +1,25 @@ +/* + * 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.hudi.util; + +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.type.Type; + +/** + * Strategy interface for handling different types of synthesized columns + */ +public interface SynthesizedColumnStrategy +{ + void appendToBlock(BlockBuilder blockBuilder, Type type); +} diff --git a/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/TupleDomainUtils.java b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/TupleDomainUtils.java new file mode 100644 index 0000000000000..4cc8091c54b93 --- /dev/null +++ b/hudi-trino-plugin/src/main/java/io/trino/plugin/hudi/util/TupleDomainUtils.java @@ -0,0 +1,162 @@ +/* + * 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.hudi.util; + +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.TupleDomain; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +public class TupleDomainUtils +{ + // Utility classes should not have a public or default constructor. + private TupleDomainUtils() {} + + /** + * Get all columns that are referenced in the provided tupleDomain predicates. + */ + public static List getReferencedColumns(TupleDomain tupleDomain) + { + if (tupleDomain.getDomains().isEmpty()) { + return List.of(); + } + return tupleDomain.getDomains().get().keySet().stream().toList(); + } + + /** + * Check if all of the provided source fields are referenced in the tupleDomain predicates. + */ + public static boolean areAllFieldsReferenced(TupleDomain tupleDomain, List sourceFields) + { + Set referenceColSet = new HashSet<>(TupleDomainUtils.getReferencedColumns(tupleDomain)); + Set sourceFieldSet = new HashSet<>(sourceFields); + + return referenceColSet.containsAll(sourceFieldSet); + } + + /** + * Check if at least one of the provided source field is referenced in the tupleDomain predicates. + */ + public static boolean areSomeFieldsReferenced(TupleDomain tupleDomain, List sourceFields) + { + Set referenceColSet = new HashSet<>(TupleDomainUtils.getReferencedColumns(tupleDomain)); + for (String sourceField : sourceFields) { + if (referenceColSet.contains(sourceField)) { + return true; + } + } + return false; + } + + /** + * Check all columns referencing sourceFields are either IN or EQUAL predicates. + */ + public static boolean areDomainsInOrEqualOnly(TupleDomain tupleDomain, List sourceFields) + { + // If no recordKeys or no recordKeyDomains, return empty list + if (sourceFields == null || sourceFields.isEmpty() || tupleDomain.isAll() || tupleDomain.isNone()) { + return false; + } + + Optional> domainsOpt = tupleDomain.getDomains(); + // Not really necessary, as tupleDomain.isNone() already checks for this + if (domainsOpt.isEmpty()) { + return false; + } + + boolean areReferencedInOrEqual = true; + for (String sourceField : sourceFields) { + Domain domain = domainsOpt.get().get(sourceField); + // For cases where sourceField does not exist in tupleDomain + if (domain == null) { + return false; + } + areReferencedInOrEqual &= (domain.isSingleValue() || domain.getValues().isDiscreteSet()); + } + return areReferencedInOrEqual; + } + + /** + * Checks if a specific Domain represents ONLY an 'IS NULL' constraint. + * This means null is allowed, and no other non-null values are allowed. + * Important: Not handling `= NULL` predicates as colA `= NULL` does not evaluate to TRUE or FALSE, it evaluates to UNKNOWN, which is treated as false. + * + * @param domain The Domain to check. + * @return true if the domain represents 'IS NULL', false otherwise. + */ + private static boolean isOnlyNullConstraint(Domain domain) + { + // Null must be allowed, and the ValueSet must allow *no* non-null values. + return domain.isNullAllowed() && domain.getValues().isNone(); + } + + /** + * Checks if a specific Domain represents ONLY an 'IS NOT NULL' constraint. + * This means null is not allowed, and all non-null values are allowed (no other range/value restrictions). + * Important: Not handling `!= NULL` or `<> NULL` predicates as this does not evaluate to TRUE or FALSE, it evaluates to UNKNOWN, which is treated as false. + * + * @param domain The Domain to check. + * @return true if the domain represents 'IS NOT NULL', false otherwise. + */ + private static boolean isOnlyNotNullConstraint(Domain domain) + { + // Null must *NOT* be allowed, and the ValueSet must allow *ALL* possible non-null values. + return !domain.isNullAllowed() && domain.getValues().isAll(); + } + + /** + * Overloaded function to test if a Domain contains null checks or not. + * + * @param domain The Domain to check. + * @return true if the domain represents 'IS NOT NULL' or 'IS NULL', false otherwise. + */ + public static boolean hasSimpleNullCheck(Domain domain) + { + return isOnlyNullConstraint(domain) || isOnlyNotNullConstraint(domain); + } + + /** + * Checks if a TupleDomain contains at least one column Domain that represents + * exclusively an 'IS NULL' or 'IS NOT NULL' constraint. + * + * @param tupleDomain The TupleDomain to inspect. + * @return true if a simple null check constraint exists, false otherwise. + */ + public static boolean hasSimpleNullCheck(TupleDomain tupleDomain) + { + // A 'None' TupleDomain implies contradiction, not a simple null check + if (tupleDomain.isNone()) { + return false; + } + Optional> domains = tupleDomain.getDomains(); + // An 'All' TupleDomain has no constraints + if (domains.isEmpty()) { + return false; + } + + // Iterate through the domains for each column in the TupleDomain + for (Domain domain : domains.get().values()) { + if (hasSimpleNullCheck(domain)) { + // Found a domain that is purely an IS NULL or IS NOT NULL check + return true; + } + } + // No domain matched the simple null check patterns + return false; + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/BaseHudiConnectorSmokeTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/BaseHudiConnectorSmokeTest.java new file mode 100644 index 0000000000000..c1d23e7284270 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/BaseHudiConnectorSmokeTest.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.hudi; + +import io.trino.testing.BaseConnectorSmokeTest; +import io.trino.testing.TestingConnectorBehavior; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +public abstract class BaseHudiConnectorSmokeTest + extends BaseConnectorSmokeTest +{ + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + return switch (connectorBehavior) { + case SUPPORTS_INSERT, + SUPPORTS_DELETE, + SUPPORTS_UPDATE, + SUPPORTS_MERGE, + SUPPORTS_CREATE_SCHEMA, + SUPPORTS_CREATE_TABLE, + SUPPORTS_RENAME_TABLE, + SUPPORTS_CREATE_VIEW, + SUPPORTS_CREATE_MATERIALIZED_VIEW, + SUPPORTS_COMMENT_ON_COLUMN -> false; + default -> super.hasBehavior(connectorBehavior); + }; + } + + @Test + @Override + public void testShowCreateTable() + { + // Override because Hudi connector contains 'location' table property + String schema = getSession().getSchema().orElseThrow(); + assertThat((String) computeScalar("SHOW CREATE TABLE region")) + .matches("\\QCREATE TABLE hudi." + schema + ".region (\n" + + " regionkey bigint,\n" + + " name varchar(25),\n" + + " comment varchar(152)\n" + + ")\n" + + "WITH (\n" + + " location = \\E'.*/region'\n\\Q" + + ")"); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java new file mode 100644 index 0000000000000..c13e2b2827276 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java @@ -0,0 +1,170 @@ +/* + * 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.hudi; + +import com.google.errorprone.annotations.CanIgnoreReturnValue; +import io.airlift.log.Level; +import io.airlift.log.Logger; +import io.airlift.log.Logging; +import io.trino.filesystem.Location; +import io.trino.metastore.Database; +import io.trino.metastore.HiveMetastoreFactory; +import io.trino.plugin.base.util.Closables; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; +import io.trino.plugin.hudi.testing.HudiTablesInitializer; +import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; +import io.trino.plugin.hudi.testing.TpchHudiTablesInitializer; +import io.trino.spi.security.PrincipalType; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.tpch.TpchTable; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static io.trino.testing.TestingSession.testSessionBuilder; +import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY; +import static io.trino.testing.containers.Minio.MINIO_REGION; +import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; +import static java.util.Objects.requireNonNull; + +public final class HudiQueryRunner +{ + private HudiQueryRunner() {} + + static { + Logging logging = Logging.initialize(); + logging.setLevel("org.apache.hudi", Level.OFF); + } + + private static final String SCHEMA_NAME = "tests"; + + public static Builder builder() + { + return new Builder("local:///"); + } + + public static Builder builder(Hive3MinioDataLake hiveMinioDataLake) + { + return new Builder("s3://" + hiveMinioDataLake.getBucketName() + "/") + .addConnectorProperty("fs.native-s3.enabled", "true") + .addConnectorProperty("s3.aws-access-key", MINIO_ACCESS_KEY) + .addConnectorProperty("s3.aws-secret-key", MINIO_SECRET_KEY) + .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 String schemaLocation; + private HudiTablesInitializer dataLoader; + private final Map connectorProperties = new HashMap<>(); + + protected Builder(String schemaLocation) + { + super(testSessionBuilder() + .setCatalog("hudi") + .setSchema(SCHEMA_NAME) + .build()); + this.schemaLocation = requireNonNull(schemaLocation, "schemaLocation is null"); + } + + @CanIgnoreReturnValue + public Builder setDataLoader(HudiTablesInitializer 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 + { + DistributedQueryRunner queryRunner = super.build(); + try { + queryRunner.installPlugin(new TestingHudiPlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hudi_data"))); + queryRunner.createCatalog("hudi", "hudi", connectorProperties); + + // Hudi connector does not support creating schema or any other write operations + ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()) + .createDatabase(Database.builder() + .setDatabaseName(SCHEMA_NAME) + .setOwnerName(Optional.of("public")) + .setOwnerType(Optional.of(PrincipalType.ROLE)) + .build()); + + dataLoader.initializeTables(queryRunner, Location.of(schemaLocation), SCHEMA_NAME); + return queryRunner; + } + catch (Throwable e) { + Closables.closeAllSuppress(e, queryRunner); + throw e; + } + } + } + + public static final class DefaultHudiQueryRunnerMain + { + private DefaultHudiQueryRunnerMain() {} + + public static void main(String[] args) + throws Exception + { + Logging.initialize(); + Logger log = Logger.get(DefaultHudiQueryRunnerMain.class); + + QueryRunner queryRunner = builder() + .addCoordinatorProperty("http-server.http.port", "8080") + .setDataLoader(new ResourceHudiTablesInitializer()) + .build(); + + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } + } + + public static final class HudiMinioQueryRunnerMain + { + private HudiMinioQueryRunnerMain() {} + + public static void main(String[] args) + throws Exception + { + Logging.initialize(); + Logger log = Logger.get(HudiMinioQueryRunnerMain.class); + + Hive3MinioDataLake hiveMinioDataLake = new Hive3MinioDataLake("test-bucket"); + hiveMinioDataLake.start(); + QueryRunner queryRunner = builder(hiveMinioDataLake) + .addCoordinatorProperty("http-server.http.port", "8080") + .setDataLoader(new TpchHudiTablesInitializer(TpchTable.getTables())) + .build(); + + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/SessionBuilder.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/SessionBuilder.java new file mode 100644 index 0000000000000..6a7d80e98e399 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/SessionBuilder.java @@ -0,0 +1,137 @@ +/* + * 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.hudi; + +import io.trino.Session; + +import static io.trino.SystemSessionProperties.ENABLE_DYNAMIC_FILTERING; +import static io.trino.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; +import static io.trino.plugin.hudi.HudiSessionProperties.COLUMN_STATS_INDEX_ENABLED; +import static io.trino.plugin.hudi.HudiSessionProperties.COLUMN_STATS_WAIT_TIMEOUT; +import static io.trino.plugin.hudi.HudiSessionProperties.DYNAMIC_FILTERING_WAIT_TIMEOUT; +import static io.trino.plugin.hudi.HudiSessionProperties.METADATA_TABLE_ENABLED; +import static io.trino.plugin.hudi.HudiSessionProperties.PARTITION_STATS_INDEX_ENABLED; +import static io.trino.plugin.hudi.HudiSessionProperties.QUERY_PARTITION_FILTER_REQUIRED; +import static io.trino.plugin.hudi.HudiSessionProperties.RECORD_LEVEL_INDEX_ENABLED; +import static io.trino.plugin.hudi.HudiSessionProperties.SECONDARY_INDEX_ENABLED; +import static io.trino.plugin.hudi.HudiSessionProperties.SECONDARY_INDEX_WAIT_TIMEOUT; +import static io.trino.plugin.hudi.HudiSessionProperties.TABLE_STATISTICS_ENABLED; +import static java.util.Objects.requireNonNull; + +public class SessionBuilder +{ + private final Session.SessionBuilder sessionBuilder; + private final String catalogName; + + private SessionBuilder(Session session) + { + requireNonNull(session, "Initial session cannot be null"); + this.sessionBuilder = Session.builder(session); + this.catalogName = session.getCatalog() + .orElseThrow(() -> new IllegalStateException("Session must have a catalog to configure properties.")); + } + + /** + * Creates a new SessionPropertyConfigurator from an existing session. + * + * @param session The base session to build upon. + * @return A new instance of SessionPropertyConfigurator. + */ + public static SessionBuilder from(Session session) + { + return new SessionBuilder(session); + } + + private SessionBuilder setCatalogProperty(String propertyName, String propertyValue) + { + this.sessionBuilder.setCatalogSessionProperty(catalogName, propertyName, propertyValue); + return this; + } + + private SessionBuilder setSystemProperty(String propertyName, String propertyValue) + { + this.sessionBuilder.setSystemProperty(propertyName, propertyValue); + return this; + } + + /** + * Builds the new Session with the configured properties. + * + * @return The newly configured Session object. + */ + public Session build() + { + return this.sessionBuilder.build(); + } + + public SessionBuilder withJoinDistributionType(String joinDistributionType) + { + return setSystemProperty(JOIN_DISTRIBUTION_TYPE, joinDistributionType); + } + + public SessionBuilder withPartitionFilterRequired(boolean required) + { + return setCatalogProperty(QUERY_PARTITION_FILTER_REQUIRED, String.valueOf(required)); + } + + public SessionBuilder withTableStatisticsEnabled(boolean enabled) + { + return setCatalogProperty(TABLE_STATISTICS_ENABLED, String.valueOf(enabled)); + } + + public SessionBuilder withMdtEnabled(boolean enabled) + { + return setCatalogProperty(METADATA_TABLE_ENABLED, String.valueOf(enabled)); + } + + public SessionBuilder withDynamicFilterEnabled(boolean isDynamicFilterEnabled) + { + return setSystemProperty(ENABLE_DYNAMIC_FILTERING, String.valueOf(isDynamicFilterEnabled)); + } + + public SessionBuilder withDynamicFilterTimeout(String durationProp) + { + return setCatalogProperty(DYNAMIC_FILTERING_WAIT_TIMEOUT, durationProp); + } + + public SessionBuilder withColStatsIndexEnabled(boolean enabled) + { + return setCatalogProperty(COLUMN_STATS_INDEX_ENABLED, String.valueOf(enabled)); + } + + public SessionBuilder withRecordLevelIndexEnabled(boolean enabled) + { + return setCatalogProperty(RECORD_LEVEL_INDEX_ENABLED, String.valueOf(enabled)); + } + + public SessionBuilder withSecondaryIndexEnabled(boolean enabled) + { + return setCatalogProperty(SECONDARY_INDEX_ENABLED, String.valueOf(enabled)); + } + + public SessionBuilder withPartitionStatsIndexEnabled(boolean enabled) + { + return setCatalogProperty(PARTITION_STATS_INDEX_ENABLED, String.valueOf(enabled)); + } + + public SessionBuilder withColumnStatsTimeout(String durationProp) + { + return setCatalogProperty(COLUMN_STATS_WAIT_TIMEOUT, durationProp); + } + + public SessionBuilder withSecondaryIndexTimeout(String durationProp) + { + return setCatalogProperty(SECONDARY_INDEX_WAIT_TIMEOUT, durationProp); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java new file mode 100644 index 0000000000000..5e07fedd920b1 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java @@ -0,0 +1,122 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static io.airlift.units.DataSize.Unit.MEGABYTE; + +public class TestHudiConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(HudiConfig.class) + .setColumnsToHide(ImmutableList.of()) + .setTableStatisticsEnabled(true) + .setMetadataEnabled(false) + .setUseParquetColumnNames(true) + .setUseParquetColumnIndex(false) + .setTableStatisticsExecutorParallelism(4) + .setSizeBasedSplitWeightsEnabled(true) + .setStandardSplitWeightSize(DataSize.of(128, MEGABYTE)) + .setMinimumAssignedSplitWeight(0.05) + .setTargetSplitSize(DataSize.of(128, MEGABYTE)) + .setMaxSplitsPerSecond(Integer.MAX_VALUE) + .setMaxOutstandingSplits(1000) + .setSplitLoaderParallelism(4) + .setSplitGeneratorParallelism(4) + .setPerTransactionMetastoreCacheMaximumSize(2000) + .setQueryPartitionFilterRequired(false) + .setIgnoreAbsentPartitions(false) + .setRecordLevelIndexEnabled(true) + .setSecondaryIndexEnabled(true) + .setColumnStatsIndexEnabled(true) + .setPartitionStatsIndexEnabled(true) + .setDynamicFilteringWaitTimeout(Duration.valueOf("1s")) + .setColumnStatsWaitTimeout(Duration.valueOf("1s")) + .setRecordIndexWaitTimeout(Duration.valueOf("2s")) + .setSecondaryIndexWaitTimeout(Duration.valueOf("2s"))); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = ImmutableMap.builder() + .put("hudi.columns-to-hide", "_hoodie_record_key") + .put("hudi.table-statistics-enabled", "false") + .put("hudi.metadata-enabled", "true") + .put("hudi.parquet.use-column-names", "false") + .put("hudi.parquet.use-column-index", "true") + .put("hudi.table-statistics-executor-parallelism", "16") + .put("hudi.size-based-split-weights-enabled", "false") + .put("hudi.standard-split-weight-size", "64MB") + .put("hudi.minimum-assigned-split-weight", "0.1") + .put("hudi.target-split-size", "32MB") + .put("hudi.max-splits-per-second", "100") + .put("hudi.max-outstanding-splits", "100") + .put("hudi.split-loader-parallelism", "16") + .put("hudi.split-generator-parallelism", "32") + .put("hudi.per-transaction-metastore-cache-maximum-size", "1000") + .put("hudi.query-partition-filter-required", "true") + .put("hudi.ignore-absent-partitions", "true") + .put("hudi.index.record-level-index-enabled", "false") + .put("hudi.index.secondary-index-enabled", "false") + .put("hudi.index.column-stats-index-enabled", "false") + .put("hudi.index.partition-stats-index-enabled", "false") + .put("hudi.dynamic-filtering.wait-timeout", "2s") + .put("hudi.index.column-stats.wait-timeout", "2s") + .put("hudi.index.record-index.wait-timeout", "1s") + .put("hudi.index.secondary-index.wait-timeout", "1s") + .buildOrThrow(); + + HudiConfig expected = new HudiConfig() + .setColumnsToHide(ImmutableList.of("_hoodie_record_key")) + .setTableStatisticsEnabled(false) + .setMetadataEnabled(true) + .setUseParquetColumnNames(false) + .setUseParquetColumnIndex(true) + .setTableStatisticsExecutorParallelism(16) + .setSizeBasedSplitWeightsEnabled(false) + .setStandardSplitWeightSize(DataSize.of(64, MEGABYTE)) + .setMinimumAssignedSplitWeight(0.1) + .setTargetSplitSize(DataSize.of(32, MEGABYTE)) + .setMaxSplitsPerSecond(100) + .setMaxOutstandingSplits(100) + .setSplitLoaderParallelism(16) + .setSplitGeneratorParallelism(32) + .setPerTransactionMetastoreCacheMaximumSize(1000) + .setQueryPartitionFilterRequired(true) + .setIgnoreAbsentPartitions(true) + .setRecordLevelIndexEnabled(false) + .setSecondaryIndexEnabled(false) + .setColumnStatsIndexEnabled(false) + .setPartitionStatsIndexEnabled(false) + .setDynamicFilteringWaitTimeout(Duration.valueOf("2s")) + .setColumnStatsWaitTimeout(Duration.valueOf("2s")) + .setRecordIndexWaitTimeout(Duration.valueOf("1s")) + .setSecondaryIndexWaitTimeout(Duration.valueOf("1s")); + + assertFullMapping(properties, expected); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java new file mode 100644 index 0000000000000..2e1095a535a20 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java @@ -0,0 +1,70 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.testing.TestingConnectorContext; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static io.trino.spi.transaction.IsolationLevel.READ_UNCOMMITTED; +import static io.trino.testing.TestingConnectorSession.SESSION; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestHudiConnectorFactory +{ + @Test + public void testCreateConnector() + { + assertCreateConnector("thrift://localhost:1234"); + assertCreateConnector("thrift://localhost:1234,thrift://192.0.2.3:5678"); + + assertCreateConnectorFails("abc", "metastoreUri scheme is missing: abc"); + assertCreateConnectorFails("thrift://:8090", "metastoreUri host is missing: thrift://:8090"); + assertCreateConnectorFails("thrift://localhost", "metastoreUri port is missing: thrift://localhost"); + assertCreateConnectorFails("abc::", "metastoreUri scheme must be thrift: abc::"); + assertCreateConnectorFails("", "metastoreUris must specify at least one URI"); + assertCreateConnectorFails("thrift://localhost:1234,thrift://test-1", "metastoreUri port is missing: thrift://test-1"); + } + + private static void assertCreateConnector(String metastoreUri) + { + Map config = ImmutableMap.of( + "hive.metastore.uri", metastoreUri, + "bootstrap.quiet", "true"); + + ConnectorFactory factory = new HudiConnectorFactory(); + Connector connector = factory.create("test", config, new TestingConnectorContext()); + ConnectorTransactionHandle transaction = connector.beginTransaction(READ_UNCOMMITTED, true, true); + assertThat(connector.getMetadata(SESSION, transaction)).isInstanceOf(ClassLoaderSafeConnectorMetadata.class); + assertThat(connector.getSplitManager()).isInstanceOf(ClassLoaderSafeConnectorSplitManager.class); + assertThat(connector.getPageSourceProvider()).isInstanceOf(ConnectorPageSourceProvider.class); + connector.commit(transaction); + } + + private static void assertCreateConnectorFails(String metastoreUri, String exceptionString) + { + assertThatThrownBy(() -> assertCreateConnector(metastoreUri)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining(exceptionString); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorParquetColumnNamesTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorParquetColumnNamesTest.java new file mode 100644 index 0000000000000..7e938ba3aed1e --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorParquetColumnNamesTest.java @@ -0,0 +1,31 @@ +/* + * 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.hudi; + +import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; +import io.trino.testing.QueryRunner; + +public class TestHudiConnectorParquetColumnNamesTest + extends TestHudiSmokeTest +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return HudiQueryRunner.builder() + .addConnectorProperty("hudi.parquet.use-column-names", "false") + .setDataLoader(new ResourceHudiTablesInitializer()) + .build(); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorTest.java new file mode 100644 index 0000000000000..84a4cbd604402 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiConnectorTest.java @@ -0,0 +1,90 @@ +/* + * 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.hudi; + +import io.trino.plugin.hudi.testing.TpchHudiTablesInitializer; +import io.trino.testing.BaseConnectorTest; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.hudi.testing.HudiTestUtils.COLUMNS_TO_HIDE; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestHudiConnectorTest + extends BaseConnectorTest +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return HudiQueryRunner.builder() + .addConnectorProperty("hudi.columns-to-hide", COLUMNS_TO_HIDE) + .setDataLoader(new TpchHudiTablesInitializer(REQUIRED_TPCH_TABLES)) + .build(); + } + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + return switch (connectorBehavior) { + case SUPPORTS_ADD_COLUMN, + SUPPORTS_COMMENT_ON_COLUMN, + SUPPORTS_COMMENT_ON_TABLE, + SUPPORTS_CREATE_MATERIALIZED_VIEW, + SUPPORTS_CREATE_SCHEMA, + SUPPORTS_CREATE_TABLE, + SUPPORTS_CREATE_VIEW, + SUPPORTS_DELETE, + SUPPORTS_DEREFERENCE_PUSHDOWN, + SUPPORTS_INSERT, + SUPPORTS_MERGE, + SUPPORTS_RENAME_COLUMN, + SUPPORTS_RENAME_TABLE, + SUPPORTS_SET_COLUMN_TYPE, + SUPPORTS_TOPN_PUSHDOWN, + SUPPORTS_UPDATE -> false; + default -> super.hasBehavior(connectorBehavior); + }; + } + + @Test + @Override + public void testShowCreateTable() + { + String schema = getSession().getSchema().orElseThrow(); + assertThat((String) computeScalar("SHOW CREATE TABLE orders")) + .matches("\\QCREATE TABLE hudi." + schema + ".orders (\n" + + " orderkey bigint,\n" + + " custkey bigint,\n" + + " orderstatus varchar(1),\n" + + " totalprice double,\n" + + " orderdate date,\n" + + " orderpriority varchar(15),\n" + + " clerk varchar(15),\n" + + " shippriority integer,\n" + + " comment varchar(79)\n" + + ")\n" + + "WITH (\n" + + " location = \\E'.*/orders'\n\\Q" + + ")"); + } + + @Test + public void testHideHiveSysSchema() + { + assertThat(computeActual("SHOW SCHEMAS").getOnlyColumnAsSet()).doesNotContain("sys"); + assertQueryFails("SHOW TABLES IN hudi.sys", ".*Schema 'sys' does not exist"); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiMinioConnectorSmokeTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiMinioConnectorSmokeTest.java new file mode 100644 index 0000000000000..5fa1332073cdf --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiMinioConnectorSmokeTest.java @@ -0,0 +1,41 @@ +/* + * 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.hudi; + +import io.trino.plugin.hive.containers.Hive3MinioDataLake; +import io.trino.plugin.hudi.testing.TpchHudiTablesInitializer; +import io.trino.testing.QueryRunner; + +import static io.trino.plugin.hive.containers.HiveHadoop.HIVE3_IMAGE; +import static io.trino.plugin.hudi.testing.HudiTestUtils.COLUMNS_TO_HIDE; +import static io.trino.testing.TestingNames.randomNameSuffix; + +public class TestHudiMinioConnectorSmokeTest + extends BaseHudiConnectorSmokeTest +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + String bucketName = "test-hudi-connector-" + randomNameSuffix(); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName, HIVE3_IMAGE)); + hiveMinioDataLake.start(); + hiveMinioDataLake.getMinioClient().ensureBucketExists(bucketName); + + return HudiQueryRunner.builder(hiveMinioDataLake) + .addConnectorProperty("hudi.columns-to-hide", COLUMNS_TO_HIDE) + .setDataLoader(new TpchHudiTablesInitializer(REQUIRED_TPCH_TABLES)) + .build(); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPageSource.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPageSource.java new file mode 100644 index 0000000000000..8bb46f8fb3f5a --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPageSource.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.hudi; + +import io.trino.spi.connector.ConnectorPageSource; +import org.junit.jupiter.api.Test; + +import static io.trino.spi.testing.InterfaceTestUtils.assertAllMethodsOverridden; + +public class TestHudiPageSource +{ + @Test + public void testEverythingImplemented() + { + assertAllMethodsOverridden(ConnectorPageSource.class, HudiPageSource.class); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPageSourceProviderTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPageSourceProviderTest.java new file mode 100644 index 0000000000000..8966553939742 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPageSourceProviderTest.java @@ -0,0 +1,242 @@ +/* + * 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.hudi; + +import io.trino.metastore.HiveType; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.Type; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Types; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Optional; + +import static io.trino.plugin.hudi.HudiPageSourceProvider.remapColumnIndicesToPhysical; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +class TestHudiPageSourceProviderTest +{ + @Test + public void testRemapSimpleMatchCaseInsensitive() + { + // Physical Schema: [col_a (int), col_b (string)] + MessageType fileSchema = new MessageType("file_schema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, OPTIONAL).named("col_a"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, OPTIONAL).as(LogicalTypeAnnotation.stringType()).named("col_b")); + + // Requested Columns (same order, different case) + List requestedColumns = List.of( + createDummyHandle("COL_A", 0, HiveType.HIVE_INT, INTEGER), + createDummyHandle("COL_B", 1, HiveType.HIVE_STRING, VARCHAR)); + + // Perform remapping (case-insensitive) + List remapped = remapColumnIndicesToPhysical(fileSchema, requestedColumns, false); + + assertThat(remapped).hasSize(2); + // First requested column "COL_A" should map to physical index 0 + assertHandle(remapped.get(0), "COL_A", 0, HiveType.HIVE_INT, INTEGER); + // Second requested column "COL_B" should map to physical index 1 + assertHandle(remapped.get(1), "COL_B", 1, HiveType.HIVE_STRING, VARCHAR); + } + + @Test + public void testRemapSimpleMatchCaseSensitive() + { + // Physical Schema: [col_a (int), Col_B (string)] - Note the case difference + MessageType fileSchema = new MessageType("file_schema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, OPTIONAL).named("col_a"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, OPTIONAL).as(LogicalTypeAnnotation.stringType()).named("Col_B")); + + // Requested Columns (matching case) + List requestedColumns = List.of( + createDummyHandle("col_a", 0, HiveType.HIVE_INT, INTEGER), + createDummyHandle("Col_B", 1, HiveType.HIVE_STRING, VARCHAR)); + + // Perform remapping (case-sensitive) + List remapped = remapColumnIndicesToPhysical(fileSchema, requestedColumns, true); + + assertThat(remapped).hasSize(2); + assertHandle(remapped.get(0), "col_a", 0, HiveType.HIVE_INT, INTEGER); + assertHandle(remapped.get(1), "Col_B", 1, HiveType.HIVE_STRING, VARCHAR); + } + + @Test + public void testRemapCaseSensitiveMismatch() + { + // Physical Schema: [col_a (int), col_b (string)] + MessageType fileSchema = new MessageType("file_schema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, OPTIONAL).named("col_a"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, OPTIONAL).as(LogicalTypeAnnotation.stringType()).named("col_b")); + + // Requested Columns (different case) + List requestedColumns = List.of( + createDummyHandle("COL_A", 0, HiveType.HIVE_INT, INTEGER), // This will mismatch + createDummyHandle("col_b", 1, HiveType.HIVE_STRING, VARCHAR)); + + // Perform remapping (case-sensitive) - Expect NPE because "COL_A" won't be found + assertThatThrownBy(() -> remapColumnIndicesToPhysical(fileSchema, requestedColumns, true)) + .isInstanceOf(NullPointerException.class); // Check the exception type + } + + @Test + public void testRemapDifferentOrder() + { + // Physical Schema: [id (int), name (string), timestamp (long)] + MessageType fileSchema = new MessageType("file_schema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, OPTIONAL).named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, OPTIONAL).as(LogicalTypeAnnotation.stringType()).named("name"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, OPTIONAL).named("timestamp")); + + // Requested Columns (different order) + List requestedColumns = List.of( + // Original index irrelevant + createDummyHandle("name", 99, HiveType.HIVE_STRING, VARCHAR), + createDummyHandle("timestamp", 5, HiveType.HIVE_LONG, BigintType.BIGINT), + createDummyHandle("id", 0, HiveType.HIVE_INT, INTEGER)); + + // Perform remapping (case-insensitive) + List remapped = remapColumnIndicesToPhysical(fileSchema, requestedColumns, false); + + assertThat(remapped).hasSize(3); + // First requested "name" -> physical index 1 + assertHandle(remapped.get(0), "name", 1, HiveType.HIVE_STRING, VARCHAR); + // Second requested "timestamp" -> physical index 2 + assertHandle(remapped.get(1), "timestamp", 2, HiveType.HIVE_LONG, BigintType.BIGINT); + // Third requested "id" -> physical index 0 + assertHandle(remapped.get(2), "id", 0, HiveType.HIVE_INT, INTEGER); + } + + @Test + public void testRemapSubset() + { + // Physical Schema: [col_a, col_b, col_c, col_d] + MessageType fileSchema = new MessageType("file_schema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, OPTIONAL).named("col_a"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, OPTIONAL).as(LogicalTypeAnnotation.stringType()).named("col_b"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, OPTIONAL).named("col_c"), + Types.primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, OPTIONAL).named("col_d")); + + // Requested Columns (subset and different order) + List requestedColumns = List.of( + createDummyHandle("col_d", 1, HiveType.HIVE_DOUBLE, DOUBLE), + createDummyHandle("col_a", 0, HiveType.HIVE_INT, INTEGER)); + + // Perform remapping (case-insensitive) + List remapped = remapColumnIndicesToPhysical(fileSchema, requestedColumns, false); + + assertThat(remapped).hasSize(2); + // First requested "col_d" -> physical index 3 + assertHandle(remapped.get(0), "col_d", 3, HiveType.HIVE_DOUBLE, DOUBLE); + // Second requested "col_a" -> physical index 0 + assertHandle(remapped.get(1), "col_a", 0, HiveType.HIVE_INT, INTEGER); + } + + @Test + public void testRemapEmptyRequested() + { + // Physical Schema: [col_a, col_b] + MessageType fileSchema = new MessageType("file_schema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, OPTIONAL).named("col_a"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, OPTIONAL).as(LogicalTypeAnnotation.stringType()).named("col_b")); + + // Requested Columns (empty list) + List requestedColumns = List.of(); + + // Perform remapping + List remapped = remapColumnIndicesToPhysical(fileSchema, requestedColumns, false); + + assertThat(remapped).isEmpty(); + } + + @Test + public void testRemapColumnNotFound() + { + // Physical Schema: [col_a] + MessageType fileSchema = new MessageType("file_schema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, OPTIONAL).named("col_a")); + + // Requested Columns (includes a non-existent column) + List requestedColumns = List.of( + createDummyHandle("col_a", 0, HiveType.HIVE_INT, INTEGER), + // Not in schema + createDummyHandle("col_x", 1, HiveType.HIVE_STRING, VARCHAR)); + + // Perform remapping (case-insensitive) - Expect NPE because "col_x" won't be found + assertThatThrownBy(() -> remapColumnIndicesToPhysical(fileSchema, requestedColumns, false)) + .isInstanceOf(NullPointerException.class); + } + + /** + * Creates a basic HiveColumnHandle for testing. + * Assumes REGULAR column type and no projection info or comments. + * The initial hiveColumnIndex is often irrelevant for this specific test, as we are testing the remapping logic. + * + * @param name Name of the column handle + * @param initialIndex The original index before remapping which might not be the physical one + * @param hiveType Hive type of column handle + * @param trinoType Trino type of column handle + */ + private HiveColumnHandle createDummyHandle( + String name, + int initialIndex, + HiveType hiveType, + Type trinoType) + { + return new HiveColumnHandle( + name, + initialIndex, + hiveType, + trinoType, + Optional.empty(), + HiveColumnHandle.ColumnType.REGULAR, + Optional.empty()); + } + + /** + * Asserts that a HiveColumnHandle has the expected properties after remapping. + */ + private void assertHandle( + HiveColumnHandle handle, + String expectedBaseName, + int expectedPhysicalIndex, + HiveType expectedHiveType, + Type expectedTrinoType) + { + assertThat(handle.getBaseColumnName()) + .as("BaseColumnName mismatch for %s", expectedBaseName) + .isEqualTo(expectedBaseName); + assertThat(handle.getBaseHiveColumnIndex()) + .as("BaseHiveColumnIndex (physical) mismatch for %s", expectedBaseName) + .isEqualTo(expectedPhysicalIndex); + assertThat(handle.getBaseHiveType()) + .as("BaseHiveType mismatch for %s", expectedBaseName) + .isEqualTo(expectedHiveType); + assertThat(handle.getType()) + .as("Trino Type mismatch for %s", expectedBaseName) + .isEqualTo(expectedTrinoType); + // Assert that other fields if they are relevant + assertThat(handle.getColumnType()) + .as("ColumnType mismatch for %s", expectedBaseName) + .isEqualTo(HiveColumnHandle.ColumnType.REGULAR); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPlugin.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPlugin.java new file mode 100644 index 0000000000000..943a7cb9ca035 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiPlugin.java @@ -0,0 +1,38 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.testing.TestingConnectorContext; +import org.junit.jupiter.api.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; + +final class TestHudiPlugin +{ + @Test + void testCreateConnector() + { + ConnectorFactory factory = getOnlyElement(new HudiPlugin().getConnectorFactories()); + factory.create( + "test", + ImmutableMap.builder() + .put("hive.metastore.uri", "thrift://foo:1234") + .put("bootstrap.quiet", "true") + .buildOrThrow(), + new TestingConnectorContext()) + .shutdown(); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSessionProperties.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSessionProperties.java new file mode 100644 index 0000000000000..c711a324b06d9 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSessionProperties.java @@ -0,0 +1,39 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.spi.connector.ConnectorSession; +import io.trino.testing.TestingConnectorSession; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.hudi.HudiSessionProperties.getColumnsToHide; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestHudiSessionProperties +{ + @Test + public void testSessionPropertyColumnsToHide() + { + HudiConfig config = new HudiConfig() + .setColumnsToHide(ImmutableList.of("col1", "col2")); + HudiSessionProperties sessionProperties = new HudiSessionProperties(config, new ParquetReaderConfig()); + ConnectorSession session = TestingConnectorSession.builder() + .setPropertyMetadata(sessionProperties.getSessionProperties()) + .build(); + assertThat(getColumnsToHide(session)) + .containsExactlyInAnyOrderElementsOf(ImmutableList.of("col1", "col2")); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSharedMetastore.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSharedMetastore.java new file mode 100644 index 0000000000000..32994032265fe --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSharedMetastore.java @@ -0,0 +1,178 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.filesystem.Location; +import io.trino.plugin.hive.TestingHivePlugin; +import io.trino.plugin.hudi.testing.TpchHudiTablesInitializer; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.tpch.TpchTable; +import org.junit.jupiter.api.Test; + +import java.nio.file.Path; +import java.util.List; + +import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.trino.testing.QueryAssertions.copyTpchTables; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static io.trino.testing.TestingSession.testSessionBuilder; +import static io.trino.tpch.TpchTable.NATION; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestHudiSharedMetastore + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + Session hudiSession = testSessionBuilder() + .setCatalog("hudi") + .setSchema("default") + .build(); + Session hiveSession = testSessionBuilder() + .setCatalog("hive") + .setSchema("default") + .build(); + + QueryRunner queryRunner = DistributedQueryRunner.builder(hudiSession).build(); + + Path dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("hudi_data"); + dataDirectory.toFile().deleteOnExit(); + + queryRunner.installPlugin(new HudiPlugin()); + queryRunner.createCatalog( + "hudi", + "hudi", + ImmutableMap.of( + "hive.metastore", "file", + "hive.metastore.catalog.dir", dataDirectory.toString(), + "fs.hadoop.enabled", "true")); + + queryRunner.installPlugin(new TestingHivePlugin(dataDirectory)); + queryRunner.createCatalog("hive", "hive"); + + queryRunner.execute("CREATE SCHEMA hive.default"); + + TpchHudiTablesInitializer tpchHudiTablesInitializer = new TpchHudiTablesInitializer(List.of(NATION)); + tpchHudiTablesInitializer.initializeTables(queryRunner, Location.of(dataDirectory.toString()), "default"); + + copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, hiveSession, ImmutableList.of(TpchTable.REGION)); + + return queryRunner; + } + + @Test + void testHudiSelectFromHiveTable() + { + String tableName = "test_hudi_select_from_hive_" + randomNameSuffix(); + assertUpdate("CREATE TABLE hive.default." + tableName + "(a bigint)"); + + assertQueryFails("SELECT * FROM hudi.default." + tableName, "Not a Hudi table: default." + tableName); + assertQueryFails("SELECT * FROM hudi.default.\"" + tableName + "$data\"", ".* Table .* does not exist"); + assertQueryFails("SELECT * FROM hudi.default.\"" + tableName + "$timeline\"", ".* Table .* does not exist"); + assertQueryFails("SELECT * FROM hudi.default.\"" + tableName + "$files\"", ".* Table .* does not exist"); + + assertUpdate("DROP TABLE hive.default." + tableName); + } + + @Test + void testHiveSelectFromHudiTable() + { + String tableName = "test_hive_select_from_hudi_" + randomNameSuffix(); + assertQueryFails("CREATE TABLE hudi.default." + tableName + "(a bigint)", "This connector does not support creating tables"); + + // TODO should be "Cannot query Hudi table" once CREATE TABLE is supported + assertQueryFails("SELECT * FROM hive.default." + tableName, ".* Table .* does not exist"); + assertQueryFails("SELECT * FROM hive.default.\"" + tableName + "$partitions\"", ".* Table .* does not exist"); + assertQueryFails("SELECT * FROM hive.default.\"" + tableName + "$properties\"", "Table .* not found"); + } + + @Test + void testHudiCannotCreateTableNamesakeToHiveTable() + { + String tableName = "test_hudi_create_namesake_hive_table_" + randomNameSuffix(); + assertUpdate("CREATE TABLE hive.default." + tableName + "(a bigint)"); + + assertQueryFails("CREATE TABLE hudi.default." + tableName + "(a bigint)", ".* Table .* of unsupported type already exists"); + + assertUpdate("DROP TABLE hive.default." + tableName); + } + + @Test + void testHiveCannotCreateTableNamesakeToHudiTable() + { + String tableName = "test_hive_create_namesake_hudi_table_" + randomNameSuffix(); + assertQueryFails("CREATE TABLE hudi.default." + tableName + "(a bigint)", "This connector does not support creating tables"); + // TODO implement test like testHiveCannotCreateTableNamesakeToIcebergTable when CREATE TABLE supported + } + + @Test + void testHiveSelectTableColumns() + { + assertThat(query("SELECT table_cat, table_schem, table_name, column_name FROM system.jdbc.columns WHERE table_cat = 'hive' AND table_schem = 'default' AND table_name = 'region'")) + .skippingTypesCheck() + .matches("VALUES " + + "('hive', '" + "default" + "', 'region', 'regionkey')," + + "('hive', '" + "default" + "', 'region', 'name')," + + "('hive', '" + "default" + "', 'region', 'comment')"); + + // Hive does not show any information about tables with unsupported format + assertQueryReturnsEmptyResult("SELECT table_cat, table_schem, table_name, column_name FROM system.jdbc.columns WHERE table_cat = 'hive' AND table_schem = 'default' AND table_name = 'nation'"); + } + + @Test + void testHiveListsHudiTable() + { + String tableName = "test_hive_lists_hudi_table_" + randomNameSuffix(); + assertQueryFails("CREATE TABLE hudi.default." + tableName + "(a bigint)", "This connector does not support creating tables"); + // TODO change doesNotContain to contains once CREATE TABLE supported + assertThat(query("SHOW TABLES FROM hive.default")).result().onlyColumnAsSet().doesNotContain(tableName); + } + + @Test + void testHudiListsHiveTable() + { + String tableName = "test_hudi_lists_hive_table_" + randomNameSuffix(); + assertUpdate("CREATE TABLE hive.default." + tableName + "(a bigint)"); + assertThat(query("SHOW TABLES FROM hudi.default")).result().onlyColumnAsSet().contains(tableName); + assertUpdate("DROP TABLE hive.default." + tableName); + } + + @Test + void testHudiSelectFromHiveView() + { + String tableName = "hudi_from_hive_table_" + randomNameSuffix(); + String viewName = "hudi_from_trino_hive_view_" + randomNameSuffix(); + assertUpdate("CREATE TABLE hive.default." + tableName + " AS SELECT 1 a", 1); + assertUpdate("CREATE VIEW hive.default." + viewName + " AS TABLE hive.default." + tableName); + + assertQueryFails("SELECT * FROM hudi.default." + viewName, "Not a Hudi table: .*"); + + assertUpdate("DROP VIEW hive.default." + viewName); + assertUpdate("DROP TABLE hive.default." + tableName); + } + + @Test + void testHiveSelectFromHudiView() + { + assertQueryFails("CREATE VIEW hudi.default.a_new_view AS SELECT 1 a", "This connector does not support creating views"); + // TODO test reading via Hive once Hudi supports CREATE VIEW + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java new file mode 100644 index 0000000000000..3f7300576789d --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java @@ -0,0 +1,1301 @@ +/* + * 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.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.common.io.Resources; +import io.trino.Session; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoInputFile; +import io.trino.filesystem.local.LocalInputFile; +import io.trino.parquet.ParquetReaderOptions; +import io.trino.plugin.base.metrics.FileFormatDataSourceStats; +import io.trino.plugin.hive.HiveTimestampPrecision; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.plugin.hudi.file.HudiBaseFile; +import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; +import io.trino.spi.SplitWeight; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.spi.type.Type; +import io.trino.sql.planner.OptimizerConfig; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.MaterializedResult; +import io.trino.testing.MaterializedRow; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorSession; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.util.Option; +import org.intellij.lang.annotations.Language; +import org.joda.time.DateTimeZone; +import org.json.JSONArray; +import org.json.JSONObject; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.File; +import java.time.LocalDateTime; +import java.time.ZonedDateTime; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static io.trino.metastore.HiveType.HIVE_TIMESTAMP; +import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; +import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; +import static io.trino.plugin.hudi.HudiPageSourceProvider.createPageSource; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_COMPREHENSIVE_TYPES_V6_MOR; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_COMPREHENSIVE_TYPES_V8_MOR; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_COW_PT_TBL; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_CUSTOM_KEYGEN_PT_V8_MOR; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_MULTI_PT_V8_MOR; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_NON_PART_COW; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_STOCK_TICKS_COW; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_STOCK_TICKS_MOR; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_TIMESTAMP_KEYGEN_PT_EPOCH_TO_YYYY_MM_DD_HH_V8_MOR; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_TIMESTAMP_KEYGEN_PT_SCALAR_TO_YYYY_MM_DD_HH_V8_MOR; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_TRIPS_COW_V8; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.STOCK_TICKS_COW; +import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.STOCK_TICKS_MOR; +import static io.trino.spi.type.TimestampType.createTimestampType; +import static io.trino.testing.MaterializedResult.materializeSourceDataStream; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestHudiSmokeTest + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return HudiQueryRunner.builder() + .setDataLoader(new ResourceHudiTablesInitializer()) + .build(); + } + + @Test + public void testReadNonPartitionedTable() + { + assertQuery( + "SELECT id, name FROM " + HUDI_NON_PART_COW, + "SELECT * FROM VALUES (1, 'a1'), (2, 'a2')"); + } + + @Test + public void testReadPartitionedTables() + { + assertQuery("SELECT symbol, max(ts) FROM " + STOCK_TICKS_COW + " GROUP BY symbol HAVING symbol = 'GOOG'", + "SELECT * FROM VALUES ('GOOG', '2018-08-31 10:59:00')"); + + assertQuery("SELECT symbol, max(ts) FROM " + STOCK_TICKS_MOR + " GROUP BY symbol HAVING symbol = 'GOOG'", + "SELECT * FROM VALUES ('GOOG', '2018-08-31 10:59:00')"); + System.out.println(getQueryRunner().execute(getSession(), "EXPLAIN ANALYZE SELECT * FROM " + HUDI_STOCK_TICKS_COW).toString()); + + System.out.println("test start"); + getQueryRunner().execute(getSession(), "SET SESSION hudi.metadata_enabled=true"); + String res = getQueryRunner().execute(getSession(), "SELECT * FROM " + HUDI_STOCK_TICKS_COW).toString(); + System.out.println(res); + assertQuery("SELECT dt, count(1) FROM " + STOCK_TICKS_MOR + " GROUP BY dt", + "SELECT * FROM VALUES ('2018-08-31', '99')"); + } + + @Test + public void testReadPartitionedCOWTableVer8() + { + String res = getQueryRunner().execute(getSession(), "SELECT * FROM " + HUDI_STOCK_TICKS_COW).toString(); + System.out.println(res); + assertQuery("SELECT date FROM " + HUDI_STOCK_TICKS_COW + " GROUP BY date", + "SELECT * FROM VALUES ('2018-08-31')"); + assertQuery("SELECT date, count(1) FROM " + HUDI_STOCK_TICKS_COW + " GROUP BY date", + "SELECT * FROM VALUES ('2018-08-31', '99')"); + } + + @Test + public void testReadPartitionedMORTableVer8() + { + getQueryRunner().execute(getSession(), "SET SESSION hudi.metadata_enabled=true"); + String res = getQueryRunner().execute(getSession(), "SELECT * FROM " + HUDI_STOCK_TICKS_COW).toString(); + System.out.println(res); + assertQuery("SELECT date FROM " + HUDI_STOCK_TICKS_MOR + " GROUP BY date", + "SELECT * FROM VALUES ('2018-08-31')"); + assertQuery("SELECT date, count(1) FROM " + HUDI_STOCK_TICKS_COW + " GROUP BY date", + "SELECT * FROM VALUES ('2018-08-31', '99')"); + } + + @Test + public void testBaseFileOnlyReadWithProjection() + { + Session session = SessionBuilder.from(getSession()).build(); + MaterializedResult countResult = getQueryRunner().execute( + session, "SELECT count(*) FROM " + HUDI_TRIPS_COW_V8); + assertThat(countResult.getOnlyValue()).isEqualTo(40000L); + assertThat(countResult.getStatementStats().get().getPhysicalInputBytes()).isLessThan(500000L); + MaterializedResult groupByResult = getQueryRunner().execute( + session, "SELECT driver, count(*) FROM " + HUDI_TRIPS_COW_V8 + " group by 1"); + assertThat(groupByResult.getMaterializedRows().size()).isEqualTo(1); + assertThat(groupByResult.getMaterializedRows().getFirst().getFieldCount()).isEqualTo(2); + assertThat(groupByResult.getMaterializedRows().getFirst().getField(0)).isEqualTo("driver-563"); + assertThat(groupByResult.getMaterializedRows().getFirst().getField(1)).isEqualTo(40000L); + assertThat(groupByResult.getStatementStats().get().getPhysicalInputBytes()).isLessThan(500000L); + } + + @Test + public void testReadPartitionedMORTables() + { + getQueryRunner().execute(getSession(), "SET SESSION hudi.metadata_enabled=true"); + String res = getQueryRunner().execute(getSession(), "SELECT * FROM " + HUDI_STOCK_TICKS_MOR).toString(); + System.out.println(res); + } + + @Test + public void testMultiPartitionedTable() + { + assertQuery("SELECT _hoodie_partition_path, id, name, ts, dt, hh FROM " + HUDI_COW_PT_TBL + " WHERE id = 1", + "SELECT * FROM VALUES ('dt=2021-12-09/hh=10', 1, 'a1', 1000, '2021-12-09', '10')"); + assertQuery("SELECT _hoodie_partition_path, id, name, ts, dt, hh FROM " + HUDI_COW_PT_TBL + " WHERE id = 2", + "SELECT * FROM VALUES ('dt=2021-12-09/hh=11', 2, 'a2', 1000, '2021-12-09', '11')"); + } + + @Test + public void testShowCreateTable() + { + assertThat((String) computeActual("SHOW CREATE TABLE " + STOCK_TICKS_COW).getOnlyValue()) + .matches("CREATE TABLE \\w+\\.\\w+\\.stock_ticks_cow \\Q(\n" + + " _hoodie_commit_time varchar,\n" + + " _hoodie_commit_seqno varchar,\n" + + " _hoodie_record_key varchar,\n" + + " _hoodie_partition_path varchar,\n" + + " _hoodie_file_name varchar,\n" + + " volume bigint,\n" + + " ts varchar,\n" + + " symbol varchar,\n" + + " year integer,\n" + + " month varchar,\n" + + " high double,\n" + + " low double,\n" + + " key varchar,\n" + + " date varchar,\n" + + " close double,\n" + + " open double,\n" + + " day varchar,\n" + + " dt varchar\n" + + ")\n" + + "WITH (\n" + + " location = \\E'.*/stock_ticks_cow',\n\\Q" + + " partitioned_by = ARRAY['dt']\n" + + ")"); + // multi-partitioned table + assertThat((String) computeActual("SHOW CREATE TABLE " + HUDI_COW_PT_TBL).getOnlyValue()) + .matches("CREATE TABLE \\w+\\.\\w+\\.hudi_cow_pt_tbl \\Q(\n" + + " _hoodie_commit_time varchar,\n" + + " _hoodie_commit_seqno varchar,\n" + + " _hoodie_record_key varchar,\n" + + " _hoodie_partition_path varchar,\n" + + " _hoodie_file_name varchar,\n" + + " id bigint,\n" + + " name varchar,\n" + + " ts bigint,\n" + + " dt varchar,\n" + + " hh varchar\n" + + ")\n" + + "WITH (\n" + + " location = \\E'.*/hudi_cow_pt_tbl',\n\\Q" + + " partitioned_by = ARRAY['dt','hh']\n" + + ")"); + } + + @Test + public void testMetaColumns() + { + assertQuery("SELECT _hoodie_commit_time FROM hudi_cow_pt_tbl", "VALUES ('20220906063435640'), ('20220906063456550')"); + assertQuery("SELECT _hoodie_commit_seqno FROM hudi_cow_pt_tbl", "VALUES ('20220906063435640_0_0'), ('20220906063456550_0_0')"); + assertQuery("SELECT _hoodie_record_key FROM hudi_cow_pt_tbl", "VALUES ('id:1'), ('id:2')"); + assertQuery("SELECT _hoodie_partition_path FROM hudi_cow_pt_tbl", "VALUES ('dt=2021-12-09/hh=10'), ('dt=2021-12-09/hh=11')"); + assertQuery( + "SELECT _hoodie_file_name FROM hudi_cow_pt_tbl", + "VALUES ('719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet'), ('4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet')"); + } + + @Test + public void testPathColumn() + throws Exception + { + String path1 = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + assertThat(toInputFile(path1).exists()).isTrue(); + String path2 = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_STOCK_TICKS_MOR + " WHERE volume = 6794"); + assertThat(toInputFile(path2).exists()).isTrue(); + } + + @Test + public void testFileSizeColumn() + throws Exception + { + String path = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + long fileSize = (long) computeScalar("SELECT \"$file_size\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + assertThat(fileSize).isEqualTo(toInputFile(path).length()); + } + + @Test + public void testFileModifiedColumn() + throws Exception + { + String path = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + ZonedDateTime fileModifiedTime = (ZonedDateTime) computeScalar("SELECT \"$file_modified_time\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + assertThat(fileModifiedTime.toInstant().toEpochMilli()) + .isEqualTo(toInputFile(path).lastModified().toEpochMilli()); + } + + @Test + public void testPartitionColumn() + { + assertQuery("SELECT \"$partition\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1", "VALUES 'dt=2021-12-09/hh=10'"); + assertQuery("SELECT \"$partition\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 2", "VALUES 'dt=2021-12-09/hh=11'"); + + assertQueryFails("SELECT \"$partition\" FROM " + HUDI_NON_PART_COW, ".* Column '\\$partition' cannot be resolved"); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testTableStatistics(boolean tableStatisticsEnabled) + throws InterruptedException + { + @Language("SQL") String query = "EXPLAIN (FORMAT JSON) SELECT t1.uuid, t1.driver, t1.fare, t1.ts " + + "FROM " + HUDI_TRIPS_COW_V8 + " t1 " + + "INNER JOIN " + HUDI_TRIPS_COW_V8 + " t2 ON t1.uuid = t2.uuid " + + "WHERE t2.ts > 0"; + Session session = SessionBuilder.from(getSession()) + .withJoinDistributionType(OptimizerConfig.JoinDistributionType.AUTOMATIC.name()) + .withTableStatisticsEnabled(tableStatisticsEnabled) + .withMdtEnabled(true) + .build(); + QueryRunner queryRunner = getQueryRunner(); + // First time the asynchronous table statistics calculation is triggered + queryRunner.execute(session, query); + Thread.sleep(2000); + // Second time the table statistics is available for CBO and the join distribution type should be REPLICATED + String jsonPlanString = (String) queryRunner.execute(session, query).getOnlyValue(); + + // Navigate to the ScanFilterProject node + String tableName = "tests.hudi_trips_cow_v8"; + JSONObject scanNode = findNodeInPlan(jsonPlanString, "ScanFilterProject", Option.of(tableName)); + assertThat(scanNode).isNotNull(); + + // Verify the estimates are based on the table statistics if enabled + JSONArray estimatesArray = scanNode.getJSONArray("estimates"); + assertThat(estimatesArray).isNotNull(); + assertThat(estimatesArray.length()).isGreaterThan(0); + JSONObject estimates = estimatesArray.getJSONObject(0); + assertThat(estimates).isNotNull(); + if (tableStatisticsEnabled) { + assertThat(estimates.getDouble("outputRowCount")).isEqualTo(40000.0); + assertThat(estimates.getDouble("outputSizeInBytes")).isGreaterThan(20000.0); + } + else { + assertThat(estimates.getString("outputRowCount")).isEqualTo("NaN"); + assertThat(estimates.getString("outputSizeInBytes")).isEqualTo("NaN"); + } + + // Verify the join distribution type is REPLICATED if table statistics is enabled; PARTITIONED otherwise + JSONObject joinNode = findNodeInPlan(jsonPlanString, "InnerJoin", Option.empty()); + String distributionDetails = findDetailContaining(joinNode, "Distribution"); + assertThat(distributionDetails).isNotNull(); + String distribution = distributionDetails.split(":")[1].trim(); + assertThat(distribution).isEqualTo(tableStatisticsEnabled ? "REPLICATED" : "PARTITIONED"); + } + + @Test + public void testPartitionFilterRequired() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQueryFails( + session, + "SELECT * FROM " + HUDI_COW_PT_TBL, + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredPredicateOnNonPartitionColumn() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQueryFails( + session, + "SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE id = 1", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredNestedQueryWithInnerPartitionPredicate() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQuery(session, "SELECT name FROM (SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE dt = '2021-12-09') WHERE id = 1", "VALUES 'a1'"); + } + + @Test + public void testPartitionFilterRequiredNestedQueryWithOuterPartitionPredicate() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQuery(session, "SELECT name FROM (SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE id = 1) WHERE dt = '2021-12-09'", "VALUES 'a1'"); + } + + @Test + public void testPartitionFilterRequiredNestedWithIsNotNullFilter() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE dt IS NOT null", "VALUES 'a1', 'a2'"); + } + + @Test + public void testPartitionFilterRequiredFilterRemovedByPlanner() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQueryFails( + session, + "SELECT id FROM " + HUDI_COW_PT_TBL + " WHERE dt IS NOT null OR true", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredOnJoin() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + + // ON with partition column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt)", + errorMessage); + // ON with partition column and WHERE with same left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with same right table's regular column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with different left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.hh = '10'", + "VALUES ('a1', 'a1'), ('a1', 'a2')"); + // ON with partition column and WHERE with different regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.hh = '10'", + errorMessage); + // ON with partition column and WHERE with regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.id = 1", + errorMessage); + + // ON with regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.id = t2.id)", + errorMessage); + // ON with regular column and WHERE with left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.id = t2.id) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2')"); + // ON with partition column and WHERE with right table's regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.id = 1", + errorMessage); + } + + @Test + public void testPartitionFilterRequiredOnJoinBothTablePartitioned() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + // ON with partition column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt)", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + // ON with partition column and WHERE with same left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with same right table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + // ON with partition column and WHERE with different left table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t1.hh = '10'", errorMessage); + // ON with partition column and WHERE with different right table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.hh = '10'", errorMessage); + // ON with partition column and WHERE with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.id = 1", errorMessage); + + // ON with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id)", errorMessage); + // ON with regular column and WHERE with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id) WHERE t1.id = 1", errorMessage); + // ON with regular column and WHERE with left table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id) WHERE t1.dt = '2021-12-09'", errorMessage); + } + + @Test + public void testPartitionFilterRequiredWithLike() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + assertQueryFails( + session, + "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1'", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredFilterIncluded() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh = '10'", "VALUES 'a1'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh < '12'", "VALUES 2"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE Hh < '11'", "VALUES 1"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE HH < '10'", "VALUES 0"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 = 1 and hh IS NOT NULL", "VALUES 'a2'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh IS NULL", "VALUES 0"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh IS NOT NULL", "VALUES 2"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh LIKE '10'", "VALUES 'a1'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh LIKE '1%'", "VALUES 'a1', 'a2'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE id = 1 AND dt = '2021-12-09'", "VALUES 'a1'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh = '11' AND dt = '2021-12-09'", "VALUES 'a2'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '12' AND dt = '2021-12-19'", "VALUES 0"); + + // Predicate which could not be translated into tuple domain + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 = 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) - 11 = 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) * 2 = 20", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 > 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1' OR hh LIKE '%1'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1' AND hh LIKE '%0'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE id = 1 OR dt = '2021-12-09'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '11' OR dt = '2021-12-09'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '12' OR dt = '2021-12-19'", errorMessage); + assertQueryFails(session, "SELECT count(*) AS COUNT FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) > 2 GROUP BY name ", errorMessage); + } + + @Test + public void testHudiLongTimestampType() + throws Exception + { + testTimestampMicros(HiveTimestampPrecision.MILLISECONDS, LocalDateTime.parse("2020-10-12T16:26:02.907")); + testTimestampMicros(HiveTimestampPrecision.MICROSECONDS, LocalDateTime.parse("2020-10-12T16:26:02.906668")); + testTimestampMicros(HiveTimestampPrecision.NANOSECONDS, LocalDateTime.parse("2020-10-12T16:26:02.906668")); + } + + @Test + public void testHudiCustomKeygenEpochMillisPartitionedTables() + { + // NOTE: As of now, the partition_path value that is synced to metastore will be returned instead of the raw value that is used by the keygen + Session session = SessionBuilder.from(getSession()).build(); + @Language("SQL") String actualQuery = "SELECT _hoodie_partition_path, partition_field_country, partition_field_date FROM " + HUDI_CUSTOM_KEYGEN_PT_V8_MOR; + @Language("SQL") String expectedQuery = "VALUES ('partition_field_country=MY/partition_field_date=2025-05-13', 'MY', '2025-05-13')," + + "('partition_field_country=CN/partition_field_date=2025-06-05', 'CN', '2025-06-05')," + + "('partition_field_country=US/partition_field_date=2025-06-06', 'US', '2025-06-06')," + + "('partition_field_country=SG/partition_field_date=2025-06-06', 'SG', '2025-06-06')," + + "('partition_field_country=SG/partition_field_date=2025-06-06', 'SG', '2025-06-06')," + + "('partition_field_country=SG/partition_field_date=2025-06-07', 'SG', '2025-06-07')," + + "('partition_field_country=SG/partition_field_date=2025-06-07', 'SG', '2025-06-07')"; + assertQuery(session, actualQuery, expectedQuery); + + // Ensure that partition pruning is working (using partition_path value) of level 3 partition_path value + @Language("SQL") String actualPartitionPruningQuery = actualQuery + " WHERE partition_field_date='2025-06-06'"; + MaterializedResult partitionPruningResult = getQueryRunner().execute(session, actualPartitionPruningQuery); + // Only one split in the partition, hence, only one split processed + assertThat(partitionPruningResult.getStatementStats().get().getTotalSplits()).isEqualTo(2); + // 2 splits/filegroups, but 3 rows + assertQuery(actualPartitionPruningQuery, "VALUES ('partition_field_country=US/partition_field_date=2025-06-06', 'US', '2025-06-06'), " + + "('partition_field_country=SG/partition_field_date=2025-06-06', 'SG', '2025-06-06'), " + + "('partition_field_country=SG/partition_field_date=2025-06-06', 'SG', '2025-06-06')"); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testHudiTimestampKeygenEpochMillisPartitionedTables(boolean isRtTable) + { + String tableName = isRtTable ? HUDI_TIMESTAMP_KEYGEN_PT_EPOCH_TO_YYYY_MM_DD_HH_V8_MOR.getRtTableName() + : HUDI_TIMESTAMP_KEYGEN_PT_EPOCH_TO_YYYY_MM_DD_HH_V8_MOR.getRoTableName(); + // NOTE: As of now, the partition_path value that is synced to metastore will be returned instead of the raw value that is used by the keygen + Session session = SessionBuilder.from(getSession()).build(); + @Language("SQL") String actualQuery = "SELECT _hoodie_partition_path, partition_field FROM " + tableName; + @Language("SQL") String expectedQuery = "VALUES ('2025-06-07 08', '2025-06-07 08'), ('2025-06-06 10', '2025-06-06 10'), ('2025-06-06 09', '2025-06-06 09'), " + + "('2025-06-05 05', '2025-06-05 05'), ('2025-05-13 02', '2025-05-13 02')"; + assertQuery(session, actualQuery, expectedQuery); + + // Ensure that partition pruning is working (using partition_path value) + @Language("SQL") String actualPartPruningQuery = actualQuery + " WHERE partition_field='2025-06-07 08'"; + MaterializedResult partPruneRes = getQueryRunner().execute(session, actualPartPruningQuery); + // Only one split in the partition, hence, only one split processed + assertThat(partPruneRes.getStatementStats().get().getTotalSplits()).isEqualTo(1); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testHudiTimestampKeygenScalarPartitionedTables(boolean isRtTable) + { + String tableName = isRtTable ? HUDI_TIMESTAMP_KEYGEN_PT_SCALAR_TO_YYYY_MM_DD_HH_V8_MOR.getRtTableName() + : HUDI_TIMESTAMP_KEYGEN_PT_SCALAR_TO_YYYY_MM_DD_HH_V8_MOR.getRoTableName(); + // NOTE: As of now, the partition_path value that is synced to metastore will be returned instead of the raw value that is used by the keygen + Session session = SessionBuilder.from(getSession()).build(); + @Language("SQL") String actualQuery = "SELECT _hoodie_partition_path, partition_field FROM " + + tableName; + @Language("SQL") String expectedQuery = "VALUES ('2024-10-04 12', '2024-10-04 12'), ('2024-10-05 12', '2024-10-05 12'), ('2024-10-06 12', '2024-10-06 12'), " + + "('2024-10-07 12', '2024-10-07 12'), ('2024-10-08 12', '2024-10-08 12')"; + assertQuery(session, actualQuery, expectedQuery); + + // Ensure that partition pruning is working (using partition_path value) + @Language("SQL") String actualPartPruningQuery = actualQuery + " WHERE partition_field='2024-10-04 12'"; + MaterializedResult partPruneRes = getQueryRunner().execute(session, actualPartPruningQuery); + // Only one split in the partition, hence, only one split processed + assertThat(partPruneRes.getStatementStats().get().getTotalSplits()).isEqualTo(1); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testPartitionPruningReadMultiFgPartitionedMOR(ResourceHudiTablesInitializer.TestingTable table) + { + // Test for partition pruning without MDT (i.e. w/o partition pruning using partition stats index) + Session session = SessionBuilder.from(getSession()).build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + " WHERE country='SG'"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With partition pruning, only 2 splits in the partition should be returned + assertThat(prunedSplits).isEqualTo(2); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testColStatsFileSkipping(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(true) + .withColumnStatsTimeout("1s") + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(false) + .build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + " WHERE country='SG' AND name='a1'"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With colstats file skipping, only 1 split should be returned + assertThat(prunedSplits).isEqualTo(1); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testRecordLevelFileSkipping(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(true) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(false) + .withColumnStatsTimeout("1s") + .build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + + " WHERE country='SG' AND id IN (1, 3) AND name = 'a1'"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With RLI file skipping, only 1 split should be returned + assertThat(prunedSplits).isEqualTo(1); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testSecondaryIndexFileSkipping(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(true) + .withPartitionStatsIndexEnabled(false) + .withSecondaryIndexTimeout("10s") + .build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + + " WHERE country='SG' AND price = 101.00"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + assertThat(prunedSplits).isLessThan(totalSplits); + // SI is only available for table versions >= 8 + // With SI file skipping, only 1 split should be returned + int expectedSplits = table.getHoodieTableVersion() + .greaterThanOrEquals(HoodieTableVersion.EIGHT) ? 1 : 2; + assertThat(prunedSplits).isEqualTo(expectedSplits); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testPartitionStatsIndexPartitionPruning(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(true) + .build(); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + // Add a constraint that is in colstats + + " WHERE ts < 1001 " + + // Add a constraint that is in colstats + "AND price < 200.00 " + + // Add a constraint on a column that is not in colstats + "AND _hoodie_file_name = 'abc' " + + // Add a simple null check constraint + "AND id is not null"); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + + // Partition stats index is only available for table versions >= 8 + // With PSI, only 2 splits in the SG partitions will be scanned + int expectedSplits = table.getHoodieTableVersion() + .greaterThanOrEquals(HoodieTableVersion.EIGHT) ? 2 : 4; + assertThat(prunedSplits).isEqualTo(expectedSplits); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testDynamicFilterEnabledPredicatePushdown(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder + .from(getSession()) + .withDynamicFilterTimeout("10s") + .build(); + final String tableIdentifier = "hudi:tests." + table.getRoTableName(); + + @Language("SQL") String query = "SELECT t1.id, t1.name, t1.price, t1.ts FROM " + + table + " t1 " + + "INNER JOIN " + table + " t2 ON t1.id = t2.id " + + "WHERE t2.price <= 102"; + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isTrue(); + + // matcher#group() must be invoked after matcher#find() + String rowsInputString = matcher.group(1); + long actualInputRows = Long.parseLong(rowsInputString); + long expectedInputRowsAfterFiltering = 2; + assertThat(actualInputRows) + .describedAs("Number of rows input to the ScanFilter for the probe side table (%s) should reflect effective dynamic filtering", tableIdentifier) + .isEqualTo(expectedInputRowsAfterFiltering); + + // Exercise query and check output + assertQuery(query, "VALUES (1, 'a1', 100.0, 1000), (3, 'a3', 101.0, 1001)"); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testDynamicFilterDisabledPredicatePushdown(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withDynamicFilterEnabled(false) + .build(); + final String tableIdentifier = "hudi:tests." + table.getRoTableName(); + + @Language("SQL") String query = "SELECT t1.id, t1.name, t1.price, t1.ts FROM " + + table + " t1 " + + "INNER JOIN " + table + " t2 ON t1.id = t2.id " + + "WHERE t2.price <= 102"; + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + // Should not find any dynamic filtering keywords + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isFalse(); + + // Skip check on whether optimization is not applied or not, just check that output is queryable + assertQuery(query, "VALUES (1, 'a1', 100.0, 1000), (3, 'a3', 101.0, 1001)"); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testDynamicFilterEnabled_withPartitionPruningUsingDynamicFilter(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withDynamicFilterTimeout("10s") + .build(); + final String tableIdentifier = "hudi:tests." + table.getRoTableName(); + // Query is joined-on partitionField + @Language("SQL") String query = "SELECT t1.id, t1.name, t1.price, t1.ts, t1.country FROM " + + table + " t1 " + + "INNER JOIN " + table + " t2 ON t1.country = t2.country " + + "WHERE t2.price <= 102"; + + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isTrue(); + + // matcher#group() must be invoked after matcher#find() + String rowsInputString = matcher.group(1); + long actualInputRows = Long.parseLong(rowsInputString); + long expectedInputRowsAfterFiltering = 2; + assertThat(actualInputRows) + .describedAs("Number of rows input to the ScanFilter for the probe side table (%s) should reflect effective dynamic filtering", tableIdentifier) + .isEqualTo(expectedInputRowsAfterFiltering); + + // Exercise query and check output + assertQuery(query, "VALUES (1, 'a1', 100.0, 1000, 'SG'), (3, 'a3', 101.0, 1001, 'SG'), (1, 'a1', 100.0, 1000, 'SG'), (3, 'a3', 101.0, 1001, 'SG')"); + } + + @Test + public void testDynamicFilterEnabled_withPartitionPruningUsingDynamicFilterOnNestedPartitions() + { + Session session = SessionBuilder.from(getSession()) + .withDynamicFilterTimeout("10s") + .build(); + final String tableIdentifier = "hudi:tests." + HUDI_MULTI_PT_V8_MOR.getRoTableName(); + // Query is joined-on recordKey and partitionField + @Language("SQL") String query = "SELECT t1.id FROM " + + HUDI_MULTI_PT_V8_MOR + " t1 " + + "INNER JOIN " + HUDI_MULTI_PT_V8_MOR + " t2 ON t1.id = t2.id AND t1.part_int = t2.part_int " + + "WHERE t2.part_int = 2023"; + + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isTrue(); + + // matcher#group() must be invoked after matcher#find() + String rowsInputString = matcher.group(1); + long actualInputRows = Long.parseLong(rowsInputString); + // 1 row in each split, should only scan 3 splits, i.e. 3 rows + // For a more strict search, we can check the number of splits scanned on the builder side + long expectedInputRowsAfterFiltering = 3; + assertThat(actualInputRows) + .describedAs("Number of rows input to the ScanFilter for the probe side table (%s) should reflect effective dynamic filtering", tableIdentifier) + .isEqualTo(expectedInputRowsAfterFiltering); + + // Exercise query and check output + assertQuery(query, "VALUES (1), (2), (4)"); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testDynamicFilterDisabled_withPartitionPruningUsingDynamicFilter(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withDynamicFilterEnabled(false) + .build(); + final String tableIdentifier = "hudi:tests." + table.getRoTableName(); + + // Query is joined-on recordKey and partitionField + @Language("SQL") String query = "SELECT t1.id, t1.name, t1.price, t1.ts, t1.country FROM " + + table + " t1 " + + "INNER JOIN " + table + " t2 ON t1.country = t2.country " + + "WHERE t2.price <= 102"; + + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isFalse(); + + // Skip check on whether optimization is not applied or not, just check that output is queryable + // Cartesian product of result is produced since we are joining by partition column + assertQuery(query, "VALUES (1, 'a1', 100.0, 1000, 'SG'), (3, 'a3', 101.0, 1001, 'SG'), (1, 'a1', 100.0, 1000, 'SG'), (3, 'a3', 101.0, 1001, 'SG')"); + } + + @Test + public void testPartitionPruningOnNestedPartitions() + { + // Should only scan paths that match the part_str=*/part_int=2023/part_date=*/part_bigint=*/part_decimal=*/part_timestamp=*/part_bool=* + Session session = getSession(); + // No partition pruning + @Language("SQL") String actualQuery = "SELECT part_str, part_int, part_date, part_bigint, part_bool FROM " + HUDI_MULTI_PT_V8_MOR; + MaterializedResult actualRes = getQueryRunner().execute(session, actualQuery); + int actualTotalSplits = actualRes.getStatementStats().get().getTotalSplits(); + assertThat(actualTotalSplits).isEqualTo(5); + + // With partition pruning + @Language("SQL") String actualPartPruneQuery = actualQuery + " WHERE part_int = 2023"; + MaterializedResult actualPartPruneRes = getQueryRunner().execute(session, actualPartPruneQuery); + int actualPartPruneSplits = actualPartPruneRes.getStatementStats().get().getTotalSplits(); + assertThat(actualPartPruneSplits).isLessThan(actualTotalSplits); + assertThat(actualPartPruneSplits).isEqualTo(3); + } + + @ParameterizedTest + @MethodSource("comprehensiveTestParameters") + public void testComprehensiveTypes(ResourceHudiTablesInitializer.TestingTable table, boolean isRtTable) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .build(); + // Not using #assertQuery() as it uses H2QueryRunner, which restricts the types which can be defined, particularly MAP types + // Use #getQueryRunner(), which uses TrinoQueryRunner instead + + // Define the columns that are being tested: + ImmutableList columnsToTest = ImmutableList.of( + "uuid", "precombine_field", "col_boolean", "col_tinyint", "col_smallint", "col_int", "col_bigint", "col_float", "col_double", "col_decimal", "col_string", + "col_varchar", "col_char", "col_binary", "col_date", "col_timestamp", "col_array_int", "col_array_string", "col_map_string_int", "col_struct", "col_array_struct", + "col_map_string_struct", "col_array_struct_with_map", "col_map_struct_with_array", "col_struct_nested_struct", "col_array_array_int", "col_map_string_array_double", + "col_map_string_map_string_date", "col_struct_array_struct", "col_struct_map", "part_col"); + + // Define expected output + ImmutableList> expectedRowValues = ImmutableList.of( + // uuid STRING + ImmutableList.of("'uuid1'", "'uuid2'", "'uuid3'"), + // precombine_field LONG -> BIGINT + ImmutableList.of( + // Updates were performed (RT table holds the updated value) + isRtTable ? "BIGINT '1001'" : "BIGINT '1000'", + isRtTable ? "BIGINT '1006'" : "BIGINT '1005'", + isRtTable ? "BIGINT '1101'" : "BIGINT '1100'"), + // col_boolean BOOLEAN + ImmutableList.of("true", "false", "CAST(NULL AS BOOLEAN)"), + // col_tinyint TINYINT + ImmutableList.of("TINYINT '1'", "TINYINT '2'", "CAST(NULL AS TINYINT)"), + // col_smallint SMALLINT + ImmutableList.of("SMALLINT '100'", "SMALLINT '200'", "CAST(NULL AS SMALLINT)"), + // col_int + ImmutableList.of("INTEGER '1000'", "INTEGER '2000'", "CAST(NULL AS INTEGER)"), + // col_bigint BIGINT + ImmutableList.of("BIGINT '100000'", "BIGINT '200000'", "CAST(NULL AS BIGINT)"), + // col_float + ImmutableList.of("REAL '1.1'", "REAL '2.2'", "CAST(NULL AS REAL)"), + // col_double + ImmutableList.of( + // Updates were performed on partition A values (RT table holds the updated value) + isRtTable ? "DOUBLE '110.123'" : "DOUBLE '10.123'", + isRtTable ? "DOUBLE '120.456'" : "DOUBLE '20.456'", + "CAST(NULL AS DOUBLE)"), + // col_decimal + ImmutableList.of("DECIMAL '123.45'", "DECIMAL '234.56'", "CAST(NULL AS DECIMAL(10,2))"), + // col_string + ImmutableList.of( + "'string val 1'", + "'string val 2'", + // Updates were performed on partition B values (RT table holds the updated value) + isRtTable ? "'updated string'" : "NULL"), + // col_varchar + ImmutableList.of("CAST('varchar val 1' AS VARCHAR(50))", "CAST('varchar val 2' AS VARCHAR(50))", "CAST(NULL AS VARCHAR(50))"), + // col_char + ImmutableList.of("CAST('charval1' AS CHAR(10))", "CAST('charval2' AS CHAR(10))", "CAST(NULL AS CHAR(10))"), + // col_binary BINARY -> VARBINARY: UTF-8 bytes of "binary1", "binary2", null + ImmutableList.of("X'62696e61727931'", "X'62696e61727932'", "CAST(NULL AS VARBINARY)"), + // col_date + ImmutableList.of("DATE '2025-01-15'", "DATE '2025-02-20'", "CAST(NULL AS DATE)"), + // col_timestamp TIMESTAMP + ImmutableList.of("TIMESTAMP '2025-01-15 11:30:00.000'", "TIMESTAMP '2025-02-20 12:45:00.000'", "CAST(NULL AS TIMESTAMP)"), + // col_array_int ARRAY + ImmutableList.of("ARRAY[1, 2, 3]", "ARRAY[4, 5]", "CAST(NULL AS ARRAY)"), + // col_array_string ARRAY + ImmutableList.of("ARRAY['a', 'b', 'c']", "ARRAY['d', 'e', 'f']", "CAST(NULL AS ARRAY)"), + // col_map_string_int MAP + ImmutableList.of("MAP(ARRAY['key1', 'key2'], ARRAY[10, 20])", "MAP(ARRAY['key3'], ARRAY[30])", "CAST(NULL AS MAP(VARCHAR, INTEGER))"), + // col_struct + ImmutableList.of( + "CAST(ROW('struct_str1', 55, false) AS ROW(f1 VARCHAR, f2 INTEGER, f3 BOOLEAN))", + "CAST(ROW('struct_str2', 66, true) AS ROW(f1 VARCHAR, f2 INTEGER, f3 BOOLEAN))", + "CAST(NULL AS ROW(f1 VARCHAR, f2 INTEGER, f3 BOOLEAN))"), + // col_array_struct + ImmutableList.of( + "ARRAY[CAST(ROW(1.1E0, ARRAY['n1','n2']) AS ROW(nested_f1 DOUBLE, nested_f2 ARRAY)), CAST(ROW(2.2E0, ARRAY['n3']) AS ROW(nested_f1 DOUBLE, nested_f2 ARRAY))]", + "CAST(NULL AS ARRAY)>)", + "ARRAY[CAST(ROW(3.3E0, ARRAY['n4']) AS ROW(nested_f1 DOUBLE, nested_f2 ARRAY))]"), + // col_map_string_struct + ImmutableList.of( + "MAP(ARRAY['mapkey1'], ARRAY[CAST(ROW(DATE '2024-11-01', DECIMAL '9.80') AS ROW(nested_f3 DATE, nested_f4 DECIMAL(5,2)))])", + "MAP(ARRAY['mapkey2'], ARRAY[CAST(ROW(DATE '2024-12-10', DECIMAL '7.60') AS ROW(nested_f3 DATE, nested_f4 DECIMAL(5,2)))])", + "CAST(NULL AS MAP)"), + // col_array_struct_with_map + ImmutableList.of( + "ARRAY[CAST(ROW('arr_struct1', MAP(ARRAY['map_in_struct_k1'], ARRAY[1])) AS ROW(f_arr_struct_str VARCHAR, f_arr_struct_map MAP)), CAST(ROW('arr_struct2', MAP(ARRAY['map_in_struct_k2', 'map_in_struct_k3'], ARRAY[2, 3])) AS ROW(f_arr_struct_str VARCHAR, f_arr_struct_map MAP))]", + // inner map is null + "ARRAY[CAST(ROW('arr_struct3', MAP(ARRAY['map_in_struct_k4'], ARRAY[4])) AS ROW(f_arr_struct_str VARCHAR, f_arr_struct_map MAP)), CAST(ROW('arr_struct4', CAST(NULL AS MAP)) AS ROW(f_arr_struct_str VARCHAR, f_arr_struct_map MAP))]", + "CAST(NULL AS ARRAY)>)"), + // col_map_struct_with_array + ImmutableList.of( + "MAP(ARRAY['map_struct1', 'map_struct2'], ARRAY[CAST(ROW(ARRAY[true, false], TIMESTAMP '2025-01-01 01:01:01.000') AS ROW(f_map_struct_arr ARRAY, f_map_struct_ts TIMESTAMP(3))), CAST(ROW(ARRAY[false], TIMESTAMP '2025-02-02 02:02:02.000') AS ROW(f_map_struct_arr ARRAY, f_map_struct_ts TIMESTAMP(3)))])", + // inner map is null + "MAP(ARRAY['map_struct3', 'map_struct4'], ARRAY[CAST(ROW(CAST(NULL AS ARRAY), TIMESTAMP '2025-03-03 03:03:03.000') AS ROW(f_map_struct_arr ARRAY, f_map_struct_ts TIMESTAMP(3))), CAST(ROW(ARRAY[true], CAST(NULL AS TIMESTAMP(3))) AS ROW(f_map_struct_arr ARRAY, f_map_struct_ts TIMESTAMP(3)))])", + "CAST(NULL AS MAP, f_map_struct_ts TIMESTAMP(3))>)"), + // col_struct_nested_struct + ImmutableList.of( + "CAST(ROW(101, CAST(ROW('inner_str_1', true) AS ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN))) AS ROW(outer_f1 INTEGER, nested_struct ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN)))", + // inner struct is null + "CAST(ROW(102, CAST(NULL AS ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN))) AS ROW(outer_f1 INTEGER, nested_struct ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN)))", + "CAST(NULL AS ROW(outer_f1 INTEGER, nested_struct ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN)))"), + // col_array_array_int + ImmutableList.of("ARRAY[ARRAY[1, 2], ARRAY[3, 4, 5]]", "ARRAY[ARRAY[6], ARRAY[7, 8]]", "CAST(NULL AS ARRAY>)"), + // col_map_string_array_double + ImmutableList.of( + "MAP(ARRAY['arr_key1', 'arr_key2'], ARRAY[ARRAY[1.1E0, 2.2E0], ARRAY[3.3E0]])", + // inner array is null + "MAP(ARRAY['arr_key3'], ARRAY[CAST(NULL AS ARRAY)])", + "CAST(NULL AS MAP>)"), + // col_map_string_map_string_date + ImmutableList.of( + "MAP(ARRAY['map_key1'], ARRAY[MAP(ARRAY['mapkey10', 'mapkey20'], ARRAY[DATE '2024-01-01', DATE '2024-02-02'])])", + // inner map value/map is null, assuming int key 30 coerced to '30' + "MAP(ARRAY['map_key2', 'map_key3'], ARRAY[MAP(ARRAY[CAST('30' AS VARCHAR)], ARRAY[CAST(NULL AS DATE)]), CAST(NULL AS MAP)])", + "CAST(NULL AS MAP>)"), + // col_struct_array_struct + ImmutableList.of( + "CAST(ROW('outer_str_1', ARRAY[CAST(ROW(TIMESTAMP '2023-11-11 11:11:11.000', 'inner_str_1') AS ROW(inner_f3 TIMESTAMP(3), inner_f4 VARCHAR))]) AS ROW(outer_f2 VARCHAR, struct_array ARRAY))", + "CAST(ROW('outer_str_2', ARRAY[CAST(ROW(TIMESTAMP '2023-12-12 12:12:12.000', 'inner_str_2') AS ROW(inner_f3 TIMESTAMP(3), inner_f4 VARCHAR))]) AS ROW(outer_f2 VARCHAR, struct_array ARRAY))", + "CAST(NULL AS ROW(outer_f2 VARCHAR, struct_array ARRAY))"), + // col_struct_map (BIGINT literals don't need L) + ImmutableList.of( + "CAST(ROW(true, MAP(ARRAY['struct_map_k1', 'struct_map_k2'], ARRAY[1000, 2000])) AS ROW(outer_f3 BOOLEAN, struct_map MAP))", + // inner map is null + "CAST(ROW(false, CAST(NULL AS MAP)) AS ROW(outer_f3 BOOLEAN, struct_map MAP))", + "CAST(NULL AS ROW(outer_f3 BOOLEAN, struct_map MAP))"), + // part_col + ImmutableList.of("'A'", "'A'", "'B'")); + + // "Zip" results up for convenient lookup + Map> mapping = listsToMap(columnsToTest, expectedRowValues); + + // Determine which table to use base on test parameters + final String sourceTable = isRtTable ? table.getRtTableName() : table.getTableName(); + + // Test each column individually so that errors thrown are more specific/useful + for (String column : mapping.keySet()) { + // Use UNION ALL so that de-dupes will not happen + @Language("SQL") String expectedQuery = mapping.get(column).stream().map(l -> "SELECT " + l).collect(Collectors.joining(" UNION ALL ")); + @Language("SQL") String actualQuery = "SELECT " + column + " FROM " + sourceTable; + + MaterializedResult actualResults = getQueryRunner().execute(session, actualQuery); + MaterializedResult expectedResults = getQueryRunner().execute(session, expectedQuery); + assertThat(actualResults.getMaterializedRows()) + .describedAs("failedQuery: " + actualQuery) + .hasSameSizeAs(expectedResults.getMaterializedRows()) + .containsAll(expectedResults.getMaterializedRows()); + } + + // Perform same test on all columns together + int numRows = expectedRowValues.getFirst().size(); + @Language("SQL") String expectedQuery = IntStream.range(0, numRows) + .mapToObj(rowIndex -> { + // For each row, collect the corresponding values for all columns in the defined order + String rowValuesString = columnsToTest.stream() + .map(columnName -> { + List columnData = mapping.get(columnName); + return columnData.get(rowIndex); + }) + .collect(Collectors.joining(", ")); // Joins column values: "val1, val2, val3" + return "SELECT " + rowValuesString; // Forms: "SELECT val1, val2, val3" + }) + .collect(Collectors.joining(" UNION ALL ")); + @Language("SQL") String actualQuery = "SELECT " + String.join(", ", columnsToTest) + " FROM " + sourceTable; + MaterializedResult actualResults = getQueryRunner().execute(session, actualQuery); + MaterializedResult expectedResults = getQueryRunner().execute(session, expectedQuery); + assertThat(actualResults.getMaterializedRows()) + .hasSameSizeAs(expectedResults.getMaterializedRows()) + .containsAll(expectedResults.getMaterializedRows()); + + // Perform test on selecting nested field + String columnToTest = "col_map_string_struct"; + // 1. Extract all values from the map into an array. Since each map has one entry, this array will have one ROW (or be NULL if the map is NULL). + // 2. Access the first (and only) ROW object from this array. (Using 1-based indexing for arrays, which Trino and Presto uses) + // 3. Access the 'nested_f4' field from that ROW object. + @Language("SQL") String nestedFieldQuery = "SELECT (map_values(" + columnToTest + "))[1].nested_f4 AS extracted_nested_f4 FROM " + sourceTable; + @Language("SQL") String expectedNestedFieldQuery = "WITH " + sourceTable + " AS ( " + + mapping.get(columnToTest).stream() + .map(l -> "SELECT " + l + " AS " + columnToTest) + .collect(Collectors.joining(" UNION ALL ")) + + ") " + + nestedFieldQuery; + @Language("SQL") String actualNestedFieldQuery = nestedFieldQuery; + MaterializedResult expectedNestedResult = getQueryRunner().execute(session, expectedNestedFieldQuery); + MaterializedResult actualNestedResult = getQueryRunner().execute(session, actualNestedFieldQuery); + assertThat(actualNestedResult.getMaterializedRows()) + .hasSameSizeAs(expectedNestedResult.getMaterializedRows()) + .containsAll(expectedNestedResult.getMaterializedRows()); + } + + @Test + public void testHudiPartitionFieldsWithMultipleTypes() + { + Session session = getSession(); + @Language("SQL") String actualQuery = "SELECT part_str, part_int, part_date, part_bigint, part_bool FROM " + HUDI_MULTI_PT_V8_MOR; + @Language("SQL") String expectedQuery = "VALUES " + + "('apparel', 2024, DATE '2024-01-05', 20000000001, false), " + + "('books', 2023, DATE '2023-01-15', 10000000001, true), " + + "('books', 2024, DATE '2024-02-20', 10000000003, true), " + + "('electronics', 2023, DATE '2023-03-10', 10000000002, false), " + + "('electronics', 2023, DATE '2023-03-10', 10000000002, true) "; + assertQuery(session, actualQuery, expectedQuery); + } + + private void testTimestampMicros(HiveTimestampPrecision timestampPrecision, LocalDateTime expected) + throws Exception + { + File parquetFile = new File(Resources.getResource("long_timestamp.parquet").toURI()); + Type columnType = createTimestampType(timestampPrecision.getPrecision()); + HudiSplit hudiSplit = new HudiSplit( + new HudiBaseFile(parquetFile.getPath(), parquetFile.getName(), parquetFile.length(), parquetFile.lastModified(), 0, parquetFile.length()), + ImmutableList.of(), + "000", + TupleDomain.all(), + ImmutableList.of(), + SplitWeight.standard()); + + HudiConfig config = new HudiConfig().setUseParquetColumnNames(false); + HudiSessionProperties sessionProperties = new HudiSessionProperties(config, new ParquetReaderConfig()); + ConnectorSession session = TestingConnectorSession.builder() + .setPropertyMetadata(sessionProperties.getSessionProperties()) + .build(); + + try (ConnectorPageSource pageSource = createPageSource( + session, + List.of(createBaseColumn("created", 0, HIVE_TIMESTAMP, columnType, REGULAR, Optional.empty())), + hudiSplit, + new LocalInputFile(parquetFile), + new FileFormatDataSourceStats(), + new ParquetReaderOptions(), + DateTimeZone.UTC, DynamicFilter.EMPTY, true)) { + MaterializedResult result = materializeSourceDataStream(session, pageSource, List.of(columnType)).toTestTypes(); + assertThat(result.getMaterializedRows()) + .containsOnly(new MaterializedRow(List.of(expected))); + } + } + + private static Pattern getScanFilterInputRowsPattern(String tableIdentifier) + { + // Regex to find the ScanFilter for the specific table that received a dynamic filter and extract the 'Input: X rows' value associated with it. + return Pattern.compile( + // Match the ScanFilter line for the specific table, ensuring dynamicFilters is present + "ScanFilter\\[table = " + Pattern.quote(tableIdentifier) + ".*dynamicFilters = \\{.*?\\}.*?\\]" + + ".*?" + // Match subsequent lines non-greedily until the target line is found + "\\n\\s+Input:\\s+(\\d+)\\s+rows", // Match the 'Input: X rows' line, ensuring it's indented relative to ScanFilter + Pattern.DOTALL); + } + + private TrinoInputFile toInputFile(String path) + { + return ((HudiConnector) getDistributedQueryRunner().getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")) + .newInputFile(Location.of(path)); + } + + public static Map listsToMap(List keys, List values) + { + if (keys == null || values == null) { + throw new IllegalArgumentException("Key and Value lists cannot be null."); + } + + // Determine the number of entries based on the shorter list + int limit = Math.min(keys.size(), values.size()); + + return IntStream.range(0, limit) + .boxed() + .collect(Collectors.toMap( + keys::get, + values::get, + // Merge function for duplicate keys, last one wins + (_, newValue) -> newValue)); + } + + private static Stream comprehensiveTestParameters() + { + ResourceHudiTablesInitializer.TestingTable[] tablesToTest = { + HUDI_COMPREHENSIVE_TYPES_V6_MOR, + HUDI_COMPREHENSIVE_TYPES_V8_MOR + }; + Boolean[] booleanValues = {true, false}; + + return Stream.of(tablesToTest) + .flatMap(table -> + Stream.of(booleanValues) + .map(boolValue -> Arguments.of(table, boolValue))); + } + + /** + * Entry point for finding a node in the complete JSON plan string. + * It iterates through each plan fragment ("0", "1", etc.) and starts the recursive search. + * + * @param jsonPlanString the complete JSON string of the execution plan + * @param nodeType the "name" of the node type to find (e.g., "InnerJoin", "ScanFilterProject") + * @param tableName the name of the table to match for nodes like "ScanFilterProject". can be empty + * @return The found JSONObject, or null if not found in any fragment. + */ + public static JSONObject findNodeInPlan(String jsonPlanString, String nodeType, Option tableName) + { + JSONObject fullPlan = new JSONObject(jsonPlanString); + + // Iterate over the fragment keys ("0", "1", etc.) + Iterator fragmentKeys = fullPlan.keys(); + while (fragmentKeys.hasNext()) { + String key = fragmentKeys.next(); + JSONObject fragmentNode = fullPlan.getJSONObject(key); + + // Start the recursive search from the root node of the fragment + JSONObject result = findNodeRecursive(fragmentNode, nodeType, tableName); + if (result != null) { + return result; + } + } + return null; + } + + /** + * Recursively searches for a node within a plan tree starting from the given node. + * + * @param currentNode the current {@link JSONObject} node in the plan tree to inspect + * @param nodeType the "name" of the node type to find + * @param tableName the table name to match for specific node types. can be empty + * @return the found {@link JSONObject}, or null if not found. + */ + private static JSONObject findNodeRecursive(JSONObject currentNode, String nodeType, Option tableName) + { + if (currentNode.has("name") && currentNode.getString("name").equals(nodeType)) { + // If a tableName is specified, we must match it (for Scan/Filter/Project nodes) + if (tableName.isPresent()) { + if (currentNode.has("descriptor") && currentNode.getJSONObject("descriptor").has("table")) { + String table = currentNode.getJSONObject("descriptor").getString("table"); + if (table.contains(tableName.get())) { + return currentNode; + } + } + } + else { + // If no tableName is required, found a match by nodeType alone + return currentNode; + } + } + + // If not a match, recurse into the children + if (currentNode.has("children")) { + JSONArray children = currentNode.getJSONArray("children"); + for (int i = 0; i < children.length(); i++) { + JSONObject childNode = children.getJSONObject(i); + JSONObject result = findNodeRecursive(childNode, nodeType, tableName); + if (result != null) { + return result; + } + } + } + + return null; + } + + /** + * Searches the "details" array of a given plan node for a string containing specific text. + * + * @param node the {@link JSONObject} plan node to search within + * @param content the substring to search for in the details array + * @return the full text of the first matching detail, or null if no match is found. + */ + public static String findDetailContaining(JSONObject node, String content) + { + if (node != null && node.has("details")) { + JSONArray details = node.getJSONArray("details"); + for (int i = 0; i < details.length(); i++) { + String detailText = details.getString(i); + if (detailText.contains(content)) { + return detailText; + } + } + } + return null; + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSystemTables.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSystemTables.java new file mode 100644 index 0000000000000..1bca5dfaa0dfe --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestHudiSystemTables.java @@ -0,0 +1,61 @@ +/* + * 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.hudi; + +import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import org.junit.jupiter.api.Test; + +public class TestHudiSystemTables + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return HudiQueryRunner.builder() + .setDataLoader(new ResourceHudiTablesInitializer()) + .build(); + } + + @Test + public void testTimelineTable() + { + assertQuery("SHOW COLUMNS FROM tests.\"hudi_cow_pt_tbl$timeline\"", + "VALUES ('timestamp', 'varchar', '', '')," + + "('action', 'varchar', '', '')," + + "('state', 'varchar', '', '')"); + + assertQuery("SELECT timestamp, action, state FROM tests.\"hudi_cow_pt_tbl$timeline\"", + "VALUES ('20220906063435640', 'commit', 'COMPLETED'), ('20220906063456550', 'commit', 'COMPLETED')"); + + assertQueryFails("SELECT timestamp, action, state FROM tests.\"non_existing$timeline\"", + ".*Table 'hudi.tests.\"non_existing\\$timeline\"' does not exist"); + } + + @Test + public void testDataTable() + { + assertQueryFails("SELECT * FROM tests.\"hudi_cow_pt_tbl$data\"", + ".*Table 'hudi.tests.\"hudi_cow_pt_tbl\\$data\"' does not exist"); + } + + @Test + public void testInvalidTable() + { + assertQueryFails("SELECT * FROM tests.\"hudi_cow_pt_tbl$invalid\"", + ".*Table 'hudi.tests.\"hudi_cow_pt_tbl\\$invalid\"' does not exist"); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestingHudiConnectorFactory.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestingHudiConnectorFactory.java new file mode 100644 index 0000000000000..de804a9acd258 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestingHudiConnectorFactory.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.hudi; + +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; +import static io.trino.plugin.hudi.HudiConnectorFactory.createConnector; + +public class TestingHudiConnectorFactory + implements ConnectorFactory +{ + private final Path localFileSystemRootPath; + + public TestingHudiConnectorFactory(Path localFileSystemRootPath) + { + localFileSystemRootPath.toFile().mkdirs(); + this.localFileSystemRootPath = localFileSystemRootPath; + } + + @Override + public String getName() + { + return "hudi"; + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + ImmutableMap.Builder configBuilder = ImmutableMap.builder() + .putAll(config) + .put("bootstrap.quiet", "true"); + if (!config.containsKey("hive.metastore")) { + configBuilder.put("hive.metastore", "file"); + } + return createConnector(catalogName, configBuilder.buildOrThrow(), context, Optional.of(binder -> { + newMapBinder(binder, String.class, TrinoFileSystemFactory.class) + .addBinding("local").toInstance(new LocalFileSystemFactory(localFileSystemRootPath)); + configBinder(binder).bindConfigDefaults(FileHiveMetastoreConfig.class, metastoreConfig -> metastoreConfig.setCatalogDirectory("local:///managed/")); + })); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestingHudiPlugin.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestingHudiPlugin.java new file mode 100644 index 0000000000000..02744377e2dad --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/TestingHudiPlugin.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.hudi; + +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 TestingHudiPlugin + extends HudiPlugin +{ + private final Path localFileSystemRootPath; + + public TestingHudiPlugin(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 TestingHudiConnectorFactory(localFileSystemRootPath)); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/query/index/HudiRecordLevelIndexSupportTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/query/index/HudiRecordLevelIndexSupportTest.java new file mode 100644 index 0000000000000..70101870b478a --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/query/index/HudiRecordLevelIndexSupportTest.java @@ -0,0 +1,484 @@ +/* + * 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.hudi.query.index; + +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.ValueSet; +import io.trino.spi.type.IntegerType; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static io.trino.plugin.hudi.query.index.HudiRecordLevelIndexSupport.extractPredicatesForColumns; +import static io.trino.spi.predicate.Domain.singleValue; +import static io.trino.spi.predicate.Range.range; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static org.assertj.core.api.Assertions.assertThat; + +public class HudiRecordLevelIndexSupportTest +{ + /** + * Example unit test case on how to initialize domains and run tests. + */ + @Test + public void testExtractPredicatesForColumns() + { + // Define column symbols + TestingColumnHandle idColumn = new TestingColumnHandle("id"); + TestingColumnHandle nameColumn = new TestingColumnHandle("name"); + TestingColumnHandle valueColumn = new TestingColumnHandle("value"); + TestingColumnHandle timestampColumn = new TestingColumnHandle("timestamp"); + + // Create domains for columns + Domain idDomain = singleValue(BIGINT, 42L); + Domain nameDomain = singleValue(VARCHAR, Slices.utf8Slice("test")); + Domain valueDomain = Domain.create( + ValueSet.ofRanges(range(BIGINT, 10L, true, 20L, false)), + false); + Domain timestampDomain = singleValue(BIGINT, 1715882800000L); + + // Build TupleDomain with all columns + Map domains = new HashMap<>(); + domains.put(idColumn, idDomain); + domains.put(nameColumn, nameDomain); + domains.put(valueColumn, valueDomain); + domains.put(timestampColumn, timestampDomain); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(domains).transformKeys(TestingColumnHandle::getName); + + // Define the columns we want to extract + List columnsToExtract = List.of(idColumn.getName(), nameColumn.getName()); + + // Extract predicates + TupleDomain extractedDomain = extractPredicatesForColumns(tupleDomain, columnsToExtract); + + // Verify the result + Map extractedDomains = extractedDomain.getDomains().get(); + assertThat(extractedDomains).hasSize(2); + assertThat(extractedDomains).containsKey(idColumn.getName()); + assertThat(extractedDomains).containsKey(nameColumn.getName()); + assertThat(extractedDomains).doesNotContainKey(valueColumn.getName()); + assertThat(extractedDomains).doesNotContainKey(timestampColumn.getName()); + + assertThat(extractedDomains.get(idColumn.getName())).isEqualTo(idDomain); + assertThat(extractedDomains.get(nameColumn.getName())).isEqualTo(nameDomain); + } + + @Test + public void testExtractPredicatesForColumns_None() + { + // When TupleDomain is None + TupleDomain tupleDomain = TupleDomain.none(); + List columns = List.of(); + + TupleDomain result = HudiRecordLevelIndexSupport.extractPredicatesForColumns(tupleDomain, columns); + + assertThat(result.isNone()).isTrue(); + } + + @Test + public void testExtractPredicatesForColumns_All() + { + // When TupleDomain is All + TupleDomain tupleDomain = TupleDomain.all(); + List columns = List.of("col1"); + + TupleDomain result = HudiRecordLevelIndexSupport.extractPredicatesForColumns(tupleDomain, columns); + + assertThat(result.isAll()).isTrue(); + } + + @Test + public void testExtractPredicatesForColumns_WithInClause() + { + // Setup columns + TestingColumnHandle col1 = new TestingColumnHandle("col1"); + TestingColumnHandle col2 = new TestingColumnHandle("col2"); + + // Create a TupleDomain with col1 having multiple values (IN clause) + Map domains = new HashMap<>(); + domains.put(col1, createMultiValueDomain(List.of("value1", "value2", "value3"))); // IN ('value1', 'value2', 'value3') + domains.put(col2, createStringDomain("value4")); // = 'value4' + TupleDomain tupleDomain = TupleDomain.withColumnDomains(domains).transformKeys(TestingColumnHandle::getName); + + // Request predicates for col1 only + List columns = List.of(col1.getName()); + + // Should return TupleDomain with only col1 containing all its values + TupleDomain result = HudiRecordLevelIndexSupport.extractPredicatesForColumns(tupleDomain, columns); + + assertThat(result.isNone()).isFalse(); + assertThat(result.isAll()).isFalse(); + + Map resultDomains = result.getDomains().get(); + assertThat(resultDomains).hasSize(1); + assertThat(resultDomains).containsKey(col1.getName()); + assertThat(resultDomains).doesNotContainKey(col2.getName()); + + List values = getMultiValue(resultDomains.get(col1.getName())); + assertThat(values).hasSize(3); + assertThat(values).contains("value1", "value2", "value3"); + } + + @Test + public void testExtractPredicatesForColumns_MixedPredicates() + { + // Setup columns + TestingColumnHandle col1 = new TestingColumnHandle("col1"); // Simple equality + TestingColumnHandle col2 = new TestingColumnHandle("col2"); // IN clause + TestingColumnHandle col3 = new TestingColumnHandle("col3"); // Range predicate + TestingColumnHandle col4 = new TestingColumnHandle("col4"); // Not part of record key set + + // Create domains with different predicate types + Map domains = new HashMap<>(); + domains.put(col1, createStringDomain("value1")); + domains.put(col2, createMultiValueDomain(List.of("a", "b", "c"))); + + // Create a simple range domain simulation from 10 (inclusive) to 50 (exclusive) + Range range = Range.range(IntegerType.INTEGER, 10L, true, 50L, false); + ValueSet valueSet = ValueSet.ofRanges(range); + Domain rangeDomain = Domain.create(valueSet, false); + domains.put(col3, rangeDomain); + + domains.put(col4, createStringDomain("value4")); + + TupleDomain tupleDomain = TupleDomain.withColumnDomains(domains).transformKeys(TestingColumnHandle::getName); + + // Request predicates for col1, col2, and col3 + List columns = List.of(col1.getName(), col2.getName(), col3.getName()); + + TupleDomain result = HudiRecordLevelIndexSupport.extractPredicatesForColumns(tupleDomain, columns); + + Map resultDomains = result.getDomains().get(); + assertThat(resultDomains).hasSize(3); + + // Check col1 (equality) + List valuesCol1 = getMultiValue(resultDomains.get(col1.getName())); + assertThat(resultDomains).containsKey(col1.getName()); + assertThat(valuesCol1).hasSize(1); + assertThat(valuesCol1.getFirst()).isEqualTo("value1"); + + // Check col2 (IN clause) + List valuesCol2 = getMultiValue(resultDomains.get(col2.getName())); + assertThat(resultDomains).containsKey(col2.getName()); + assertThat(valuesCol2).hasSize(3); + assertThat(valuesCol2).containsAll(Arrays.asList("a", "b", "c")); + + // Check col3 (range) + assertThat(resultDomains).containsKey(col3.getName()); + assertThat(resultDomains.get(col3.getName()).getValues().getRanges().getSpan().getLowValue().get()).isEqualTo(10L); + assertThat(resultDomains.get(col3.getName()).getValues().getRanges().getSpan().getHighValue().get()).isEqualTo(50L); + + // Check col4 (not requested) + assertThat(resultDomains).doesNotContainKey(col4.getName()); + } + + @Test + public void testExtractPredicatesForColumns_NoMatchingColumns() + { + // Setup columns that don't match + TestingColumnHandle col1 = new TestingColumnHandle("col1"); + TestingColumnHandle col2 = new TestingColumnHandle("col2"); + TestingColumnHandle col3 = new TestingColumnHandle("col3"); + + // Create a TupleDomain with col1 and col2 + Map domains = new HashMap<>(); + domains.put(col1, createStringDomain("value1")); + domains.put(col2, createStringDomain("value2")); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(domains).transformKeys(TestingColumnHandle::getName); + + // Request predicates for col3 only + List columns = List.of(col3.getName()); + + // Since no matching columns, should return TupleDomain.all() + TupleDomain result = HudiRecordLevelIndexSupport.extractPredicatesForColumns(tupleDomain, columns); + + // domains.isPresent() && domains.get().isEmpty() + assertThat(result.isAll()).isTrue(); + } + + @Test + public void testExtractPredicatesForColumns_PartialMatchingColumns() + { + // Setup columns + TestingColumnHandle col1 = new TestingColumnHandle("col1"); + TestingColumnHandle col2 = new TestingColumnHandle("col2"); + TestingColumnHandle col3 = new TestingColumnHandle("col3"); + + // Create a TupleDomain with col1, col2, and col3 + Map domains = new HashMap<>(); + domains.put(col1, createStringDomain("value1")); + domains.put(col2, createStringDomain("value2")); + domains.put(col3, createStringDomain("value3")); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(domains).transformKeys(TestingColumnHandle::getName); + + // Request predicates for col1 and col3 only + List columns = List.of(col1.getName(), col3.getName()); + + // Should return TupleDomain with only col1 and col3 + TupleDomain result = HudiRecordLevelIndexSupport.extractPredicatesForColumns(tupleDomain, columns); + + assertThat(result.isNone()).isFalse(); + assertThat(result.isAll()).isFalse(); + + Map resultDomains = result.getDomains().get(); + assertThat(resultDomains).hasSize(2); + assertThat(resultDomains).containsKey(col1.getName()); + assertThat(resultDomains).containsKey(col3.getName()); + assertThat(resultDomains).doesNotContainKey(col2.getName()); + + assertThat(getSingleValue(resultDomains.get(col1.getName()))).isEqualTo("value1"); + assertThat(getSingleValue(resultDomains.get(col3.getName()))).isEqualTo("value3"); + } + + @Test + public void testExtractPredicatesForColumns_AllMatchingColumns() + { + // Setup columns + TestingColumnHandle col1 = new TestingColumnHandle("col1"); + TestingColumnHandle col2 = new TestingColumnHandle("col2"); + + // Create a TupleDomain with col1 and col2 + Map domains = new HashMap<>(); + domains.put(col1, createStringDomain("value1")); + domains.put(col2, createStringDomain("value2")); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(domains).transformKeys(TestingColumnHandle::getName); + + // Request predicates for all columns + List columns = List.of(col1.getName(), col2.getName()); + + // Should return the original TupleDomain + TupleDomain result = HudiRecordLevelIndexSupport.extractPredicatesForColumns(tupleDomain, columns); + + assertThat(result.isNone()).isFalse(); + assertThat(result.isAll()).isFalse(); + + Map resultDomains = result.getDomains().get(); + assertThat(resultDomains).hasSize(2); + assertThat(resultDomains).containsKey(col1.getName()); + assertThat(resultDomains).containsKey(col2.getName()); + + assertThat(getSingleValue(resultDomains.get(col1.getName()))).isEqualTo("value1"); + assertThat(getSingleValue(resultDomains.get(col2.getName()))).isEqualTo("value2"); + } + + @Test + public void testConstructRecordKeys_WithInClause() + { + // Domain with multiple values for a key simulating an IN clause + Map domains = new HashMap<>(); + domains.put("key1", createMultiValueDomain(List.of("value1", "value2", "value3"))); + TupleDomain recordKeyDomains = TupleDomain.withColumnDomains(domains); + List recordKeys = List.of("key1"); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + // The code should take the first value from the IN list for building the key + assertThat(result).hasSize(3); + assertThat(result).isEqualTo(List.of("value1", "value2", "value3")); + } + + @Test + public void testConstructRecordKeys_ComplexKeyWithInClause() + { + Map domains = new HashMap<>(); + // Domain with multiple values for the first key + domains.put("part1", createMultiValueDomain(List.of("val1a", "val1b", "val1c"))); + // Regular single-value domain for second key + domains.put("part2", createStringDomain("value2")); + + TupleDomain recordKeyDomains = TupleDomain.withColumnDomains(domains); + List recordKeys = List.of("part1", "part2"); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + assertThat(result).hasSize(3); + // Expecting the first value of the IN clause to be used + assertThat(result.get(0)).isEqualTo("part1:val1a,part2:value2"); + } + + @Test + public void testConstructRecordKeys_MultipleKeysWithMultipleValues() + { + Map domains = new HashMap<>(); + // Multiple IN clauses + domains.put("part1", createMultiValueDomain(List.of("val1a", "val1b", "val1c"))); + domains.put("part2", createMultiValueDomain(List.of("val2a", "val2b"))); + + TupleDomain recordKeyDomains = TupleDomain.withColumnDomains(domains); + List recordKeys = List.of("part1", "part2"); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + // Verify only the first value from each IN clause is used + assertThat(result).hasSize(6); + assertThat(result).isEqualTo( + List.of("part1:val1a,part2:val2a", "part1:val1a,part2:val2b", + "part1:val1b,part2:val2a", "part1:val1b,part2:val2b", + "part1:val1c,part2:val2a", "part1:val1c,part2:val2b")); + } + + @Test + public void testConstructRecordKeys_MultipleKeysWithRange() + { + Map domains = new HashMap<>(); + // Multiple IN clauses + domains.put("part1", createMultiValueDomain(List.of("val1a", "val1b", "val1c"))); + domains.put("part2", createMultiValueDomain(List.of("val2a", "val2b"))); + + Range range = Range.range(IntegerType.INTEGER, 10L, true, 50L, false); + ValueSet valueSet = ValueSet.ofRanges(range); + Domain rangeDomain = Domain.create(valueSet, false); + domains.put("part3", rangeDomain); + + TupleDomain recordKeyDomains = TupleDomain.withColumnDomains(domains); + List recordKeys = List.of("part1", "part2", "part3"); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + // Can only handle IN and EQUAL cases + assertThat(result).isEmpty(); + } + + @Test + public void testConstructRecordKeys_NullRecordKeys() + { + TupleDomain recordKeyDomains = createStringTupleDomain(Map.of("key1", "value1")); + List recordKeys = null; + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + assertThat(result).isEmpty(); + } + + @Test + public void testConstructRecordKeys_EmptyRecordKeys() + { + TupleDomain recordKeyDomains = createStringTupleDomain(Map.of("key1", "value1")); + List recordKeys = Collections.emptyList(); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + assertThat(result).isEmpty(); + } + + @Test + public void testConstructRecordKeys_EmptyDomains() + { + TupleDomain recordKeyDomains = TupleDomain.withColumnDomains(Collections.emptyMap()); + List recordKeys = List.of("key1"); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + assertThat(result).isEmpty(); + } + + @Test + public void testConstructRecordKeys_MissingDomainForKey() + { + TupleDomain recordKeyDomains = createStringTupleDomain(Map.of("key1", "value1")); + List recordKeys = List.of("key2"); // Key not in domains + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + assertThat(result).isEmpty(); + } + + @Test + public void testConstructRecordKeys_SingleKey() + { + TupleDomain recordKeyDomains = createStringTupleDomain(Map.of("key1", "value1")); + List recordKeys = List.of("key1"); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + assertThat(result).hasSize(1); + assertThat(result.getFirst()).isEqualTo("value1"); + } + + @Test + public void testConstructRecordKeys_ComplexKey() + { + Map keyValues = new HashMap<>(); + keyValues.put("part1", "value1"); + keyValues.put("part2", "value2"); + keyValues.put("part3", "value3"); + + TupleDomain recordKeyDomains = createStringTupleDomain(keyValues); + List recordKeys = List.of("part1", "part2", "part3"); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + assertThat(result).hasSize(1); + assertThat(result.getFirst()).isEqualTo("part1:value1,part2:value2,part3:value3"); + } + + @Test + public void testConstructRecordKeys_ComplexKeyWithMissingPart() + { + Map keyValues = new HashMap<>(); + keyValues.put("part1", "value1"); + // part2 is missing + keyValues.put("part3", "value3"); + + TupleDomain recordKeyDomains = createStringTupleDomain(keyValues); + List recordKeys = List.of("part1", "part2", "part3"); + + List result = HudiRecordLevelIndexSupport.constructRecordKeys(recordKeyDomains, recordKeys); + + // Since one key is missing, should return empty list + assertThat(result).isEmpty(); + } + + // Helper methods for test data creation + private Domain createStringDomain(String value) + { + return Domain.singleValue(VARCHAR, Slices.utf8Slice(value)); + } + + private Domain createMultiValueDomain(List values) + { + return Domain.multipleValues(VARCHAR, values.stream().map(Slices::utf8Slice).collect(Collectors.toList())); + } + + private TupleDomain createStringTupleDomain(Map keyValues) + { + Map domains = keyValues.entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> createStringDomain(entry.getValue()))); + return TupleDomain.withColumnDomains(domains); + } + + private String getSingleValue(Domain domain) + { + return ((Slice) domain.getSingleValue()).toStringUtf8(); + } + + private List getMultiValue(Domain domain) + { + return domain.getNullableDiscreteSet().getNonNullValues().stream() + .map(x -> ((Slice) x) + .toStringUtf8()).toList(); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/query/index/TestingColumnHandle.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/query/index/TestingColumnHandle.java new file mode 100644 index 0000000000000..98f11e350f0b0 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/query/index/TestingColumnHandle.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.hudi.query.index; + +import io.trino.spi.connector.ColumnHandle; + +/** + * Test implementation of ColumnHandle for unit tests + */ +public class TestingColumnHandle + implements ColumnHandle +{ + private final String name; + + public TestingColumnHandle(String name) + { + this.name = name; + } + + public String getName() + { + return name; + } + + @Override + public int hashCode() + { + return name.hashCode(); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + TestingColumnHandle other = (TestingColumnHandle) obj; + return name.equals(other.name); + } + + @Override + public String toString() + { + return "Column:" + name; + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/split/TestHudiSplitFactory.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/split/TestHudiSplitFactory.java new file mode 100644 index 0000000000000..37d4eac6e88b1 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/split/TestHudiSplitFactory.java @@ -0,0 +1,202 @@ +/* + * 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.hudi.split; + +import com.google.common.collect.ImmutableList; +import io.airlift.units.DataSize; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hudi.HudiSplit; +import io.trino.plugin.hudi.HudiTableHandle; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathInfo; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestHudiSplitFactory +{ + private static final String COMMIT_TIME = "20250625153731546"; + private static final List PARTITION_KEYS = ImmutableList.of(); + + @Test + public void testCreateHudiSplitsWithSmallBaseFile() + { + // Test with 20MB target split size and 10MB base file + // - should create 1 split + testSplitCreation( + DataSize.of(20, MEGABYTE), + DataSize.of(10, MEGABYTE), + Option.empty(), + ImmutableList.of( + Pair.of(0L, DataSize.of(10, MEGABYTE)))); + } + + @Test + public void testCreateHudiSplitsWithExactSplitDivide() + { + // Test with 20MB target and 60MB base file + // - should create 3 splits + testSplitCreation( + DataSize.of(20, MEGABYTE), + DataSize.of(60, MEGABYTE), + Option.empty(), + ImmutableList.of( + Pair.of(0L, DataSize.of(20, MEGABYTE)), + Pair.of(DataSize.of(20, MEGABYTE).toBytes(), DataSize.of(20, MEGABYTE)), + Pair.of(DataSize.of(40, MEGABYTE).toBytes(), DataSize.of(20, MEGABYTE)))); + } + + @Test + public void testCreateHudiSplitsWithSlightlyOversizedFile() + { + // Test with 20MB target and 61MB base file + // - should create 3 splits (61/20 = 3.05, 0.05 is within split slop of 0.1) + testSplitCreation( + DataSize.of(20, MEGABYTE), + DataSize.of(61, MEGABYTE), + Option.empty(), + ImmutableList.of( + Pair.of(0L, DataSize.of(20, MEGABYTE)), + Pair.of(DataSize.of(20, MEGABYTE).toBytes(), DataSize.of(20, MEGABYTE)), + Pair.of(DataSize.of(40, MEGABYTE).toBytes(), DataSize.of(21, MEGABYTE)))); + } + + @Test + public void testCreateHudiSplitsWithOversizedFileExceedingSlop() + { + // Test with 20MB target and 65MB base file + // - should create 4 splits (65/20 = 3.25) + testSplitCreation( + DataSize.of(20, MEGABYTE), + DataSize.of(65, MEGABYTE), + Option.empty(), + ImmutableList.of( + Pair.of(0L, DataSize.of(20, MEGABYTE)), + Pair.of(DataSize.of(20, MEGABYTE).toBytes(), DataSize.of(20, MEGABYTE)), + Pair.of(DataSize.of(40, MEGABYTE).toBytes(), DataSize.of(20, MEGABYTE)), + Pair.of(DataSize.of(60, MEGABYTE).toBytes(), DataSize.of(5, MEGABYTE)))); + } + + @Test + public void testCreateHudiSplitsWithLargerBlockSize() + { + // Test with 1MB target split size and 32MB base file + // - should create 4 splits because the block size of 8MB is larger than the target split size + testSplitCreation( + DataSize.of(1, MEGABYTE), + DataSize.of(32, MEGABYTE), + Option.empty(), + ImmutableList.of( + Pair.of(0L, DataSize.of(8, MEGABYTE)), + Pair.of(DataSize.of(8, MEGABYTE).toBytes(), DataSize.of(8, MEGABYTE)), + Pair.of(DataSize.of(16, MEGABYTE).toBytes(), DataSize.of(8, MEGABYTE)), + Pair.of(DataSize.of(24, MEGABYTE).toBytes(), DataSize.of(8, MEGABYTE)))); + } + + @Test + public void testCreateHudiSplitsWithLogFile() + { + // Test with 20MB target and 65MB base file and 10MB log file + // - should create 1 split regardless of size + testSplitCreation( + DataSize.of(20, MEGABYTE), + DataSize.of(65, MEGABYTE), + Option.of(DataSize.of(10, MEGABYTE)), + ImmutableList.of( + Pair.of(0L, DataSize.of(65, MEGABYTE)))); + } + + @Test + public void testCreateHudiSplitsWithZeroSizeFile() + { + // Test with zero-size file - should create 1 split with zero size + testSplitCreation( + DataSize.of(128, MEGABYTE), + DataSize.of(0, MEGABYTE), + Option.empty(), + ImmutableList.of(Pair.of(0L, DataSize.of(0, MEGABYTE)))); + } + + private static void testSplitCreation( + DataSize targetSplitSize, + DataSize baseFileSize, + Option logFileSize, + List> expectedSplitInfo) + { + HudiTableHandle tableHandle = createTableHandle(); + HudiSplitWeightProvider weightProvider = new SizeBasedSplitWeightProvider(0.05, DataSize.of(128, MEGABYTE)); + + FileSlice fileSlice = createFileSlice(baseFileSize, logFileSize); + + List splits = HudiSplitFactory.createHudiSplits( + tableHandle, PARTITION_KEYS, fileSlice, COMMIT_TIME, weightProvider, targetSplitSize); + + assertThat(splits).hasSize(expectedSplitInfo.size()); + + for (int i = 0; i < expectedSplitInfo.size(); i++) { + HudiSplit split = splits.get(i); + assertThat(split.getBaseFile()).isPresent(); + assertThat(split.getBaseFile().get().getFileSize()).isEqualTo(baseFileSize.toBytes()); + assertThat(split.getBaseFile().get().getStart()) + .isEqualTo(expectedSplitInfo.get(i).getLeft()); + assertThat(split.getBaseFile().get().getLength()) + .isEqualTo(expectedSplitInfo.get(i).getRight().toBytes()); + assertThat(split.getCommitTime()).isEqualTo(COMMIT_TIME); + assertThat(split.getLogFiles().size()).isEqualTo(logFileSize.isPresent() ? 1 : 0); + long totalSize = logFileSize.isPresent() ? + baseFileSize.toBytes() + logFileSize.get().toBytes() : expectedSplitInfo.get(i).getRight().toBytes(); + assertThat(split.getSplitWeight()).isEqualTo(weightProvider.calculateSplitWeight(totalSize)); + } + } + + private static HudiTableHandle createTableHandle() + { + return new HudiTableHandle( + "test_schema", + "test_table", + "/test/path", + HoodieTableType.MERGE_ON_READ, + ImmutableList.of(), + TupleDomain.all(), + TupleDomain.all()); + } + + private static FileSlice createFileSlice(DataSize baseFileSize, Option logFileSize) + { + String fileId = "5a4f6a70-0306-40a8-952b-045b0d8ff0d4-0"; + HoodieFileGroupId fileGroupId = new HoodieFileGroupId("partition", fileId); + long blockSize = 8L * 1024 * 1024; + String baseFilePath = "/test/path/" + fileGroupId + "_4-19-0_" + COMMIT_TIME + ".parquet"; + String logFilePath = "/test/path/." + fileId + "_2025062515374131546.log.1_0-53-80"; + StoragePathInfo baseFileInfo = new StoragePathInfo( + new StoragePath(baseFilePath), baseFileSize.toBytes(), false, (short) 0, blockSize, System.currentTimeMillis()); + StoragePathInfo logFileInfo = new StoragePathInfo( + new StoragePath(logFilePath), logFileSize.isPresent() ? logFileSize.get().toBytes() : 0L, + false, (short) 0, blockSize, System.currentTimeMillis()); + HoodieBaseFile baseFile = new HoodieBaseFile(baseFileInfo); + return new FileSlice(fileGroupId, COMMIT_TIME, baseFile, + logFileSize.isPresent() ? ImmutableList.of(new HoodieLogFile(logFileInfo)) : ImmutableList.of()); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTableUnzipper.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTableUnzipper.java new file mode 100644 index 0000000000000..ac1405819127d --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTableUnzipper.java @@ -0,0 +1,125 @@ +/* + * 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.hudi.testing; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; +import java.util.stream.Stream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; + +import static com.google.common.io.Resources.getResource; +import static java.util.Objects.requireNonNull; + +public class HudiTableUnzipper +{ + private static final String ZIP_EXT = ".zip"; + + private HudiTableUnzipper() {} + + public static void unzipAllItemsInResource(String resourceName, Path outputPath) + throws IOException, URISyntaxException + { + requireNonNull(resourceName, "Resource name cannot be null or empty."); + + URL resourceUrl = HudiTableUnzipper.class.getClassLoader().getResource(resourceName); + if (resourceUrl == null) { + throw new IOException("Resource not found: " + resourceName); + } + + for (File file : Path.of(getResource(resourceName).toURI()).toFile().listFiles()) { + if (file.isFile() && file.getName().endsWith(ZIP_EXT)) { + // Only handle zip files + unzipFile(file.toURI().toURL(), outputPath); + } + } + } + + private static void unzipFile(URL resourceUrl, Path targetDirectory) + throws IOException + { + try (InputStream is = resourceUrl.openStream(); ZipInputStream zis = new ZipInputStream(is)) { + ZipEntry zipEntry = zis.getNextEntry(); + byte[] buffer = new byte[1024]; + + while (zipEntry != null) { + Path newFilePath = targetDirectory.resolve(zipEntry.getName()); + + // Prevent Zip Slip vulnerability (Do not want files written outside the our target dir) + if (!newFilePath.normalize().startsWith(targetDirectory.normalize())) { + throw new IOException("Bad zip entry: " + zipEntry.getName()); + } + + if (zipEntry.isDirectory()) { + // Handle directories + if (!Files.exists(newFilePath)) { + Files.createDirectories(newFilePath); + } + } + else { + // Ensure parent directory exists before handling files + Path parentDir = newFilePath.getParent(); + if (parentDir != null && !Files.exists(parentDir)) { + Files.createDirectories(parentDir); + } + + try (FileOutputStream fos = new FileOutputStream(newFilePath.toFile())) { + int len; + while ((len = zis.read(buffer)) > 0) { + fos.write(buffer, 0, len); + } + } + } + zis.closeEntry(); + zipEntry = zis.getNextEntry(); + } + } + } + + public static void deleteInflatedFiles(Path path) + throws URISyntaxException, IOException + { + requireNonNull(path, "path cannot be null or empty."); + for (File file : path.toFile().listFiles()) { + // Ignore all zip files + if (file.isFile() && file.getName().endsWith(ZIP_EXT)) { + continue; + } + // Not really required, as we are in the test-classes directory + // Ensure that we are only deleting deflated folders of zip + if (path.resolve(file.getName() + ZIP_EXT).toFile().exists()) { + deleteFilesInDirectory(file.toPath()); + } + } + } + + private static void deleteFilesInDirectory(Path pathToDelete) + throws IOException + { + // Recursively delete all files in path + try (Stream filesInDir = Files.walk(pathToDelete)) { + // Reverse order to delete all children before parent + filesInDir.sorted(Comparator.reverseOrder()) + .map(Path::toFile) + .forEach(File::delete); + } + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTablesInitializer.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTablesInitializer.java new file mode 100644 index 0000000000000..4c8788c40e283 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTablesInitializer.java @@ -0,0 +1,23 @@ +/* + * 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.hudi.testing; + +import io.trino.filesystem.Location; +import io.trino.testing.QueryRunner; + +public interface HudiTablesInitializer +{ + void initializeTables(QueryRunner queryRunner, Location externalLocation, String schemaName) + throws Exception; +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTestUtils.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTestUtils.java new file mode 100644 index 0000000000000..26d8d663237db --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/HudiTestUtils.java @@ -0,0 +1,29 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; + +import static io.trino.plugin.hudi.testing.TpchHudiTablesInitializer.FIELD_UUID; +import static org.apache.hudi.common.model.HoodieRecord.HOODIE_META_COLUMNS; + +public final class HudiTestUtils +{ + private HudiTestUtils() {} + + public static final String COLUMNS_TO_HIDE = String.join(",", ImmutableList.builder() + .addAll(HOODIE_META_COLUMNS) + .add(FIELD_UUID) + .build()); +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java new file mode 100644 index 0000000000000..5783da409e569 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java @@ -0,0 +1,753 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.Column; +import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveType; +import io.trino.metastore.Partition; +import io.trino.metastore.PartitionStatistics; +import io.trino.metastore.PartitionWithStatistics; +import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.StorageFormat; +import io.trino.metastore.Table; +import io.trino.plugin.hudi.HudiConnector; +import io.trino.plugin.hudi.storage.HudiTrinoStorage; +import io.trino.plugin.hudi.storage.TrinoStorageConfiguration; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.testing.QueryRunner; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.security.DigestInputStream; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static io.trino.hive.formats.HiveClassNames.HUDI_PARQUET_INPUT_FORMAT; +import static io.trino.hive.formats.HiveClassNames.HUDI_PARQUET_REALTIME_INPUT_FORMAT; +import static io.trino.hive.formats.HiveClassNames.MAPRED_PARQUET_OUTPUT_FORMAT_CLASS; +import static io.trino.hive.formats.HiveClassNames.PARQUET_HIVE_SERDE_CLASS; +import static io.trino.metastore.HiveType.HIVE_BINARY; +import static io.trino.metastore.HiveType.HIVE_BOOLEAN; +import static io.trino.metastore.HiveType.HIVE_BYTE; +import static io.trino.metastore.HiveType.HIVE_DATE; +import static io.trino.metastore.HiveType.HIVE_DOUBLE; +import static io.trino.metastore.HiveType.HIVE_FLOAT; +import static io.trino.metastore.HiveType.HIVE_INT; +import static io.trino.metastore.HiveType.HIVE_LONG; +import static io.trino.metastore.HiveType.HIVE_SHORT; +import static io.trino.metastore.HiveType.HIVE_STRING; +import static io.trino.metastore.HiveType.HIVE_TIMESTAMP; +import static io.trino.metastore.type.TypeInfoFactory.getDecimalTypeInfo; +import static io.trino.metastore.type.TypeInfoFactory.getListTypeInfo; +import static io.trino.metastore.type.TypeInfoFactory.getStructTypeInfo; +import static io.trino.plugin.hive.HivePartitionManager.extractPartitionValues; +import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.ARRAY_BOOLEAN_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.ARRAY_DOUBLE_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.ARRAY_INT_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.ARRAY_STRING_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.BOOLEAN_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.DATE_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.DOUBLE_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.INT_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.MAP_STRING_DATE_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.MAP_STRING_INT_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.MAP_STRING_LONG_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.STRING_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.TIMESTAMP_TYPE_INFO; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.charHiveType; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.decimalHiveType; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.listHiveType; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.mapHiveType; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.structHiveType; +import static io.trino.plugin.hudi.testing.TypeInfoHelper.varcharHiveType; + +public class ResourceHudiTablesInitializer + implements HudiTablesInitializer +{ + private static final Logger log = Logger.get(ResourceHudiTablesInitializer.class); + private static final String HASH_ALGORITHM = "SHA-256"; + private static final String TEST_RESOURCE_NAME = "hudi-testing-data"; + + // There might be other entry points that are using this initializer, make the location unique so it is more identifiable via logs + private final String baseLocationPrefix = UUID.randomUUID().toString(); + private final Path tempDir; + + /** + * Manually declaring a temp directory here and performing a manual cleanup as this constructor is invoked in HudiQueryRunner in a @BeforeAll static function. + * This means that jupiter's managed @TempDir annotation cannot be used as the path will be passed as null. + */ + public ResourceHudiTablesInitializer() + { + // There are multiple entry points and they may perform unzipping together, ensure that they are all unzipping to different paths + try { + this.tempDir = Files.createTempDirectory(TEST_RESOURCE_NAME + "_" + baseLocationPrefix); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void initializeTables(QueryRunner queryRunner, Location externalLocation, String schemaName) + throws Exception + { + // Inflate all deflated test resource archives to a temporary directory + HudiTableUnzipper.unzipAllItemsInResource(TEST_RESOURCE_NAME, tempDir); + TrinoFileSystem fileSystem = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + String locationSuffix = schemaName + "_" + baseLocationPrefix; + Location baseLocation = externalLocation.appendSuffix(locationSuffix); + log.info("Initialized test resource directory as: %s", baseLocation.toString()); + copyDir(tempDir, fileSystem, baseLocation); + // Perform a cleanup + HudiTableUnzipper.deleteInflatedFiles(tempDir); + + for (TestingTable table : TestingTable.values()) { + String tableName = table.getTableName(); + Location tablePath = baseLocation.appendPath(tableName); + + // Always create ro table by default + createTable( + queryRunner, + schemaName, + tablePath, + tableName, + table.getDataColumns(), + table.getPartitionColumns(), + table.getPartitions(), + false); + + if (table.isCreateRtTable()) { + createTable( + queryRunner, + schemaName, + tablePath, + table.getRtTableName(), + table.getDataColumns(), + table.getPartitionColumns(), + table.getPartitions(), + true); + } + + // Set table version + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setStorage(new HudiTrinoStorage(fileSystem, new TrinoStorageConfiguration())) + .setBasePath(tablePath.toString()) + .build(); + table.setTableVersion(metaClient.getTableConfig().getTableVersion()); + } + } + + private void createTable( + QueryRunner queryRunner, + String schemaName, + Location tablePath, + String tableName, + List dataColumns, + List partitionColumns, + Map partitions, + boolean isRtTable) + { + StorageFormat roStorageFormat = StorageFormat.create( + PARQUET_HIVE_SERDE_CLASS, + HUDI_PARQUET_INPUT_FORMAT, + MAPRED_PARQUET_OUTPUT_FORMAT_CLASS); + + StorageFormat rtStorageFormat = StorageFormat.create( + PARQUET_HIVE_SERDE_CLASS, + HUDI_PARQUET_REALTIME_INPUT_FORMAT, + MAPRED_PARQUET_OUTPUT_FORMAT_CLASS); + + Table table = Table.builder() + .setDatabaseName(schemaName) + .setTableName(tableName) + .setTableType(EXTERNAL_TABLE.name()) + .setOwner(Optional.of("public")) + .setDataColumns(dataColumns) + .setPartitionColumns(partitionColumns) + .setParameters(ImmutableMap.of("serialization.format", "1", "EXTERNAL", "TRUE")) + .withStorage(storageBuilder -> storageBuilder + .setStorageFormat(isRtTable ? rtStorageFormat : roStorageFormat) + .setLocation(tablePath.toString())) + .build(); + + HiveMetastore metastore = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + metastore.createTable(table, PrincipalPrivileges.NO_PRIVILEGES); + + List partitionsToAdd = new ArrayList<>(); + partitions.forEach((partitionName, partitionPath) -> { + Partition partition = Partition.builder() + .setDatabaseName(schemaName) + .setTableName(tableName) + .setValues(extractPartitionValues(partitionName)) + .withStorage(storageBuilder -> storageBuilder + .setStorageFormat(isRtTable ? rtStorageFormat : roStorageFormat) + .setLocation(tablePath.appendPath(partitionPath).toString())) + .setColumns(dataColumns) + .build(); + partitionsToAdd.add(new PartitionWithStatistics(partition, partitionName, PartitionStatistics.empty())); + }); + metastore.addPartitions(schemaName, tableName, partitionsToAdd); + } + + private static Column column(String name, HiveType type) + { + return new Column(name, type, Optional.empty(), Map.of()); + } + + public static void copyDir(Path sourceDirectory, TrinoFileSystem fileSystem, Location destinationDirectory) + throws IOException + { + try (Stream paths = Files.walk(sourceDirectory)) { + for (Iterator iterator = paths.iterator(); iterator.hasNext(); ) { + Path path = iterator.next(); + if (path.toFile().isDirectory()) { + continue; + } + + // hudi blows up if crc files are present + if (path.toString().endsWith(".crc")) { + continue; + } + + HashAndSizeResult srcHashAndSize; + try { + srcHashAndSize = calculateHashAndSize(path); + } + catch (NoSuchAlgorithmException e) { + throw new IOException("Failed to calculate source hash: Algorithm not found", e); + } + + Location location = destinationDirectory.appendPath(sourceDirectory.relativize(path).toString()); + fileSystem.createDirectory(location.parentDirectory()); + try (OutputStream out = fileSystem.newOutputFile(location).create()) { + Files.copy(path, out); + // Flush all data before close() to ensure durability + out.flush(); + } + + HashAndSizeResult dstHashAndSize; + try { + dstHashAndSize = calculateHashAndSize(location, fileSystem); + } + catch (NoSuchAlgorithmException e) { + throw new IOException("Failed to calculate destination hash: Algorithm not found", e); + } + catch (IOException e) { + throw new IOException("Failed to read back " + location + " for hash verification", e); + } + + if (!Arrays.equals(srcHashAndSize.hash, dstHashAndSize.hash)) { + // Hashes do not match, file is corrupt or copy failed + String errorMessage = String.format( + "Hash mismatch for file: %s (source size: %d bytes) copied to %s (destination size: %d bytes). Content hashes differ", + path, + srcHashAndSize.size, + location, + dstHashAndSize.size); + throw new IOException(errorMessage); + } + } + } + } + + /** + * Helper method to calculate hash and size from an input stream + */ + private static HashAndSizeResult calculateHashAndSize(InputStream inputStream) + throws IOException, NoSuchAlgorithmException + { + MessageDigest md = MessageDigest.getInstance(HASH_ALGORITHM); + try (DigestInputStream dis = new DigestInputStream(inputStream, md)) { + byte[] buffer = new byte[8192]; + int bytesRead; + long streamSize = 0; + while ((bytesRead = dis.read(buffer)) != -1) { + streamSize += bytesRead; + } + return new HashAndSizeResult(md.digest(), streamSize); + } + } + + /** + * Helper method to calculate hash for a local Path + */ + private static HashAndSizeResult calculateHashAndSize(Path path) + throws IOException, NoSuchAlgorithmException + { + try (InputStream is = Files.newInputStream(path)) { + return calculateHashAndSize(is); + } + } + + /** + * Helper method to calculate hash for a file on TrinoFileSystem + */ + private static HashAndSizeResult calculateHashAndSize(Location location, TrinoFileSystem fileSystem) + throws IOException, NoSuchAlgorithmException + { + try (InputStream is = fileSystem.newInputFile(location).newStream()) { + return calculateHashAndSize(is); + } + } + + public enum TestingTable + { + HUDI_NON_PART_COW(nonPartitionRegularColumns()), + HUDI_TRIPS_COW_V8(tripsRegularColumns()), + HUDI_COW_PT_TBL(multiPartitionRegularColumns(), multiPartitionColumns(), multiPartitions(), false), + STOCK_TICKS_COW(stockTicksRegularColumns(), stockTicksPartitionColumns(), stockTicksPartitions(), false), + STOCK_TICKS_MOR(stockTicksRegularColumns(), stockTicksPartitionColumns(), stockTicksPartitions(), false), + HUDI_STOCK_TICKS_COW(hudiStockTicksRegularColumns(), hudiStockTicksPartitionColumns(), hudiStockTicksPartitions(), false), + HUDI_STOCK_TICKS_MOR(hudiStockTicksRegularColumns(), hudiStockTicksPartitionColumns(), hudiStockTicksPartitions(), false), + HUDI_MULTI_FG_PT_V6_MOR(hudiMultiFgRegularColumns(), hudiMultiFgPartitionsColumn(), hudiMultiFgPartitions(), false), + HUDI_MULTI_FG_PT_V8_MOR(hudiMultiFgRegularColumns(), hudiMultiFgPartitionsColumn(), hudiMultiFgPartitions(), false), + HUDI_COMPREHENSIVE_TYPES_V6_MOR(hudiComprehensiveTypesColumns(), hudiComprehensiveTypesPartitionColumns(), hudiComprehensiveTypesPartitions(), true), + HUDI_COMPREHENSIVE_TYPES_V8_MOR(hudiComprehensiveTypesColumns(), hudiComprehensiveTypesPartitionColumns(), hudiComprehensiveTypesPartitions(), true), + HUDI_MULTI_PT_V8_MOR(hudiMultiPtMorColumns(), hudiMultiPtMorPartitionColumns(), hudiMultiPtMorPartitions(), false), + HUDI_TIMESTAMP_KEYGEN_PT_EPOCH_TO_YYYY_MM_DD_HH_V8_MOR(hudiTimestampKeygenColumns(), hudiTimestampKeygenPartitionColumns(), hudiTimestampKeygenPartitions("EPOCHMILLISECONDS"), true), + HUDI_TIMESTAMP_KEYGEN_PT_SCALAR_TO_YYYY_MM_DD_HH_V8_MOR(hudiTimestampKeygenColumns(), hudiTimestampKeygenPartitionColumns(), hudiTimestampKeygenPartitions("SCALAR"), true), + HUDI_CUSTOM_KEYGEN_PT_V8_MOR(hudiCustomKeyGenColumns(), hudiCustomKeyGenPartitionColumns(), hudiCustomKeyGenPartitions(), false), + /**/; + + private static final List HUDI_META_COLUMNS = ImmutableList.of( + new Column("_hoodie_commit_time", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_commit_seqno", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_record_key", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_partition_path", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_file_name", HIVE_STRING, Optional.empty(), Map.of())); + + private final List regularColumns; + private final List partitionColumns; + private final Map partitions; + private final boolean isCreateRtTable; + private HoodieTableVersion tableVersion; + + TestingTable( + List regularColumns, + List partitionColumns, + Map partitions, + boolean isCreateRtTable) + { + this.regularColumns = regularColumns; + this.partitionColumns = partitionColumns; + this.partitions = partitions; + this.isCreateRtTable = isCreateRtTable; + } + + TestingTable(List regularColumns) + { + this(regularColumns, ImmutableList.of(), ImmutableMap.of(), false); + } + + public String getTableName() + { + return name().toLowerCase(Locale.ROOT); + } + + public String getRtTableName() + { + return name().toLowerCase(Locale.ROOT) + "_rt"; + } + + public String getRoTableName() + { + // ro tables do not have suffix + return getTableName(); + } + + public void setTableVersion(HoodieTableVersion tableVersion) + { + this.tableVersion = tableVersion; + } + + public HoodieTableVersion getHoodieTableVersion() + { + return this.tableVersion; + } + + public List getDataColumns() + { + return Stream.of(HUDI_META_COLUMNS, regularColumns) + .flatMap(Collection::stream) + .collect(Collectors.toUnmodifiableList()); + } + + public List getPartitionColumns() + { + return partitionColumns; + } + + public Map getPartitions() + { + return partitions; + } + + public boolean isCreateRtTable() + { + return isCreateRtTable; + } + + private static List nonPartitionRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_LONG), + column("name", HIVE_STRING), + column("ts", HIVE_LONG), + column("dt", HIVE_STRING), + column("hh", HIVE_STRING)); + } + + private static List tripsRegularColumns() + { + return ImmutableList.of( + column("begin_lat", HIVE_DOUBLE), + column("begin_lon", HIVE_DOUBLE), + column("driver", HIVE_STRING), + column("end_lat", HIVE_DOUBLE), + column("end_lon", HIVE_DOUBLE), + column("fare", HIVE_DOUBLE), + column("partitionpath", HIVE_STRING), + column("rider", HIVE_STRING), + column("ts", HIVE_LONG), + column("uuid", HIVE_STRING)); + } + + private static List stockTicksRegularColumns() + { + return ImmutableList.of( + column("volume", HIVE_LONG), + column("ts", HIVE_STRING), + column("symbol", HIVE_STRING), + column("year", HIVE_INT), + column("month", HIVE_STRING), + column("high", HIVE_DOUBLE), + column("low", HIVE_DOUBLE), + column("key", HIVE_STRING), + column("date", HIVE_STRING), + column("close", HIVE_DOUBLE), + column("open", HIVE_DOUBLE), + column("day", HIVE_STRING)); + } + + private static List stockTicksPartitionColumns() + { + return ImmutableList.of(column("dt", HIVE_STRING)); + } + + private static Map stockTicksPartitions() + { + return ImmutableMap.of("dt=2018-08-31", "2018/08/31"); + } + + private static List hudiStockTicksRegularColumns() + { + return ImmutableList.of( + column("volume", HIVE_LONG), + column("ts", HIVE_STRING), + column("symbol", HIVE_STRING), + column("year", HIVE_INT), + column("month", HIVE_STRING), + column("high", HIVE_DOUBLE), + column("low", HIVE_DOUBLE), + column("key", HIVE_STRING), + column("close", HIVE_DOUBLE), + column("open", HIVE_DOUBLE), + column("day", HIVE_STRING)); + } + + private static List hudiStockTicksPartitionColumns() + { + return ImmutableList.of(column("date", HIVE_STRING)); + } + + private static Map hudiStockTicksPartitions() + { + return ImmutableMap.of("date=2018-08-31", "2018/08/31"); + } + + private static List multiPartitionRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_LONG), + column("name", HIVE_STRING), + column("ts", HIVE_LONG)); + } + + private static List multiPartitionColumns() + { + return ImmutableList.of( + column("dt", HIVE_STRING), + column("hh", HIVE_STRING)); + } + + private static Map multiPartitions() + { + return ImmutableMap.of( + "dt=2021-12-09/hh=10", "dt=2021-12-09/hh=10", + "dt=2021-12-09/hh=11", "dt=2021-12-09/hh=11"); + } + + private static List hudiMultiFgRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_INT), + column("name", HIVE_STRING), + column("price", HIVE_DOUBLE), + column("ts", HIVE_LONG)); + } + + private static List hudiMultiFgPartitionsColumn() + { + return ImmutableList.of( + column("country", HIVE_STRING)); + } + + private static Map hudiMultiFgPartitions() + { + return ImmutableMap.of( + "country=SG", "country=SG", + "country=US", "country=US"); + } + + private static List hudiComprehensiveTypesColumns() + { + return ImmutableList.of( + // ----- Primary Key & Precombine ----- + column("uuid", HIVE_STRING), + column("precombine_field", HIVE_LONG), + + // ----- Numeric Types ----- + column("col_boolean", HIVE_BOOLEAN), + column("col_tinyint", HIVE_BYTE), + column("col_smallint", HIVE_SHORT), + column("col_int", HIVE_INT), + column("col_bigint", HIVE_LONG), + column("col_float", HIVE_FLOAT), + column("col_double", HIVE_DOUBLE), + column("col_decimal", decimalHiveType(10, 2)), + + // ----- String Types ----- + column("col_string", HIVE_STRING), + column("col_varchar", varcharHiveType(50)), + column("col_char", charHiveType(10)), + + // ----- Binary Type ----- + column("col_binary", HIVE_BINARY), + + // ----- Datetime Types ----- + column("col_date", HIVE_DATE), + column("col_timestamp", HIVE_TIMESTAMP), + + // ----- Complex Types ----- + // ARRAY + column("col_array_int", listHiveType(INT_TYPE_INFO)), + // ARRAY + column("col_array_string", listHiveType(STRING_TYPE_INFO)), + // MAP + column("col_map_string_int", mapHiveType(STRING_TYPE_INFO, INT_TYPE_INFO)), + // STRUCT + column("col_struct", structHiveType( + ImmutableList.of("f1", "f2", "f3"), + ImmutableList.of(STRING_TYPE_INFO, INT_TYPE_INFO, BOOLEAN_TYPE_INFO))), + // ARRAY>> + column("col_array_struct", listHiveType( + getStructTypeInfo( + ImmutableList.of("nested_f1", "nested_f2"), + ImmutableList.of(DOUBLE_TYPE_INFO, ARRAY_STRING_TYPE_INFO)))), + // MAP> + column("col_map_string_struct", mapHiveType( + STRING_TYPE_INFO, + getStructTypeInfo( + ImmutableList.of("nested_f3", "nested_f4"), + ImmutableList.of(DATE_TYPE_INFO, getDecimalTypeInfo(5, 2))))), + // ARRAY>> + column("col_array_struct_with_map", listHiveType( + getStructTypeInfo( + ImmutableList.of("f_arr_struct_str", "f_arr_struct_map"), + ImmutableList.of(STRING_TYPE_INFO, MAP_STRING_INT_TYPE_INFO)))), + // MAP, f_map_struct_ts: TIMESTAMP>> + column("col_map_struct_with_array", mapHiveType( + STRING_TYPE_INFO, + getStructTypeInfo( + ImmutableList.of("f_map_struct_arr", "f_map_struct_ts"), + ImmutableList.of(ARRAY_BOOLEAN_TYPE_INFO, TIMESTAMP_TYPE_INFO)))), + // STRUCT> + column("col_struct_nested_struct", structHiveType( + ImmutableList.of("outer_f1", "nested_struct"), + ImmutableList.of( + INT_TYPE_INFO, + getStructTypeInfo( + ImmutableList.of("inner_f1", "inner_f2"), + ImmutableList.of(STRING_TYPE_INFO, BOOLEAN_TYPE_INFO))))), + // ARRAY> + column("col_array_array_int", listHiveType(ARRAY_INT_TYPE_INFO)), + // MAP> + column("col_map_string_array_double", mapHiveType(STRING_TYPE_INFO, ARRAY_DOUBLE_TYPE_INFO)), + // MAP> + column("col_map_string_map_string_date", mapHiveType(STRING_TYPE_INFO, MAP_STRING_DATE_TYPE_INFO)), + // STRUCT>> + column("col_struct_array_struct", structHiveType( + ImmutableList.of("outer_f2", "struct_array"), + ImmutableList.of( + STRING_TYPE_INFO, + getListTypeInfo(getStructTypeInfo( + ImmutableList.of("inner_f3", "inner_f4"), + ImmutableList.of(TIMESTAMP_TYPE_INFO, STRING_TYPE_INFO)))))), + // STRUCT> + column("col_struct_map", structHiveType( + ImmutableList.of("outer_f3", "struct_map"), + ImmutableList.of(BOOLEAN_TYPE_INFO, MAP_STRING_LONG_TYPE_INFO)))); + } + + private static List hudiComprehensiveTypesPartitionColumns() + { + return ImmutableList.of(column("part_col", HIVE_STRING)); + } + + private static Map hudiComprehensiveTypesPartitions() + { + return ImmutableMap.of( + "part_col=A", "part_col=A", + "part_col=B", "part_col=B"); + } + + private static List hudiMultiPtMorColumns() + { + return ImmutableList.of( + column("id", HIVE_INT), + column("name", HIVE_STRING), + column("price", HIVE_DOUBLE), + column("ts", HIVE_LONG)); + } + + private static List hudiMultiPtMorPartitionColumns() + { + return ImmutableList.of( + column("part_str", HIVE_STRING), + column("part_int", HIVE_INT), + column("part_date", HIVE_DATE), + column("part_bigint", HIVE_LONG), + column("part_decimal", decimalHiveType(10, 2)), + column("part_timestamp", HIVE_TIMESTAMP), + column("part_bool", HIVE_BOOLEAN)); + } + + private static Map hudiMultiPtMorPartitions() + { + return ImmutableMap.of( + "part_str=apparel/part_int=2024/part_date=2024-01-05/part_bigint=20000000001/part_decimal=100.00/part_timestamp=2024-01-05 18%3A00%3A00/part_bool=false", "part_str=apparel/part_int=2024/part_date=2024-01-05/part_bigint=20000000001/part_decimal=100.00/part_timestamp=2024-01-05 18%3A00%3A00/part_bool=false", + "part_str=electronics/part_int=2023/part_date=2023-03-10/part_bigint=10000000002/part_decimal=50.00/part_timestamp=2023-03-10 12%3A30%3A00/part_bool=true", "part_str=electronics/part_int=2023/part_date=2023-03-10/part_bigint=10000000002/part_decimal=50.00/part_timestamp=2023-03-10 12%3A30%3A00/part_bool=true", + "part_str=electronics/part_int=2023/part_date=2023-03-10/part_bigint=10000000002/part_decimal=50.00/part_timestamp=2023-03-10 12%3A30%3A00/part_bool=false", "part_str=electronics/part_int=2023/part_date=2023-03-10/part_bigint=10000000002/part_decimal=50.00/part_timestamp=2023-03-10 12%3A30%3A00/part_bool=false", + "part_str=books/part_int=2023/part_date=2023-01-15/part_bigint=10000000001/part_decimal=123.00/part_timestamp=2023-01-15 10%3A00%3A00/part_bool=true", "part_str=books/part_int=2023/part_date=2023-01-15/part_bigint=10000000001/part_decimal=123.00/part_timestamp=2023-01-15 10%3A00%3A00/part_bool=true", + "part_str=books/part_int=2024/part_date=2024-02-20/part_bigint=10000000003/part_decimal=75.00/part_timestamp=2024-02-20 08%3A45%3A10/part_bool=true", "part_str=books/part_int=2024/part_date=2024-02-20/part_bigint=10000000003/part_decimal=75.00/part_timestamp=2024-02-20 08%3A45%3A10/part_bool=true"); + } + + private static List hudiTimestampKeygenColumns() + { + return ImmutableList.of( + column("id", HIVE_INT), + column("name", HIVE_STRING), + column("price", HIVE_DOUBLE), + column("ts", HIVE_LONG)); + } + + private static List hudiTimestampKeygenPartitionColumns() + { + // Data stored in files are long, but partition value that is synced to metastore is String + return ImmutableList.of(column("partition_field", HIVE_STRING)); + } + + private static Map hudiTimestampKeygenPartitions(String timestampType) + { + return switch (timestampType) { + case "EPOCHMILLISECONDS" -> ImmutableMap.of( + "partition_field=2025-05-13 02", "2025-05-13 02", + "partition_field=2025-06-05 05", "2025-06-05 05", + "partition_field=2025-06-06 09", "2025-06-06 09", + "partition_field=2025-06-06 10", "2025-06-06 10", + "partition_field=2025-06-07 08", "2025-06-07 08"); + case "SCALAR" -> ImmutableMap.of( + "partition_field=2024-10-08 12", "2024-10-08 12", + "partition_field=2024-10-07 12", "2024-10-07 12", + "partition_field=2024-10-06 12", "2024-10-06 12", + "partition_field=2024-10-05 12", "2024-10-05 12", + "partition_field=2024-10-04 12", "2024-10-04 12"); + default -> ImmutableMap.of(); + }; + } + + private static List hudiCustomKeyGenColumns() + { + return hudiMultiFgRegularColumns(); + } + + private static List hudiCustomKeyGenPartitionColumns() + { + return ImmutableList.of( + column("partition_field_country", HIVE_STRING), + column("partition_field_date", HIVE_STRING)); + } + + private static Map hudiCustomKeyGenPartitions() + { + return ImmutableMap.of( + "partition_field_country=US/partition_field_date=2025-06-06", "partition_field_country=US/partition_field_date=2025-06-06", + "partition_field_country=CN/partition_field_date=2025-06-05", "partition_field_country=CN/partition_field_date=2025-06-05", + "partition_field_country=MY/partition_field_date=2025-05-13", "partition_field_country=MY/partition_field_date=2025-05-13", + "partition_field_country=SG/partition_field_date=2025-06-06", "partition_field_country=SG/partition_field_date=2025-06-06", + "partition_field_country=SG/partition_field_date=2025-06-07", "partition_field_country=SG/partition_field_date=2025-06-07"); + } + } + + static class HashAndSizeResult + { + final byte[] hash; + final long size; + + HashAndSizeResult(byte[] hash, long size) + { + this.hash = hash; + this.size = size; + } + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java new file mode 100644 index 0000000000000..10d1b9873ad35 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java @@ -0,0 +1,400 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.hdfs.HdfsContext; +import io.trino.hdfs.HdfsEnvironment; +import io.trino.metastore.Column; +import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveType; +import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.StorageFormat; +import io.trino.metastore.Table; +import io.trino.plugin.hudi.HudiConnector; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.spi.connector.CatalogSchemaName; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.testing.MaterializedResult; +import io.trino.testing.MaterializedRow; +import io.trino.testing.QueryRunner; +import io.trino.tpch.TpchColumn; +import io.trino.tpch.TpchColumnType; +import io.trino.tpch.TpchColumnTypes; +import io.trino.tpch.TpchTable; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.marker.MarkerType; +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; +import org.intellij.lang.annotations.Language; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDate; +import java.time.temporal.ChronoField; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static io.trino.hive.formats.HiveClassNames.HUDI_PARQUET_INPUT_FORMAT; +import static io.trino.hive.formats.HiveClassNames.MAPRED_PARQUET_OUTPUT_FORMAT_CLASS; +import static io.trino.hive.formats.HiveClassNames.PARQUET_HIVE_SERDE_CLASS; +import static io.trino.metastore.HiveType.HIVE_DATE; +import static io.trino.metastore.HiveType.HIVE_DOUBLE; +import static io.trino.metastore.HiveType.HIVE_INT; +import static io.trino.metastore.HiveType.HIVE_LONG; +import static io.trino.metastore.HiveType.HIVE_STRING; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; +import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; +import static io.trino.testing.TestingConnectorSession.SESSION; +import static java.lang.String.format; +import static java.nio.file.Files.createTempDirectory; +import static java.util.Collections.unmodifiableList; +import static java.util.Objects.requireNonNull; +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; + +public class TpchHudiTablesInitializer + implements HudiTablesInitializer +{ + public static final String FIELD_UUID = "_uuid"; + private static final CatalogSchemaName TPCH_TINY = new CatalogSchemaName("tpch", "tiny"); + private static final String PARTITION_PATH = ""; + private static final Logger log = Logger.get(TpchHudiTablesInitializer.class); + private static final List HUDI_META_COLUMNS = ImmutableList.of( + new Column("_hoodie_commit_time", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_commit_seqno", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_record_key", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_partition_path", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_file_name", HIVE_STRING, Optional.empty(), Map.of())); + private static final HdfsContext CONTEXT = new HdfsContext(SESSION); + + private final List> tpchTables; + + public TpchHudiTablesInitializer(List> tpchTables) + { + this.tpchTables = requireNonNull(tpchTables, "tpchTables is null"); + } + + @Override + public void initializeTables(QueryRunner queryRunner, Location externalLocation, String schemaName) + throws Exception + { + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog(TPCH_TINY.getCatalogName(), "tpch", ImmutableMap.of()); + TrinoFileSystem fileSystem = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + HiveMetastore metastore = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + + Location dataLocation = externalLocation.appendPath("tpch"); + + java.nio.file.Path tempDir = createTempDirectory("test"); + try { + for (TpchTable tpchTable : tpchTables) { + java.nio.file.Path tempTableDir = tempDir.resolve(tpchTable.getTableName()); + load(tpchTable, queryRunner, tempTableDir); + + Location tableLocation = dataLocation.appendPath(tpchTable.getTableName()); + ResourceHudiTablesInitializer.copyDir(tempTableDir, fileSystem, tableLocation); + + Table table = createTableDefinition(schemaName, tpchTable, tableLocation); + metastore.createTable(table, PrincipalPrivileges.NO_PRIVILEGES); + } + } + finally { + deleteRecursively(tempDir, ALLOW_INSECURE); + } + } + + public void load(TpchTable tpchTables, QueryRunner queryRunner, java.nio.file.Path tableDirectory) + { + try (HoodieJavaWriteClient writeClient = createWriteClient(tpchTables, HDFS_ENVIRONMENT, new Path(tableDirectory.toUri()))) { + RecordConverter recordConverter = createRecordConverter(tpchTables); + + @Language("SQL") String sql = generateScanSql(TPCH_TINY, tpchTables); + log.info("Executing %s", sql); + MaterializedResult result = queryRunner.execute(sql); + + List> records = result.getMaterializedRows() + .stream() + .map(MaterializedRow::getFields) + .map(recordConverter::toRecord) + .collect(Collectors.toList()); + String timestamp = HoodieInstantTimeGenerator.formatDate(Date.from(Instant.now())); + writeClient.startCommitWithTime(timestamp); + writeClient.insert(records, timestamp); + } + } + + private static String generateScanSql(CatalogSchemaName catalogSchemaName, TpchTable table) + { + StringBuilder builder = new StringBuilder(); + builder.append("SELECT "); + String columnList = table.getColumns() + .stream() + .map(c -> quote(c.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 Table createTableDefinition(String schemaName, TpchTable table, Location location) + { + List columns = Stream.of(HUDI_META_COLUMNS, createMetastoreColumns(table)) + .flatMap(Collection::stream) + .toList(); + StorageFormat storageFormat = StorageFormat.create( + PARQUET_HIVE_SERDE_CLASS, + HUDI_PARQUET_INPUT_FORMAT, + MAPRED_PARQUET_OUTPUT_FORMAT_CLASS); + + return Table.builder() + .setDatabaseName(schemaName) + .setTableName(table.getTableName()) + .setTableType(EXTERNAL_TABLE.name()) + .setOwner(Optional.of("public")) + .setDataColumns(columns) + .setParameters(ImmutableMap.of("serialization.format", "1", "EXTERNAL", "TRUE")) + .withStorage(storageBuilder -> storageBuilder + .setStorageFormat(storageFormat) + .setLocation(location.toString())) + .build(); + } + + private static HoodieJavaWriteClient createWriteClient(TpchTable table, HdfsEnvironment hdfsEnvironment, Path tablePath) + { + Schema schema = createAvroSchema(table); + Configuration conf = hdfsEnvironment.getConfiguration(CONTEXT, tablePath); + + try { + HoodieTableMetaClient.newTableBuilder() + .setTableType(COPY_ON_WRITE) + .setTableName(table.getTableName()) + .setTimelineLayoutVersion(1) + .setBootstrapIndexClass(NoOpBootstrapIndex.class.getName()) + .setPayloadClassName(HoodieAvroPayload.class.getName()) + .setRecordKeyFields(FIELD_UUID) + .initTable(new HadoopStorageConfiguration(conf), tablePath.toString()); + } + catch (IOException e) { + throw new RuntimeException("Could not init table " + table.getTableName(), e); + } + + HoodieIndexConfig indexConfig = HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build(); + HoodieArchivalConfig archivalConfig = HoodieArchivalConfig.newBuilder().archiveCommitsWith(20, 30).build(); + HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder() + .withPath(tablePath.toString()) + .withSchema(schema.toString()) + .withParallelism(2, 2) + .withDeleteParallelism(2) + .forTable(table.getTableName()) + .withIndexConfig(indexConfig) + .withArchivalConfig(archivalConfig) + .withEmbeddedTimelineServerEnabled(false) + .withMarkersType(MarkerType.DIRECT.name()) + // Disabling Hudi metadata table (MDT) in tests as the support of + // reading MDT is broken after removal of Hudi dependencies from compile time + // IMPORTANT: Writing to MDT requires hbase dependencies, which is not available in Trino runtime + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); + return new HoodieJavaWriteClient<>(new HoodieJavaEngineContext(new HadoopStorageConfiguration(conf)), cfg); + } + + private static RecordConverter createRecordConverter(TpchTable table) + { + Schema schema = createAvroSchema(table); + List> columns = table.getColumns(); + + int numberOfColumns = columns.size(); + List columnNames = columns.stream() + .map(TpchColumn::getSimplifiedColumnName) + .toList(); + List> columnConverters = columns.stream() + .map(TpchColumn::getType) + .map(TpchHudiTablesInitializer::avroEncoderOf) + .toList(); + + return row -> { + checkArgument(row.size() == numberOfColumns); + + // Create a GenericRecord + GenericRecord record = new GenericData.Record(schema); + for (int i = 0; i < numberOfColumns; i++) { + record.put(columnNames.get(i), columnConverters.get(i).apply(row.get(i))); + } + // Add extra uuid column + String uuid = UUID.randomUUID().toString(); + record.put(FIELD_UUID, uuid); + + // wrap to a HoodieRecord + HoodieKey key = new HoodieKey(uuid, PARTITION_PATH); + HoodieAvroPayload data = new HoodieAvroPayload(Option.of(record)); + return new HoodieAvroRecord<>(key, data, null); + }; + } + + private static Schema createAvroSchema(TpchTable table) + { + List> tpchColumns = table.getColumns(); + List fields = new ArrayList<>(tpchColumns.size() + 1); + for (TpchColumn column : tpchColumns) { + String columnName = column.getSimplifiedColumnName(); + Schema.Type columnSchemaType = toSchemaType(column.getType()); + // Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(type)); + fields.add(new Schema.Field(columnName, Schema.create(columnSchemaType))); + } + fields.add(new Schema.Field(FIELD_UUID, Schema.create(Schema.Type.STRING))); + String name = table.getTableName(); + return Schema.createRecord(name, null, null, false, fields); + } + + private static List createMetastoreColumns(TpchTable table) + { + List> tpchColumns = table.getColumns(); + List columns = new ArrayList<>(tpchColumns.size() + 1); + for (TpchColumn c : tpchColumns) { + HiveType hiveType = TpchColumnTypeAdapter.toHiveType(c.getType()); + columns.add(new Column(c.getSimplifiedColumnName(), hiveType, Optional.empty(), Map.of())); + } + columns.add(new Column(FIELD_UUID, HIVE_STRING, Optional.empty(), Map.of())); + return unmodifiableList(columns); + } + + private static Schema.Type toSchemaType(TpchColumnType columnType) + { + return TpchColumnTypeAdapter.of(columnType).avroType; + } + + private static Function avroEncoderOf(TpchColumnType columnType) + { + return TpchColumnTypeAdapter.of(columnType).avroEncoder; + } + + private static String quote(String name) + { + return "\"" + name + "\""; + } + + private enum TpchColumnTypeAdapter + { + INTEGER(Schema.Type.INT, hiveTypeOf(HIVE_INT), Function.identity()), + IDENTIFIER(Schema.Type.LONG, hiveTypeOf(HIVE_LONG), Function.identity()), + DATE(Schema.Type.INT, hiveTypeOf(HIVE_DATE), TpchColumnTypeAdapter::convertDate), + DOUBLE(Schema.Type.DOUBLE, hiveTypeOf(HIVE_DOUBLE), Function.identity()), + VARCHAR(Schema.Type.STRING, TpchColumnTypeAdapter::hiveVarcharOf, Function.identity()), + /**/; + + static TpchColumnTypeAdapter of(TpchColumnType columnType) + { + if (columnType == TpchColumnTypes.INTEGER) { + return INTEGER; + } + else if (columnType == TpchColumnTypes.IDENTIFIER) { + return IDENTIFIER; + } + else if (columnType == TpchColumnTypes.DATE) { + return DATE; + } + else if (columnType == TpchColumnTypes.DOUBLE) { + return DOUBLE; + } + else { + if (columnType.getBase() != TpchColumnType.Base.VARCHAR || columnType.getPrecision().isEmpty()) { + throw new IllegalArgumentException("Illegal column type: " + columnType); + } + return VARCHAR; + } + } + + static HiveType toHiveType(TpchColumnType columnType) + { + return of(columnType).hiveTypeConverter.apply(columnType); + } + + private final Schema.Type avroType; + private final Function hiveTypeConverter; + private final Function avroEncoder; + + TpchColumnTypeAdapter( + Schema.Type avroType, + Function hiveTypeConverter, + Function avroEncoder) + { + this.avroType = avroType; + this.hiveTypeConverter = hiveTypeConverter; + this.avroEncoder = avroEncoder; + } + + private static Function hiveTypeOf(HiveType hiveType) + { + return _ -> hiveType; + } + + private static HiveType hiveVarcharOf(TpchColumnType type) + { + verify(type.getPrecision().isPresent()); + return HiveType.valueOf("varchar(" + type.getPrecision().get() + ")"); + } + + private static Object convertDate(Object input) + { + LocalDate date = (LocalDate) input; + return (int) date.getLong(ChronoField.EPOCH_DAY); + } + } + + private interface RecordConverter + { + HoodieRecord toRecord(List row); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/TypeInfoHelper.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/TypeInfoHelper.java new file mode 100644 index 0000000000000..9e21009fe6ba3 --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/testing/TypeInfoHelper.java @@ -0,0 +1,129 @@ +/* + * 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.hudi.testing; + +import io.trino.metastore.HiveType; +import io.trino.metastore.type.TypeInfo; + +import java.util.ArrayList; +import java.util.List; + +import static io.trino.metastore.type.TypeConstants.BIGINT_TYPE_NAME; +import static io.trino.metastore.type.TypeConstants.BOOLEAN_TYPE_NAME; +import static io.trino.metastore.type.TypeConstants.DATE_TYPE_NAME; +import static io.trino.metastore.type.TypeConstants.DOUBLE_TYPE_NAME; +import static io.trino.metastore.type.TypeConstants.INT_TYPE_NAME; +import static io.trino.metastore.type.TypeConstants.STRING_TYPE_NAME; +import static io.trino.metastore.type.TypeConstants.TIMESTAMP_TYPE_NAME; +import static io.trino.metastore.type.TypeInfoFactory.getCharTypeInfo; +import static io.trino.metastore.type.TypeInfoFactory.getDecimalTypeInfo; +import static io.trino.metastore.type.TypeInfoFactory.getListTypeInfo; +import static io.trino.metastore.type.TypeInfoFactory.getMapTypeInfo; +import static io.trino.metastore.type.TypeInfoFactory.getPrimitiveTypeInfo; +import static io.trino.metastore.type.TypeInfoFactory.getStructTypeInfo; +import static io.trino.metastore.type.TypeInfoFactory.getVarcharTypeInfo; + +public class TypeInfoHelper +{ + public static final TypeInfo BOOLEAN_TYPE_INFO = getPrimitiveTypeInfo(BOOLEAN_TYPE_NAME); + public static final TypeInfo INT_TYPE_INFO = getPrimitiveTypeInfo(INT_TYPE_NAME); + public static final TypeInfo LONG_TYPE_INFO = getPrimitiveTypeInfo(BIGINT_TYPE_NAME); + public static final TypeInfo DOUBLE_TYPE_INFO = getPrimitiveTypeInfo(DOUBLE_TYPE_NAME); + public static final TypeInfo STRING_TYPE_INFO = getPrimitiveTypeInfo(STRING_TYPE_NAME); + public static final TypeInfo TIMESTAMP_TYPE_INFO = getPrimitiveTypeInfo(TIMESTAMP_TYPE_NAME); + public static final TypeInfo DATE_TYPE_INFO = getPrimitiveTypeInfo(DATE_TYPE_NAME); + + public static final TypeInfo ARRAY_STRING_TYPE_INFO = getListTypeInfo(STRING_TYPE_INFO); + public static final TypeInfo ARRAY_INT_TYPE_INFO = getListTypeInfo(INT_TYPE_INFO); + public static final TypeInfo ARRAY_BOOLEAN_TYPE_INFO = getListTypeInfo(BOOLEAN_TYPE_INFO); + public static final TypeInfo ARRAY_DOUBLE_TYPE_INFO = getListTypeInfo(DOUBLE_TYPE_INFO); + + public static final TypeInfo MAP_STRING_INT_TYPE_INFO = getMapTypeInfo(STRING_TYPE_INFO, INT_TYPE_INFO); + public static final TypeInfo MAP_STRING_LONG_TYPE_INFO = getMapTypeInfo(STRING_TYPE_INFO, LONG_TYPE_INFO); + public static final TypeInfo MAP_STRING_DATE_TYPE_INFO = getMapTypeInfo(STRING_TYPE_INFO, DATE_TYPE_INFO); + + private TypeInfoHelper() + { + } + + /** + * Creates a HiveType for a list type. + * + * @param elementTypeInfo The TypeInfo of the list elements. + * @return A HiveType instance for the list. + */ + public static HiveType listHiveType(TypeInfo elementTypeInfo) + { + return HiveType.fromTypeInfo(getListTypeInfo(elementTypeInfo)); + } + + /** + * Creates a HiveType for a map type. + * + * @param keyTypeInfo The TypeInfo of the map keys. + * @param valueTypeInfo The TypeInfo of the map values. + * @return A HiveType instance for the map. + */ + public static HiveType mapHiveType(TypeInfo keyTypeInfo, TypeInfo valueTypeInfo) + { + return HiveType.fromTypeInfo(getMapTypeInfo(keyTypeInfo, valueTypeInfo)); + } + + /** + * Creates a HiveType for a struct type. + * + * @param fieldNames List of field names. + * @param fieldTypeInfos List of corresponding field TypeInfos. + * @return A HiveType instance for the struct. + */ + public static HiveType structHiveType(List fieldNames, List fieldTypeInfos) + { + // ArrayList to preserve ordering + return HiveType.fromTypeInfo(getStructTypeInfo(new ArrayList<>(fieldNames), new ArrayList<>(fieldTypeInfos))); + } + + /** + * Creates a HiveType for a decimal type. + * + * @param precision The precision. + * @param scale The scale. + * @return A HiveType instance for the decimal. + */ + public static HiveType decimalHiveType(int precision, int scale) + { + return HiveType.fromTypeInfo(getDecimalTypeInfo(precision, scale)); + } + + /** + * Creates a HiveType for a varchar type. + * + * @param length The maximum length. + * @return A HiveType instance for the varchar. + */ + public static HiveType varcharHiveType(int length) + { + return HiveType.fromTypeInfo(getVarcharTypeInfo(length)); + } + + /** + * Creates a HiveType for a char type. + * + * @param length The fixed length. + * @return A HiveType instance for the char. + */ + public static HiveType charHiveType(int length) + { + return HiveType.fromTypeInfo(getCharTypeInfo(length)); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/util/TestTupleDomainUtilsExtendedNullFilterTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/util/TestTupleDomainUtilsExtendedNullFilterTest.java new file mode 100644 index 0000000000000..c20e7f075f3ca --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/util/TestTupleDomainUtilsExtendedNullFilterTest.java @@ -0,0 +1,194 @@ +/* + * 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.hudi.util; + +import io.airlift.slice.Slices; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.ValueSet; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static io.trino.spi.type.VarcharType.VARCHAR; +import static org.assertj.core.api.Assertions.assertThat; + +class TestTupleDomainUtilsExtendedNullFilterTest +{ + @Test + void testHasSimpleNullCheck_withOnlyNullConstraint() + { + // Create a Domain that represents exactly "IS NULL" for a column. + // Domain.onlyNull(Type) creates a domain where null is allowed and the value set is 'none'. + Domain isNullDomain = Domain.onlyNull(VARCHAR); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of("col_a", isNullDomain)); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result) + .as("Check failed: TupleDomain with only an 'IS NULL' constraint should return true.") + .isTrue(); + } + + @Test + void testHasSimpleNullCheck_withOnlyNotNullConstraint() + { + // Create a Domain that represents exactly "IS NOT NULL" for a column. + // Domain.notNull(Type) creates a domain where null is *NOT* allowed and the value set is 'all'. + Domain isNotNullDomain = Domain.notNull(VARCHAR); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of("col_a", isNotNullDomain)); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result) + .as("Check failed: TupleDomain with only an 'IS NOT NULL' constraint should return true.") + .isTrue(); + } + + @Test + void testHasSimpleNullCheck_withMixedConstraintsIncludingNull() + { + // Create a TupleDomain with multiple columns, where one column has an "IS NULL" constraint and others have different constraints. + Domain isNullDomain = Domain.onlyNull(VARCHAR); + // Example of another constraint: col_b > 'abc', null allowed + Domain rangeDomain = Domain.create( + ValueSet.ofRanges(Range.greaterThan(VARCHAR, Slices.utf8Slice("abc"))), + true); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "col_a", isNullDomain, + "col_b", rangeDomain)); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result) + .as("Check failed: TupleDomain with mixed constraints including 'IS NULL' should return true.") + .isTrue(); + } + + @Test + void testHasSimpleNullCheck_withMixedConstraintsIncludingNotNull() + { + // Create a TupleDomain with multiple columns, where one column has an "IS NOT NULL" constraint. + Domain isNotNullDomain = Domain.notNull(VARCHAR); + // Add another constraint: col_b < 'xyz', null not allowed + Domain rangeDomain = Domain.create( + ValueSet.ofRanges(Range.lessThan(VARCHAR, Slices.utf8Slice("xyz"))), + false); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "col_a", isNotNullDomain, + "col_b", rangeDomain)); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result) + .as("Check failed: TupleDomain with mixed constraints including 'IS NOT NULL' should return true.") + .isTrue(); + } + + @Test + void testHasSimpleNullCheck_withNonNullAndNotNullConstraint() + { + // Create a domain that allows specific non-null values AND disallows null. + // This is *NOT* exclusively an "IS NOT NULL" constraint because ValueSet is not 'all'. + Domain specificValuesNotNullDomain = Domain.create( + // Only allows 'value1' + ValueSet.of(VARCHAR, Slices.utf8Slice("value1")), + false); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "col_a", specificValuesNotNullDomain)); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result).isFalse(); + } + + @Test + void testHasSimpleNullCheck_withNonNullAndNullConstraint() + { + // Create a domain that allows specific non-null values AND allows null. + // This is *NOT* exclusively an "IS NULL" constraint because ValueSet is not 'none'. + Domain specificValuesNullDomain = Domain.create( + // Only allows 'value1' + ValueSet.of(VARCHAR, Slices.utf8Slice("value1")), + true); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "col_a", specificValuesNullDomain)); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result).isFalse(); + } + + @Test + void testHasSimpleNullCheck_withNoSimpleNullChecks() + { + // Create a TupleDomain where columns have constraints, but none are *only* IS NULL or IS NOT NULL. + // col_a > 'abc', null allowed + Domain rangeDomain1 = Domain.create( + ValueSet.ofRanges(Range.greaterThan(VARCHAR, Slices.utf8Slice("abc"))), + true); + // col_b < 'xyz', null not allowed + Domain rangeDomain2 = Domain.create( + ValueSet.ofRanges(Range.lessThan(VARCHAR, Slices.utf8Slice("xyz"))), + false); + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "col_a", rangeDomain1, + "col_b", rangeDomain2)); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result).isFalse(); + } + + @Test + void testHasSimpleNullCheck_withAllTupleDomain() + { + // Create an 'All' TupleDomain, which represents no constraints. + TupleDomain tupleDomain = TupleDomain.all(); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result) + .as("Check failed: TupleDomain.all() should return false as it has no constraints.") + .isFalse(); + } + + @Test + void testHasSimpleNullCheck_withNoneTupleDomain() + { + // Create a 'None' TupleDomain, which represents a contradiction (always false). + TupleDomain tupleDomain = TupleDomain.none(); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result) + .as("Check failed: TupleDomain.none() should return false.") + .isFalse(); + } + + @Test + void testHasSimpleNullCheck_withEmptyTupleDomain() + { + // Create a TupleDomain using an empty map of column domains. + // This usually results in an 'All' TupleDomain. + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of()); + + boolean result = TupleDomainUtils.hasSimpleNullCheck(tupleDomain); + + assertThat(result) + .as("Check failed: TupleDomain created with an empty map (effectively All) should return false.") + .isFalse(); + } +} diff --git a/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/util/TestTupleDomainUtilsTest.java b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/util/TestTupleDomainUtilsTest.java new file mode 100644 index 0000000000000..64becba2ce60b --- /dev/null +++ b/hudi-trino-plugin/src/test/java/io/trino/plugin/hudi/util/TestTupleDomainUtilsTest.java @@ -0,0 +1,269 @@ +/* + * 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.hudi.util; + +import io.airlift.slice.Slices; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.ValueSet; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; + +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static org.assertj.core.api.Assertions.assertThat; + +class TestTupleDomainUtilsTest +{ + @Test + void testGetReferencedColumnsEmpty() + { + // Test with an empty TupleDomain + TupleDomain emptyTupleDomain = TupleDomain.none(); + assertThat(TupleDomainUtils.getReferencedColumns(emptyTupleDomain)).isEmpty(); + + // Test with a TupleDomain that effectively has no column domains + TupleDomain effectivelyEmptyTupleDomain = TupleDomain.all(); + assertThat(TupleDomainUtils.getReferencedColumns(effectivelyEmptyTupleDomain)).isEmpty(); + } + + @Test + void testGetReferencedColumnsNonEmpty() + { + // Test with a TupleDomain containing domains for specific columns + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "col_a", Domain.singleValue(BIGINT, 1L), + "col_b", Domain.onlyNull(VARCHAR))); + assertThat(TupleDomainUtils.getReferencedColumns(tupleDomain)) + .containsExactlyInAnyOrder("col_a", "col_b"); + } + + @Test + void testAreAllFieldsReferencedAllMatch() + { + // Test when all source fields are present in the TupleDomain + // 'all' counts as referenced and when passed into the builder, it will be ignored + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "id", Domain.singleValue(BIGINT, 100L), + "name", Domain.singleValue(VARCHAR, Slices.utf8Slice("test")), + "value", Domain.all(BIGINT))); + List sourceFields = List.of("id", "name"); + assertThat(TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFields)).isTrue(); + + // The constructed TupleDomain will hence not have any constraint on the domain with 'all' + List sourceFieldsIncludingAll = List.of("id", "name", "value"); + assertThat(TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFieldsIncludingAll)).isFalse(); + } + + @Test + void testAreAllFieldsReferencedSomeMatch() + { + // Test when only some source fields are present + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "id", Domain.singleValue(BIGINT, 100L), + "value", Domain.all(BIGINT))); + // "name" is missing + List sourceFields = List.of("id", "name"); + assertThat(TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreAllFieldsReferencedNoneMatch() + { + // Test when none of the source fields are present + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "id", Domain.singleValue(BIGINT, 100L), + "value", Domain.all(BIGINT))); + // All provided sourceFields are absent + List sourceFields = List.of("field_x", "field_y"); + assertThat(TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreAllFieldsReferencedEmptySourceFields() + { + // Test with an empty list of source fields (should technically be true) + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "id", Domain.singleValue(BIGINT, 100L))); + List sourceFields = List.of(); + // An empty set is a subset of any set + assertThat(TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFields)).isTrue(); + } + + @Test + void testAreAllFieldsReferencedEmptyTupleDomain() + { + // Test with an empty TupleDomain + TupleDomain emptyTupleDomain = TupleDomain.none(); + List sourceFields = List.of("id", "name"); + assertThat(TupleDomainUtils.areAllFieldsReferenced(emptyTupleDomain, sourceFields)).isFalse(); + + // Test with an empty source list and empty tuple domain + List emptySourceFields = List.of(); + assertThat(TupleDomainUtils.areAllFieldsReferenced(emptyTupleDomain, emptySourceFields)).isTrue(); + } + + @Test + void testAreSomeFieldsReferencedSomeMatch() + { + // Test when at least one source field is present + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "id", Domain.singleValue(BIGINT, 100L), + "value", Domain.all(BIGINT))); + // Only "id" is present + List sourceFields = List.of("id", "name"); + assertThat(TupleDomainUtils.areSomeFieldsReferenced(tupleDomain, sourceFields)).isTrue(); + } + + @Test + void testAreSomeFieldsReferencedAllMatch() + { + // Test when all source fields are present + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "id", Domain.singleValue(BIGINT, 100L), + "name", Domain.singleValue(VARCHAR, Slices.utf8Slice("test")))); + List sourceFields = List.of("id", "name"); + assertThat(TupleDomainUtils.areSomeFieldsReferenced(tupleDomain, sourceFields)).isTrue(); + } + + @Test + void testAreSomeFieldsReferencedNoneMatch() + { + // Test when none of the source fields are present + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "id", Domain.singleValue(BIGINT, 100L))); + List sourceFields = List.of("name", "value"); // None are present + assertThat(TupleDomainUtils.areSomeFieldsReferenced(tupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreSomeFieldsReferencedEmptySourceFields() + { + // Test with an empty list of source fields + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "id", Domain.singleValue(BIGINT, 100L))); + List sourceFields = List.of(); + assertThat(TupleDomainUtils.areSomeFieldsReferenced(tupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreSomeFieldsReferencedEmptyTupleDomain() + { + // Test with an empty TupleDomain + TupleDomain emptyTupleDomain = TupleDomain.none(); + List sourceFields = List.of("id", "name"); + assertThat(TupleDomainUtils.areSomeFieldsReferenced(emptyTupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreDomainsInOrEqualOnlyAllMatch() + { + // Test when all referenced source fields have IN or EQUALS domains + // "other_col" is an irrelevant column + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "key1", Domain.singleValue(BIGINT, 1L), // EQUALS + "key2", Domain.multipleValues(VARCHAR, List.of("a", "b")), // IN + "other_col", Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 10L)), false))); + List sourceFields = List.of("key1", "key2"); + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isTrue(); + } + + @Test + void testAreDomainsInOrEqualOnlySomeMatch() + { + // Test when one source field has a non-IN/EQUALS domain (e.g. RANGE) + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "key1", Domain.singleValue(BIGINT, 1L), // EQUALS + "key2", Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 10L)), false))); + List sourceFields = List.of("key1", "key2"); + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreDomainsInOrEqualOnlySomeMatchWithAll() + { + // Test when one source field has Domain.all() + // After creation, `key2` wil not be in the TupleDomain + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "key1", Domain.singleValue(BIGINT, 1L), // EQUALS + // ALL type (neither single value nor discrete set) + "key2", Domain.all(VARCHAR))); + List sourceFields = List.of("key1", "key2"); + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreDomainsInOrEqualOnlySomeMatchWithNull() + { + // Test when one source field has Domain.onlyNull() + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "key1", Domain.singleValue(BIGINT, 1L), // EQUALS + // onlyNull (neither single value nor discrete set) + "key2", Domain.onlyNull(VARCHAR))); + List sourceFields = List.of("key1", "key2"); + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreDomainsInOrEqualOnlySourceFieldMissing() + { + // Test when one of the source fields is not present in the TupleDomain + // key2 is missing + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "key1", Domain.singleValue(BIGINT, 1L))); + List sourceFields = List.of("key1", "key2"); + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isFalse(); + } + + @Test + void testAreDomainsInOrEqualOnlyEmptySourceFields() + { + // Test with an empty list of source fields + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "key1", Domain.singleValue(BIGINT, 1L))); + List sourceFields = List.of(); + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isFalse(); // As per implementation check + } + + @Test + void testAreDomainsInOrEqualOnlyNullSourceFields() + { + // Test with null source fields list + TupleDomain tupleDomain = TupleDomain.withColumnDomains(Map.of( + "key1", Domain.singleValue(BIGINT, 1L))); + List sourceFields = null; + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isFalse(); // As per implementation check + } + + @Test + void testAreDomainsInOrEqualOnlyTupleDomainAll() + { + // Test with TupleDomain.all() + TupleDomain tupleDomain = TupleDomain.all(); + List sourceFields = List.of("key1", "key2"); + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isFalse(); // As per implementation check + } + + @Test + void testAreDomainsInOrEqualOnlyTupleDomainNone() + { + // Test with TupleDomain.none() + TupleDomain tupleDomain = TupleDomain.none(); + List sourceFields = List.of("key1", "key2"); + assertThat(TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields)).isFalse(); + } +} diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v6_mor.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v6_mor.md new file mode 100644 index 0000000000000..9883fe6db2a77 --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v6_mor.md @@ -0,0 +1,164 @@ +## Create script + +Structure of table: +- MOR table with MDT enabled +- Revision: tag release-0.15.0 + +```scala +test("Create MOR table with comprehensive types") { + withTempDir { tmp => + val tableName = "hudi_type_test_mor" + spark.sql( + s""" + |CREATE TABLE $tableName ( + | uuid STRING, + | precombine_field LONG, + | + | -- Numeric Types + | col_boolean BOOLEAN, + | col_tinyint TINYINT, + | col_smallint SMALLINT, + | col_int INT, + | col_bigint BIGINT, + | col_float FLOAT, + | col_double DOUBLE, + | col_decimal DECIMAL(10, 2), + | + | -- String Types + | col_string STRING, + | col_varchar VARCHAR(50), + | col_char CHAR(10), + | + | -- Binary Type + | col_binary BINARY, + | + | -- Datetime Types + | col_date DATE, + | col_timestamp TIMESTAMP, + | -- col_timestamp_ntz TIMESTAMP_NTZ, (No support on Hudi for now) + | + | -- Complex types + | col_array_int ARRAY, + | col_array_string ARRAY, + | col_map_string_int MAP, + | col_struct STRUCT, + | col_array_struct ARRAY>>, + | col_map_string_struct MAP>, + | col_array_struct_with_map ARRAY>>, + | col_map_struct_with_array MAP, f_map_struct_ts: TIMESTAMP>>, + | col_struct_nested_struct STRUCT>, + | col_array_array_int ARRAY>, + | col_map_string_array_double MAP>, + | col_map_string_map_string_date MAP>, + | + | -- Array of structs with single (inner) fields do not work with parquet.version 1.13.1 + | col_struct_array_struct STRUCT>>, + | col_struct_map STRUCT>, + | + | part_col STRING + |) USING hudi + | LOCATION '${tmp.getCanonicalPath}' + | TBLPROPERTIES ( + | primaryKey = 'uuid', + | type = 'mor', + | preCombineField = 'precombine_field' + | ) + | PARTITIONED BY (part_col) + """.stripMargin) + + // To not trigger compaction scheduling, and compaction + spark.sql(s"set hoodie.compact.inline.max.delta.commits=9999") + spark.sql(s"set hoodie.compact.inline=false") + + // Directly write to new parquet file + spark.sql(s"set hoodie.parquet.small.file.limit=0") + spark.sql(s"set hoodie.metadata.compact.max.delta.commits=1") + // Partition stats index is enabled together with column stats index + spark.sql(s"set hoodie.metadata.index.column.stats.enable=true") + spark.sql(s"set hoodie.metadata.record.index.enable=true") + + // Insert row 1 into partition 'A' + spark.sql( + s""" + | INSERT INTO $tableName VALUES ( + | 'uuid1', 1000L, + | true, cast(1 as tinyint), cast(100 as smallint), 1000, 100000L, 1.1, 10.123, cast(123.45 as decimal(10,2)), + | 'string val 1', cast('varchar val 1' as varchar(50)), cast('charval1' as char(10)), + | cast('binary1' as binary), + | cast('2025-01-15' as date), cast('2025-01-15 11:30:00' as timestamp), + | -- cast('2025-01-15 11:30:00' as timestamp_ntz), + | array(1, 2, 3), array('a', 'b', 'c'), map('key1', 10, 'key2', 20), + | struct('struct_str1', 55, false), + | array(struct(1.1, array('n1','n2')), struct(2.2, array('n3'))), + | map('mapkey1', struct(cast('2024-11-01' as date), cast(9.8 as decimal(5,2)))), + | array(struct('arr_struct1', map('map_in_struct_k1', 1)), struct('arr_struct2', map('map_in_struct_k2', 2, 'map_in_struct_k3', 3))), + | map('map_struct1', struct(array(true, false), cast('2025-01-01 01:01:01' as timestamp)), 'map_struct2', struct(array(false), cast('2025-02-02 02:02:02' as timestamp))), + | struct(101, struct('inner_str_1', true)), + | array(array(1, 2), array(3, 4, 5)), + | map('arr_key1', array(1.1, 2.2), 'arr_key2', array(3.3)), + | map('map_key1', map('mapkey10', cast('2024-01-01' as date), 'mapkey20', cast('2024-02-02' as date))), + | struct('outer_str_1', array(struct(cast('2023-11-11 11:11:11' as timestamp), 'inner_str_1'))), + | struct(true, map('struct_map_k1', 1000L, 'struct_map_k2', 2000L)), + | 'A' + | ) + """.stripMargin) + + // Insert row 2 into partition 'A' + spark.sql( + s""" + | INSERT INTO $tableName VALUES ( + | 'uuid2', 1005L, + | false, cast(2 as tinyint), cast(200 as smallint), 2000, 200000L, 2.2, 20.456, cast(234.56 as decimal(10,2)), + | 'string val 2', cast('varchar val 2' as varchar(50)), cast('charval2' as char(10)), + | cast('binary2' as binary), + | cast('2025-02-20' as date), cast('2025-02-20 12:45:00' as timestamp), + | -- cast('2025-02-20 12:45:00' as timestamp_ntz), + | array(4, 5), array('d', 'e', 'f'), map('key3', 30), + | struct('struct_str2', 66, true), + | null, + | map('mapkey2', struct(cast('2024-12-10' as date), cast(7.6 as decimal(5,2)))), + | array(struct('arr_struct3', map('map_in_struct_k4', 4)), struct('arr_struct4', null)), + | map('map_struct3', struct(null, cast('2025-03-03 03:03:03' as timestamp)), 'map_struct4', struct(array(true), null)), + | -- Additional Nested Complex Types (with nulls) + | struct(102, null), + | array(array(6), array(7, 8)), + | map('arr_key3', null), + | map('map_key2', map(30, null), 'map_key3', null), + | struct('outer_str_2', array(struct(cast('2023-12-12 12:12:12' as timestamp), 'inner_str_2'))), + | struct(false, null), + | 'A' + | ) + """.stripMargin) + + // Insert row 3 into partition 'B' + spark.sql( + s""" + | INSERT INTO $tableName VALUES ( + | 'uuid3', 1100L, + | null, null, null, null, null, null, null, null, + | null, null, null, + | null, + | null, null, + | null, null, null, + | null, + | array(struct(3.3, array('n4'))), + | null, + | null, + | null, + | null, + | null, + | null, + | null, + | null, + | null, + | 'B' + | ) + """.stripMargin) + + // Generate log files through updates on partition 'A' + spark.sql(s"UPDATE $tableName SET col_double = col_double + 100, precombine_field = precombine_field + 1 WHERE part_col = 'A'") + // Generate log files through updates on partition 'B' + spark.sql(s"UPDATE $tableName SET col_string = 'updated string', precombine_field = precombine_field + 1 WHERE part_col = 'B'") + } +} +``` diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v6_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v6_mor.zip new file mode 100644 index 0000000000000..db7388988b4c9 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v6_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v8_mor.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v8_mor.md new file mode 100644 index 0000000000000..ade3e61723df0 --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v8_mor.md @@ -0,0 +1,168 @@ +## Create script + +Structure of table: +- MOR table with MDT enabled +- Revision: 444cac26cb1077fd2b7deefc7b3713bacb270f9c + +```scala +test("Create MOR table with comprehensive types") { + withTempDir { tmp => + val tableName = "hudi_type_test_mor" + spark.sql( + s""" + |CREATE TABLE $tableName ( + | uuid STRING, + | precombine_field LONG, + | + | -- Numeric Types + | col_boolean BOOLEAN, + | col_tinyint TINYINT, + | col_smallint SMALLINT, + | col_int INT, + | col_bigint BIGINT, + | col_float FLOAT, + | col_double DOUBLE, + | col_decimal DECIMAL(10, 2), + | + | -- String Types + | col_string STRING, + | col_varchar VARCHAR(50), + | col_char CHAR(10), + | + | -- Binary Type + | col_binary BINARY, + | + | -- Datetime Types + | col_date DATE, + | col_timestamp TIMESTAMP, + | -- col_timestamp_ntz TIMESTAMP_NTZ, (No support on Hudi for now) + | + | -- Complex types + | col_array_int ARRAY, + | col_array_string ARRAY, + | col_map_string_int MAP, + | col_struct STRUCT, + | col_array_struct ARRAY>>, + | col_map_string_struct MAP>, + | col_array_struct_with_map ARRAY>>, + | col_map_struct_with_array MAP, f_map_struct_ts: TIMESTAMP>>, + | col_struct_nested_struct STRUCT>, + | col_array_array_int ARRAY>, + | col_map_string_array_double MAP>, + | col_map_string_map_string_date MAP>, + | + | -- Array of structs with single (inner) fields do not work with parquet.version 1.13.1 + | col_struct_array_struct STRUCT>>, + | col_struct_map STRUCT>, + | + | part_col STRING + |) USING hudi + | LOCATION '${tmp.getCanonicalPath}' + | TBLPROPERTIES ( + | primaryKey = 'uuid', + | type = 'mor', + | preCombineField = 'precombine_field' + | ) + | PARTITIONED BY (part_col) + """.stripMargin) + + // To not trigger compaction scheduling, and compaction + spark.sql(s"set hoodie.compact.inline.max.delta.commits=9999") + spark.sql(s"set hoodie.compact.inline=false") + + // Directly write to new parquet file + spark.sql(s"set hoodie.parquet.small.file.limit=0") + spark.sql(s"set hoodie.metadata.compact.max.delta.commits=1") + // Partition stats index is enabled together with column stats index + spark.sql(s"set hoodie.metadata.index.column.stats.enable=true") + spark.sql(s"set hoodie.metadata.record.index.enable=true") + spark.sql(s"set hoodie.metadata.index.secondary.enable=true") + + // Insert row 1 into partition 'A' + spark.sql( + s""" + | INSERT INTO $tableName VALUES ( + | 'uuid1', 1000L, + | true, cast(1 as tinyint), cast(100 as smallint), 1000, 100000L, 1.1, 10.123, cast(123.45 as decimal(10,2)), + | 'string val 1', cast('varchar val 1' as varchar(50)), cast('charval1' as char(10)), + | cast('binary1' as binary), + | cast('2025-01-15' as date), cast('2025-01-15 11:30:00' as timestamp), + | -- cast('2025-01-15 11:30:00' as timestamp_ntz), + | array(1, 2, 3), array('a', 'b', 'c'), map('key1', 10, 'key2', 20), + | struct('struct_str1', 55, false), + | array(struct(1.1, array('n1','n2')), struct(2.2, array('n3'))), + | map('mapkey1', struct(cast('2024-11-01' as date), cast(9.8 as decimal(5,2)))), + | array(struct('arr_struct1', map('map_in_struct_k1', 1)), struct('arr_struct2', map('map_in_struct_k2', 2, 'map_in_struct_k3', 3))), + | map('map_struct1', struct(array(true, false), cast('2025-01-01 01:01:01' as timestamp)), 'map_struct2', struct(array(false), cast('2025-02-02 02:02:02' as timestamp))), + | struct(101, struct('inner_str_1', true)), + | array(array(1, 2), array(3, 4, 5)), + | map('arr_key1', array(1.1, 2.2), 'arr_key2', array(3.3)), + | map('map_key1', map('mapkey10', cast('2024-01-01' as date), 'mapkey20', cast('2024-02-02' as date))), + | struct('outer_str_1', array(struct(cast('2023-11-11 11:11:11' as timestamp), 'inner_str_1'))), + | struct(true, map('struct_map_k1', 1000L, 'struct_map_k2', 2000L)), + | 'A' + | ) + """.stripMargin) + + // Insert row 2 into partition 'A' + spark.sql( + s""" + | INSERT INTO $tableName VALUES ( + | 'uuid2', 1005L, + | false, cast(2 as tinyint), cast(200 as smallint), 2000, 200000L, 2.2, 20.456, cast(234.56 as decimal(10,2)), + | 'string val 2', cast('varchar val 2' as varchar(50)), cast('charval2' as char(10)), + | cast('binary2' as binary), + | cast('2025-02-20' as date), cast('2025-02-20 12:45:00' as timestamp), + | -- cast('2025-02-20 12:45:00' as timestamp_ntz), + | array(4, 5), array('d', 'e', 'f'), map('key3', 30), + | struct('struct_str2', 66, true), + | null, + | map('mapkey2', struct(cast('2024-12-10' as date), cast(7.6 as decimal(5,2)))), + | array(struct('arr_struct3', map('map_in_struct_k4', 4)), struct('arr_struct4', null)), + | map('map_struct3', struct(null, cast('2025-03-03 03:03:03' as timestamp)), 'map_struct4', struct(array(true), null)), + | -- Additional Nested Complex Types (with nulls) + | struct(102, null), + | array(array(6), array(7, 8)), + | map('arr_key3', null), + | map('map_key2', map(30, null), 'map_key3', null), + | struct('outer_str_2', array(struct(cast('2023-12-12 12:12:12' as timestamp), 'inner_str_2'))), + | struct(false, null), + | 'A' + | ) + """.stripMargin) + + // Insert row 3 into partition 'B' + spark.sql( + s""" + | INSERT INTO $tableName VALUES ( + | 'uuid3', 1100L, + | null, null, null, null, null, null, null, null, + | null, null, null, + | null, + | null, null, + | null, null, null, + | null, + | array(struct(3.3, array('n4'))), + | null, + | null, + | null, + | null, + | null, + | null, + | null, + | null, + | null, + | 'B' + | ) + """.stripMargin) + + // Create Secondary Index on col_double + spark.sql(s"CREATE INDEX idx_double ON $tableName (col_double)") + + // Generate log files through updates on partition 'A' + spark.sql(s"UPDATE $tableName SET col_double = col_double + 100, precombine_field = precombine_field + 1 WHERE part_col = 'A'") + // Generate log files through updates on partition 'B' + spark.sql(s"UPDATE $tableName SET col_string = 'updated string', precombine_field = precombine_field + 1 WHERE part_col = 'B'") + } +} +``` diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v8_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v8_mor.zip new file mode 100644 index 0000000000000..0301e37cf3367 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_comprehensive_types_v8_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl.zip new file mode 100644 index 0000000000000..2f2238b22339f Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_custom_keygen_pt_v8_mor.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_custom_keygen_pt_v8_mor.md new file mode 100644 index 0000000000000..f38d43d149c2b --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_custom_keygen_pt_v8_mor.md @@ -0,0 +1,64 @@ +## Create script + +Structure of table: +- MOR table with CustomKeyGenerator +- Revision: f7157954a9819137446d8e3a1d331d003f069414 +- No log files + + +```scala +test("Create MOR table with custom keygen partition field") { + withTempDir { tmp => + val tableName = "hudi_custom_keygen_pt_v8_mor" + + spark.sql( + s""" + |CREATE TABLE $tableName ( + | id INT, + | name STRING, + | price DOUBLE, + | ts LONG, + | -- Partition Source Fields -- + | partition_field_country STRING, + | partition_field_date BIGINT + |) USING hudi + | LOCATION '${tmp.getCanonicalPath}' + | TBLPROPERTIES ( + | primaryKey = 'id', + | type = 'mor', + | preCombineField = 'ts', + | -- Timestamp Keygen and Partition Configs -- + | hoodie.table.keygenerator.class = 'org.apache.hudi.keygen.CustomKeyGenerator', + | hoodie.datasource.write.partitionpath.field = 'partition_field_country:SIMPLE,partition_field_date:TIMESTAMP', + | hoodie.keygen.timebased.timestamp.type = 'EPOCHMILLISECONDS', + | hoodie.keygen.timebased.output.dateformat = 'yyyy-MM-dd', + | hoodie.keygen.timebased.timezone = 'UTC' + | ) PARTITIONED BY (partition_field_country, partition_field_date) + """.stripMargin) + + // To not trigger compaction scheduling, and compaction + spark.sql(s"set hoodie.compact.inline.max.delta.commits=9999") + spark.sql(s"set hoodie.compact.inline=false") + + // Configure Hudi properties + spark.sql(s"SET hoodie.metadata.enable=true") + spark.sql(s"SET hoodie.metadata.index.column.stats.enable=true") + + // Insert data with new partition values + spark.sql(s"INSERT INTO $tableName VALUES(1, 'a1', 100.0, 1000, 'SG', 1749284360000)") + spark.sql(s"INSERT INTO $tableName VALUES(2, 'a2', 200.0, 1000, 'SG', 1749204000000)") + spark.sql(s"INSERT INTO $tableName VALUES(3, 'a3', 101.0, 1001, 'US', 1749202000000)") + spark.sql(s"INSERT INTO $tableName VALUES(4, 'a4', 201.0, 1001, 'CN', 1749102000000)") + spark.sql(s"INSERT INTO $tableName VALUES(5, 'a5', 300.0, 1002, 'MY', 1747102000000)") + spark.sql(s"INSERT INTO $tableName VALUES(6, 'a6', 301.0, 1000, 'SG', 1749284360000)") + spark.sql(s"INSERT INTO $tableName VALUES(7, 'a7', 401.0, 1000, 'SG', 1749204000000)") + + // Generate logs through updates + // NOTE: The query below will throw an error + // spark.sql(s"UPDATE $tableName SET price = ROUND(price * 1.02, 2)") + + // NOTE: The query below will throw an error + // spark.sql(s"SELECT * FROM $tableName").show(false) + } +} +``` diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_custom_keygen_pt_v8_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_custom_keygen_pt_v8_mor.zip new file mode 100644 index 0000000000000..79d54bd11353c Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_custom_keygen_pt_v8_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v6_mor.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v6_mor.md new file mode 100644 index 0000000000000..733443b8619dd --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v6_mor.md @@ -0,0 +1,51 @@ +## Create script + +Structure of table: +- MOR table with MDT enabled +- Revision: tag release-0.15.0 +- Record Level Index is enabled with a recordKey of id,name +- 2 Partitions [US, SG] +- 2 Filegroups per partition + +```scala +test("Create table multi filegroup partitioned mor") { + withTempDir { tmp => + val tableName = "hudi_multi_fg_pt_mor" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | country string + |) using hudi + | location '${tmp.getCanonicalPath}' + | tblproperties ( + | primaryKey ='id,name', + | type = 'mor', + | preCombineField = 'ts' + | ) partitioned by (country) + """.stripMargin) + // directly write to new parquet file + spark.sql(s"set hoodie.parquet.small.file.limit=0") + spark.sql(s"set hoodie.metadata.compact.max.delta.commits=1") + // partition stats index is enabled together with column stats index + spark.sql(s"set hoodie.metadata.index.column.stats.enable=true") + spark.sql(s"set hoodie.metadata.record.index.enable=true") + spark.sql(s"set hoodie.metadata.index.column.stats.column.list=_hoodie_commit_time,_hoodie_partition_path,_hoodie_record_key,id,name,price,ts,country") + // 2 filegroups per partition + spark.sql(s"insert into $tableName values(1, 'a1', 100, 1000, 'SG'),(2, 'a2', 200, 1000, 'US')") + spark.sql(s"insert into $tableName values(3, 'a3', 101, 1001, 'SG'),(4, 'a3', 201, 1001, 'US')") + // generate logs through updates + spark.sql(s"update $tableName set price=price+1") + } +} +``` + +# When to use this table? +- For test cases that require multiple filegroups in a partition +- For test cases that require filegroups that have a log file +- For test cases that require column stats index +- For test cases that require partition stats index +- For test cases that require record level index diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v6_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v6_mor.zip new file mode 100644 index 0000000000000..35588f5614fd7 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v6_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v8_mor.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v8_mor.md new file mode 100644 index 0000000000000..6d78d7eb6768f --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v8_mor.md @@ -0,0 +1,56 @@ +## Create script + +Structure of table: +- MOR table with MDT enabled +- Revision: eb212c9dca876824b6c570665951777a772bc463 +- Record Level Index is enabled with a recordKey of id,name +- Secondary index created on the column `price` +- 2 Partitions [US, SG] +- 2 Filegroups per partition + +```scala +test("Create table multi filegroup partitioned mor") { + withTempDir { tmp => + val tableName = "hudi_multi_fg_pt_mor" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | country string + |) using hudi + | location '${tmp.getCanonicalPath}' + | tblproperties ( + | primaryKey ='id,name', + | type = 'mor', + | preCombineField = 'ts' + | ) partitioned by (country) + """.stripMargin) + // directly write to new parquet file + spark.sql(s"set hoodie.parquet.small.file.limit=0") + spark.sql(s"set hoodie.metadata.compact.max.delta.commits=1") + // partition stats index is enabled together with column stats index + spark.sql(s"set hoodie.metadata.index.column.stats.enable=true") + spark.sql(s"set hoodie.metadata.record.index.enable=true") + spark.sql(s"set hoodie.metadata.index.secondary.enable=true") + spark.sql(s"set hoodie.metadata.index.column.stats.column.list=_hoodie_commit_time,_hoodie_partition_path,_hoodie_record_key,id,name,price,ts,country") + // 2 filegroups per partition + spark.sql(s"insert into $tableName values(1, 'a1', 100, 1000, 'SG'),(2, 'a2', 200, 1000, 'US')") + spark.sql(s"insert into $tableName values(3, 'a3', 101, 1001, 'SG'),(4, 'a3', 201, 1001, 'US')") + // create secondary index + spark.sql(s"create index idx_price on $tableName (price)") + // generate logs through updates + spark.sql(s"update $tableName set price=price+1") + } +} +``` + +# When to use this table? +- For test cases that require multiple filegroups in a partition +- For test cases that require filegroups that have a log file +- For test cases that require column stats index +- For test cases that require partition stats index +- For test cases that require record level index +- For test cases that require secondary index diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v8_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v8_mor.zip new file mode 100644 index 0000000000000..41e046e9b31b0 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_fg_pt_v8_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_pt_v8_mor.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_pt_v8_mor.md new file mode 100644 index 0000000000000..119c8ffdd8e7c --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_pt_v8_mor.md @@ -0,0 +1,56 @@ +## Create script +Revision: 6f65998117a2d1228fc96d36053bd0d394499afe + +```scala +test("Create MOR table with multiple partition fields with multiple types") { + withTempDir { tmp => + val tableName = "hudi_multi_pt_v8_mor" + // Save current session timezone and set to UTC for consistency in test + val originalTimeZone = spark.conf.get("spark.sql.session.timeZone") + spark.conf.set("spark.sql.session.timeZone", "UTC") + + spark.sql( + s""" + |CREATE TABLE $tableName ( + | id INT, + | name STRING, + | price DOUBLE, + | ts LONG, + | -- Partition Fields -- + | part_str STRING, + | part_int INT, + | part_date DATE, + | part_bigint BIGINT, + | part_decimal DECIMAL(10,2), + | part_timestamp TIMESTAMP, + | part_bool BOOLEAN + |) USING hudi + | LOCATION '${tmp.getCanonicalPath}' + | TBLPROPERTIES ( + | primaryKey = 'id,name', + | type = 'mor', + | preCombineField = 'ts' + | ) + | PARTITIONED BY (part_str, part_int, part_date, part_bigint, part_decimal, part_timestamp, part_bool) + """.stripMargin) + + // Configure Hudi properties + spark.sql(s"SET hoodie.parquet.small.file.limit=0") // Write to a new parquet file for each commit + spark.sql(s"SET hoodie.metadata.compact.max.delta.commits=1") + spark.sql(s"SET hoodie.metadata.enable=true") + spark.sql(s"SET hoodie.metadata.index.column.stats.enable=true") + spark.sql(s"SET hoodie.compact.inline.max.delta.commits=9999") // Disable compaction plan trigger + + // Insert data with new partition values + spark.sql(s"INSERT INTO $tableName VALUES(1, 'a1', 100.0, 1000, 'books', 2023, date'2023-01-15', 10000000001L, decimal('123.45'), timestamp'2023-01-15 10:00:00.123', true)") + spark.sql(s"INSERT INTO $tableName VALUES(2, 'a2', 200.0, 1000, 'electronics', 2023, date'2023-03-10', 10000000002L, decimal('50.20'), timestamp'2023-03-10 12:30:00.000', false)") + spark.sql(s"INSERT INTO $tableName VALUES(3, 'a3', 101.0, 1001, 'books', 2024, date'2024-02-20', 10000000003L, decimal('75.00'), timestamp'2024-02-20 08:45:10.456', true)") + spark.sql(s"INSERT INTO $tableName VALUES(4, 'a4', 201.0, 1001, 'electronics', 2023, date'2023-03-10', 10000000002L, decimal('50.20'), timestamp'2023-03-10 12:30:00.000', true)") // Same as record 2 part except boolean + spark.sql(s"INSERT INTO $tableName VALUES(5, 'a5', 300.0, 1002, 'apparel', 2024, date'2024-01-05', 20000000001L, decimal('99.99'), timestamp'2024-01-05 18:00:00.789', false)") + + // Generate logs through updates + spark.sql(s"UPDATE $tableName SET price = price + 2.0 WHERE part_bool = true AND part_str = 'books'") + spark.sql(s"UPDATE $tableName SET price = ROUND(price * 1.02, 2) WHERE part_bigint = 10000000002L") + } +} +``` diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_pt_v8_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_pt_v8_mor.zip new file mode 100644 index 0000000000000..c19c703f6fd0f Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_multi_pt_v8_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_non_part_cow.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_non_part_cow.md new file mode 100644 index 0000000000000..26f65f3fcd78f --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_non_part_cow.md @@ -0,0 +1,47 @@ +# Hudi Test Resources + +## Generating Hudi Resources + +Follow these steps to create the `hudi_non_part_cow` test table and utilize it for testing. `hudi_non_part_cow` resource is generated using `423` trino version. + +### Start the Hudi environment + +Execute the following command in the terminal to initiate the Hudi environment: + +```shell +testing/bin/ptl env up --environment singlenode-hudi +``` + +### Generate Resources + +* Open the `spark-sql` terminal and initiate the `spark-sql` shell in the `ptl-spark` container. +* Execute the following Spark SQL queries to create the `hudi_non_part_cow` table: + +``` +spark-sql> CREATE TABLE default.hudi_non_part_cow ( + id bigint, + name string, + ts bigint, + dt string, + hh string + ) + USING hudi + TBLPROPERTIES ( + type = 'cow', + primaryKey = 'id', + preCombineField = 'ts' + ) + LOCATION 's3://test-bucket/hudi_non_part_cow'; + +spark-sql> INSERT INTO default.hudi_non_part_cow (id, name, ts, dt, hh) VALUES + (1, 'a1', 1000, '2021-12-09', '10'), + (2, 'a2', 2000, '2021-12-09', '11'); +``` + +### Download Resources + +Download the `hudi_non_part_cow` table from the MinIO client http://localhost:9001/buckets/test-bucket/browse. + +### Use Resources + +Unzip the downloaded `hudi_non_part_cow.zip`. Remove any unnecessary files obtained after unzipping to prepare the resource for testing. diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_non_part_cow.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_non_part_cow.zip new file mode 100644 index 0000000000000..019860e88cff3 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_non_part_cow.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_stock_ticks_cow.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_stock_ticks_cow.zip new file mode 100644 index 0000000000000..55381fd8ce6e9 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_stock_ticks_cow.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_stock_ticks_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_stock_ticks_mor.zip new file mode 100644 index 0000000000000..af79a583ccf6e Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_stock_ticks_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_epoch_to_yyyy_mm_dd_hh_v8_mor.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_epoch_to_yyyy_mm_dd_hh_v8_mor.md new file mode 100644 index 0000000000000..9e426b9fd3783 --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_epoch_to_yyyy_mm_dd_hh_v8_mor.md @@ -0,0 +1,63 @@ +## Create script + +Structure of table: +- MOR table with MDT enabled +- Timestamp key generator used (EPOCHMILLISECONDS -> yyyy-mm-dd) +- Hive style partitioning disabled +- Revision: 444cac26cb1077fd2b7deefc7b3713bacb270f9c + +```scala +test("Create MOR table with timestamp keygen partition field (EPOCHMILLISECONDS -> yyyy-mm-dd hh)") { +withTempDir { tmp => +val tableName = "hudi_timestamp_keygen_pt_epoch_to_yyyy_mm_dd_hh_v8_mor" + + spark.sql( + s""" + |CREATE TABLE $tableName ( + | id INT, + | name STRING, + | price DOUBLE, + | ts LONG, + | -- Partition Source Fields -- + | partition_field bigint + |) USING hudi + | LOCATION '${tmp.getCanonicalPath}' + | TBLPROPERTIES ( + | primaryKey = 'id', + | type = 'mor', + | preCombineField = 'ts', + | -- Hive style partitioning needs to be disabled for timestamp keygen to work -- + | hoodie.datasource.write.hive_style_partitioning = 'false', + | -- Timestamp Keygen and Partition Configs -- + | hoodie.table.keygenerator.class = 'org.apache.hudi.keygen.TimestampBasedKeyGenerator', + | hoodie.datasource.write.partitionpath.field = 'partition_field', + | hoodie.keygen.timebased.timestamp.type = 'EPOCHMILLISECONDS', + | hoodie.keygen.timebased.output.dateformat = 'yyyy-MM-dd hh', + | hoodie.keygen.timebased.timezone = 'UTC' + | ) PARTITIONED BY (partition_field) + """.stripMargin) + + // To not trigger compaction scheduling, and compaction + spark.sql(s"set hoodie.compact.inline.max.delta.commits=9999") + spark.sql(s"set hoodie.compact.inline=false") + + // Configure Hudi properties + spark.sql(s"SET hoodie.parquet.small.file.limit=0") // Write to a new parquet file for each commit + spark.sql(s"SET hoodie.metadata.compact.max.delta.commits=1") + spark.sql(s"SET hoodie.metadata.enable=true") + spark.sql(s"SET hoodie.metadata.index.column.stats.enable=true") + + // Insert data with new partition values + spark.sql(s"INSERT INTO $tableName VALUES(1, 'a1', 100.0, 1000, 1749284360000L)") + spark.sql(s"INSERT INTO $tableName VALUES(2, 'a2', 200.0, 1000, 1749204000000L)") + spark.sql(s"INSERT INTO $tableName VALUES(3, 'a3', 101.0, 1001, 1749202000000L)") + spark.sql(s"INSERT INTO $tableName VALUES(4, 'a4', 201.0, 1001, 1749102000000L)") + spark.sql(s"INSERT INTO $tableName VALUES(5, 'a5', 300.0, 1002, 1747102000000L)") + + // Generate logs through updates + spark.sql(s"UPDATE $tableName SET price = ROUND(price * 1.02, 2)") + + spark.sql(s"SELECT * FROM $tableName").show(false) + } +} +``` diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_epoch_to_yyyy_mm_dd_hh_v8_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_epoch_to_yyyy_mm_dd_hh_v8_mor.zip new file mode 100644 index 0000000000000..9726c3b40270d Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_epoch_to_yyyy_mm_dd_hh_v8_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_scalar_to_yyyy_mm_dd_hh_v8_mor.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_scalar_to_yyyy_mm_dd_hh_v8_mor.md new file mode 100644 index 0000000000000..f547b378067f3 --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_scalar_to_yyyy_mm_dd_hh_v8_mor.md @@ -0,0 +1,64 @@ +## Create script + +Structure of table: +- MOR table with MDT enabled +- Timestamp key generator used (SCALAR -> yyyy-mm-dd) +- Hive style partitioning disabled +- Revision: 444cac26cb1077fd2b7deefc7b3713bacb270f9c + +```scala +test("Create MOR table with timestamp keygen partition field (SCALAR -> yyyy-mm-dd hh)") { + withTempDir { tmp => + val tableName = "hudi_timestamp_keygen_pt_scalar_to_yyyy_mm_dd_hh_v8_mor" + + spark.sql( + s""" + |CREATE TABLE $tableName ( + | id INT, + | name STRING, + | price DOUBLE, + | ts LONG, + | -- Partition Source Fields -- + | partition_field bigint + |) USING hudi + | LOCATION '${tmp.getCanonicalPath}' + | TBLPROPERTIES ( + | primaryKey = 'id', + | type = 'mor', + | preCombineField = 'ts', + | -- Hive style partitioning needs to be disabled for timestamp keygen to work -- + | hoodie.datasource.write.hive_style_partitioning = 'false', + | -- Timestamp Keygen and Partition Configs -- + | hoodie.table.keygenerator.class = 'org.apache.hudi.keygen.TimestampBasedKeyGenerator', + | hoodie.datasource.write.partitionpath.field = 'part_source', + | hoodie.keygen.timebased.timestamp.type = 'SCALAR', + | hoodie.keygen.timebased.output.dateformat = 'yyyy-MM-dd hh', + | hoodie.keygen.timebased.timezone = 'UTC', + | hoodie.keygen.timebased.timestamp.scalar.time.unit = 'DAYS' + | ) PARTITIONED BY (partition_field) + """.stripMargin) + + // To not trigger compaction scheduling, and compaction + spark.sql(s"set hoodie.compact.inline.max.delta.commits=9999") + spark.sql(s"set hoodie.compact.inline=false") + + // Configure Hudi properties + spark.sql(s"SET hoodie.parquet.small.file.limit=0") // Write to a new parquet file for each commit + spark.sql(s"SET hoodie.metadata.compact.max.delta.commits=1") + spark.sql(s"SET hoodie.metadata.enable=true") + spark.sql(s"SET hoodie.metadata.index.column.stats.enable=true") + + // Insert data with new partition values + spark.sql(s"INSERT INTO $tableName VALUES(1, 'a1', 100.0, 1000, 20000L)") + spark.sql(s"INSERT INTO $tableName VALUES(2, 'a2', 200.0, 1000, 20001L)") + spark.sql(s"INSERT INTO $tableName VALUES(3, 'a3', 101.0, 1001, 20002L)") + spark.sql(s"INSERT INTO $tableName VALUES(4, 'a4', 201.0, 1001, 20003L)") + spark.sql(s"INSERT INTO $tableName VALUES(5, 'a5', 300.0, 1002, 20004L)") + + // Generate logs through updates + spark.sql(s"UPDATE $tableName SET price = ROUND(price * 1.02, 2)") + + spark.sql(s"SELECT * FROM $tableName").show(false) + } +} +``` diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_scalar_to_yyyy_mm_dd_hh_v8_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_scalar_to_yyyy_mm_dd_hh_v8_mor.zip new file mode 100644 index 0000000000000..9475e50e67c9c Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_timestamp_keygen_pt_scalar_to_yyyy_mm_dd_hh_v8_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_trips_cow_v8.md b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_trips_cow_v8.md new file mode 100644 index 0000000000000..a263f502e4155 --- /dev/null +++ b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_trips_cow_v8.md @@ -0,0 +1,34 @@ +## Create script + +Structure of table: +- COW table in table version 8 with MDT and column stats enabled +- Using Hudi 1.0.2 release +- Non-partitioned table +- One large parquet file for testing projection and reader + +```scala +import scala.collection.JavaConversions._ +import org.apache.spark.sql.SaveMode._ +import org.apache.hudi.DataSourceReadOptions._ +import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.common.table.HoodieTableConfig._ +import org.apache.hudi.config.HoodieWriteConfig._ +import org.apache.hudi.keygen.constant.KeyGeneratorOptions._ +import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.QuickstartUtils._ +import spark.implicits._ + +val tableName = "hudi_trips_cow_v8" +val basePath = "file:///tmp/hudi_trips_cow_v8" + +val dataGen = new DataGenerator +val inserts = convertToStringList(dataGen.generateInserts(40000)) +val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1)) +df.write.format("hudi"). + options(getQuickstartWriteConfigs). + option(RECORDKEY_FIELD_OPT_KEY, "uuid"). + option(PARTITIONPATH_FIELD_OPT_KEY, ""). + option(TABLE_NAME, tableName). + mode(Overwrite). + save(basePath) +``` diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_trips_cow_v8.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_trips_cow_v8.zip new file mode 100644 index 0000000000000..addef00823912 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_trips_cow_v8.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_v6_composite_key.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_v6_composite_key.zip new file mode 100644 index 0000000000000..9cf37b3e92d04 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/hudi_v6_composite_key.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/stock_ticks_cow.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/stock_ticks_cow.zip new file mode 100644 index 0000000000000..24a2b447e9b86 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/stock_ticks_cow.zip differ diff --git a/hudi-trino-plugin/src/test/resources/hudi-testing-data/stock_ticks_mor.zip b/hudi-trino-plugin/src/test/resources/hudi-testing-data/stock_ticks_mor.zip new file mode 100644 index 0000000000000..484ece3a78be6 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/hudi-testing-data/stock_ticks_mor.zip differ diff --git a/hudi-trino-plugin/src/test/resources/long_timestamp.parquet b/hudi-trino-plugin/src/test/resources/long_timestamp.parquet new file mode 100644 index 0000000000000..143601f3d0903 Binary files /dev/null and b/hudi-trino-plugin/src/test/resources/long_timestamp.parquet differ diff --git a/scripts/release/validate_source_copyright.sh b/scripts/release/validate_source_copyright.sh index deb7966626904..7d7e2914f328f 100755 --- a/scripts/release/validate_source_copyright.sh +++ b/scripts/release/validate_source_copyright.sh @@ -46,10 +46,10 @@ echo -e "\t\tNotice file exists ? [OK]\n" ### Licensing Check echo "Performing custom Licensing Check " -numfilesWithNoLicense=`find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.jpg' | grep -v '.json' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v emptyFile | grep -v DISCLAIMER | grep -v KEYS | grep -v '.mailmap' | grep -v '.sqltemplate' | grep -v 'banner.txt' | grep -v '.txt' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" | wc -l` +numfilesWithNoLicense=`find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.jpg' | grep -v '.json' | grep -v '.parquet' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v emptyFile | grep -v DISCLAIMER | grep -v KEYS | grep -v '.mailmap' | grep -v '.sqltemplate' | grep -v 'banner.txt' | grep -v '.txt' | grep -v "fixtures" | xargs grep -L -E "(Licensed to the Apache Software Foundation \(ASF\)|Licensed under the Apache License, Version 2\.0)" | wc -l` if [ "$numfilesWithNoLicense" -gt "0" ]; then echo "There were some source files that did not have Apache License [ERROR]" - find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.jpg' | grep -v '.json' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v emptyFile | grep -v DISCLAIMER | grep -v '.sqltemplate' | grep -v KEYS | grep -v '.mailmap' | grep -v 'banner.txt' | grep -v '.txt' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" + find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.jpg' | grep -v '.json' | grep -v '.parquet' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v emptyFile | grep -v DISCLAIMER | grep -v '.sqltemplate' | grep -v KEYS | grep -v '.mailmap' | grep -v 'banner.txt' | grep -v '.txt' | grep -v "fixtures" | xargs grep -L -E "(Licensed to the Apache Software Foundation \(ASF\)|Licensed under the Apache License, Version 2\.0)" exit 1 fi echo -e "\t\tLicensing Check Passed [OK]\n" diff --git a/style/checkstyle-suppressions.xml b/style/checkstyle-suppressions.xml index 8f6697b4083fe..61071e694fb02 100644 --- a/style/checkstyle-suppressions.xml +++ b/style/checkstyle-suppressions.xml @@ -33,4 +33,6 @@ + +