diff --git a/core/trino-server/src/main/provisio/presto.xml b/core/trino-server/src/main/provisio/presto.xml index 48190e7b21a7d..02575a105ae4d 100644 --- a/core/trino-server/src/main/provisio/presto.xml +++ b/core/trino-server/src/main/provisio/presto.xml @@ -74,6 +74,12 @@ + + + + + + diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java index 5546808b2bdbb..37f33bc6cfaa5 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java @@ -28,7 +28,6 @@ import io.trino.plugin.hive.HiveSplit.BucketConversion; import io.trino.plugin.hive.HiveSplit.BucketValidation; import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.util.HiveBucketing.BucketingVersion; @@ -100,8 +99,6 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_BAD_DATA; import static io.trino.plugin.hive.HiveErrorCode.HIVE_FILESYSTEM_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_BUCKET_FILES; -import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; -import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE; import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR; import static io.trino.plugin.hive.HivePartitionManager.partitionMatches; import static io.trino.plugin.hive.HiveSessionProperties.getMaxInitialSplitSize; @@ -114,11 +111,11 @@ import static io.trino.plugin.hive.util.HiveFileIterator.NestedDirectoryPolicy.FAIL; import static io.trino.plugin.hive.util.HiveFileIterator.NestedDirectoryPolicy.IGNORED; import static io.trino.plugin.hive.util.HiveFileIterator.NestedDirectoryPolicy.RECURSE; -import static io.trino.plugin.hive.util.HiveUtil.checkCondition; import static io.trino.plugin.hive.util.HiveUtil.getFooterCount; import static io.trino.plugin.hive.util.HiveUtil.getHeaderCount; import static io.trino.plugin.hive.util.HiveUtil.getInputFormat; import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeyColumnHandles; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static java.lang.Integer.parseInt; import static java.lang.Math.max; @@ -942,28 +939,6 @@ private static List getTargetPathsFromSymlink(FileSystem fileSystem, Path } } - private static List getPartitionKeys(Table table, Optional partition) - { - if (partition.isEmpty()) { - return ImmutableList.of(); - } - ImmutableList.Builder partitionKeys = ImmutableList.builder(); - List keys = table.getPartitionColumns(); - List values = partition.get().getValues(); - checkCondition(keys.size() == values.size(), HIVE_INVALID_METADATA, "Expected %s partition key values, but got %s", keys.size(), values.size()); - for (int i = 0; i < keys.size(); i++) { - String name = keys.get(i).getName(); - HiveType hiveType = keys.get(i).getType(); - if (!hiveType.isSupportedType(table.getStorage().getStorageFormat())) { - throw new TrinoException(NOT_SUPPORTED, format("Unsupported Hive type %s found in partition keys of table %s.%s", hiveType, table.getDatabaseName(), table.getTableName())); - } - String value = values.get(i); - checkCondition(value != null, HIVE_INVALID_PARTITION_VALUE, "partition key value cannot be null for field: %s", name); - partitionKeys.add(new HivePartitionKey(name, value)); - } - return partitionKeys.build(); - } - private static Properties getPartitionSchema(Table table, Optional partition) { if (partition.isEmpty()) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java index d72571d273633..d57d4a3909cfe 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java @@ -290,9 +290,9 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq Optional readerProjections = projectBaseColumns(columns); List baseColumns = readerProjections.map(projection -> - projection.get().stream() - .map(HiveColumnHandle.class::cast) - .collect(toUnmodifiableList())) + projection.get().stream() + .map(HiveColumnHandle.class::cast) + .collect(toUnmodifiableList())) .orElse(columns); for (HiveColumnHandle column : baseColumns) { @@ -365,7 +365,7 @@ public static Optional getColumnType(HiveColumnH return Optional.of(new GroupType(baseType.getRepetition(), baseType.getName(), ImmutableList.of(type))); } - private static Optional getColumnIndexStore( + public static Optional getColumnIndexStore( ParquetDataSource dataSource, BlockMetaData blockMetadata, Map, RichColumnDescriptor> descriptorsByPath, @@ -440,7 +440,7 @@ public static TupleDomain getParquetTupleDomain( return TupleDomain.withColumnDomains(predicate.build()); } - private static org.apache.parquet.schema.Type getParquetType(HiveColumnHandle column, MessageType messageType, boolean useParquetColumnNames) + public static org.apache.parquet.schema.Type getParquetType(HiveColumnHandle column, MessageType messageType, boolean useParquetColumnNames) { if (useParquetColumnNames) { return getParquetTypeByName(column.getBaseColumnName(), messageType); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java index c6d398402f0b3..100809bdef90e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java @@ -32,6 +32,7 @@ import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.avro.TrinoAvroSerDe; import io.trino.plugin.hive.metastore.Column; +import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.SortingColumn; import io.trino.plugin.hive.metastore.Table; import io.trino.spi.ErrorCodeSupplier; @@ -1143,4 +1144,26 @@ public static boolean isIcebergTable(Table table) { return ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(table.getParameters().get(ICEBERG_TABLE_TYPE_NAME)); } + + public static List getPartitionKeys(Table table, Optional partition) + { + if (partition.isEmpty()) { + return ImmutableList.of(); + } + ImmutableList.Builder partitionKeys = ImmutableList.builder(); + List keys = table.getPartitionColumns(); + List values = partition.get().getValues(); + checkCondition(keys.size() == values.size(), HIVE_INVALID_METADATA, "Expected %s partition key values, but got %s", keys.size(), values.size()); + for (int i = 0; i < keys.size(); i++) { + String name = keys.get(i).getName(); + HiveType hiveType = keys.get(i).getType(); + if (!hiveType.isSupportedType(table.getStorage().getStorageFormat())) { + throw new TrinoException(NOT_SUPPORTED, format("Unsupported Hive type %s found in partition keys of table %s.%s", hiveType, table.getDatabaseName(), table.getTableName())); + } + String value = values.get(i); + checkCondition(value != null, HIVE_INVALID_PARTITION_VALUE, "partition key value cannot be null for field: %s", name); + partitionKeys.add(new HivePartitionKey(name, value)); + } + return partitionKeys.build(); + } } diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml new file mode 100644 index 0000000000000..8c91dcfa1b248 --- /dev/null +++ b/plugin/trino-hudi/pom.xml @@ -0,0 +1,510 @@ + + + 4.0.0 + + + trino-root + io.trino + 368-SNAPSHOT + ../../pom.xml + + + trino-hudi + Trino - Hudi Connector + trino-plugin + + + ${project.parent.basedir} + 0.11.0-SNAPSHOT + + + + + io.trino + trino-hive + + + io.trino + trino-memory-context + + + io.trino + trino-parquet + + + io.trino + trino-plugin-toolkit + + + io.trino.hadoop + hadoop-apache + + + io.trino.hive + hive-apache + + + io.airlift + bootstrap + + + io.airlift + configuration + + + io.airlift + event + + + io.airlift + json + + + io.airlift + log + + + io.airlift + units + + + com.google.guava + guava + + + com.google.inject + guice + + + javax.inject + javax.inject + + + javax.validation + validation-api + + + joda-time + joda-time + + + org.apache.hudi + hudi-common + ${dep.hudi.version} + + + org.apache.hbase + hbase-server + + + org.apache.orc + orc-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + fluent-hc + + + org.rocksdb + rocksdbjni + + + com.esotericsoftware + kryo-shaded + + + org.apache.hadoop + hadoop-client + + + org.apache.httpcomponents + httpcore + + + org.apache.hive + hive-exec + + + org.apache.hive + hive-jdbc + + + + + org.apache.hudi + hudi-hadoop-mr + ${dep.hudi.version} + + + org.apache.hbase + hbase-server + + + org.apache.orc + orc-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + fluent-hc + + + org.rocksdb + rocksdbjni + + + com.esotericsoftware + kryo-shaded + + + org.apache.hadoop + hadoop-client + + + org.apache.httpcomponents + httpcore + + + org.apache.hive + hive-exec + + + org.apache.hive + hive-jdbc + + + + + org.weakref + jmxutils + + + io.airlift + log-manager + runtime + + + org.apache.hbase + hbase-common + 1.7.1 + runtime + + + commons-codec + commons-codec + + + commons-lang + commons-lang + + + commons-logging + commons-logging + + + log4j + log4j + + + org.apache.commons + commons-lang + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.javassist + javassist + + + + + org.apache.hbase + hbase-hadoop-compat + 1.7.1 + runtime + + + commons-logging + commons-logging + + + log4j + log4j + + + + + org.apache.hbase + hbase-server + 1.7.1 + runtime + + + asm + asm + + + com.sun.jersey + jersey-core + + + com.sun.jersey + jersey-server + + + commons-codec + commons-codec + + + commons-lang + commons-lang + + + commons-logging + commons-logging + + + io.netty + netty-handler + + + io.netty + netty-transport-native-epoll + + + log4j + log4j + + + org.apache.commons + commons-lang3 + + + io.dropwizard.metrics + metrics-core + + + jakarta.ws.rs + jakarta.ws.rs-api + + + jakarta.annotation + jakarta.annotation-api + + + jakarta.validation + jakarta.validation-api + + + org.apache.hbase.thirdparty + hbase-shaded-jetty + + + org.apache.hadoop + hadoop-annotations + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-distcp + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-mapreduce-client-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.apache.hbase + hbase-protocol + + + org.apache.hbase.thirdparty + hbase-shaded-protobuf + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.glassfish.hk2.external + jakarta.inject + + + org.javassist + javassist + + + org.jruby.jcodings + jcodings + + + org.mortbay.jetty + servlet-api-2.5 + + + tomcat + jasper-compiler + + + tomcat + jasper-runtime + + + + + org.slf4j + log4j-over-slf4j + 1.7.32 + runtime + + + + io.trino + trino-spi + provided + + + io.airlift + slice + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + org.openjdk.jol + jol-core + provided + + + + + io.trino + trino-hive + test-jar + test + + + io.trino + trino-hive-hadoop2 + 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-services + test + + + io.trino + trino-tpch + test + + + io.trino.tpch + tpch + test + + + io.airlift + testing + test + + + org.assertj + assertj-core + test + + + org.jetbrains + annotations + test + + + org.testng + testng + test + + + diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java new file mode 100644 index 0000000000000..7b10e20d7fa29 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -0,0 +1,86 @@ +/* + * 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.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.units.DataSize; +import org.apache.hudi.common.model.HoodieFileFormat; + +import javax.validation.constraints.NotNull; + +import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; + +public class HudiConfig +{ + private HoodieFileFormat fileFormat = PARQUET; + private boolean metadataEnabled; + private boolean shouldSkipMetaStoreForPartition = true; + private DataSize maxSplitSize = DataSize.ofBytes(128 * 1024 * 1024); + + @NotNull + public HoodieFileFormat getFileFormat() + { + return HoodieFileFormat.valueOf(fileFormat.name()); + } + + @Config("hudi.file-format") + public HudiConfig setFileFormat(HoodieFileFormat fileFormat) + { + this.fileFormat = fileFormat; + return this; + } + + @Config("hudi.metadata-enabled") + @ConfigDescription("Fetch the list of file names and sizes from metadata rather than storage") + public HudiConfig setMetadataEnabled(boolean metadataEnabled) + { + this.metadataEnabled = metadataEnabled; + return this; + } + + @NotNull + public boolean isMetadataEnabled() + { + return this.metadataEnabled; + } + + @Config("hudi.max-split-size") + public HudiConfig setMaxSplitSize(DataSize size) + { + this.maxSplitSize = size; + return this; + } + + @NotNull + public DataSize getMaxSplitSize() + { + return this.maxSplitSize; + } + + @Config("hudi.skip-metastore-for-partition") + @ConfigDescription("Whether to skip metastore for partition") + public HudiConfig setSkipMetaStoreForPartition(boolean shouldSkipMetaStoreForPartition) + { + this.shouldSkipMetaStoreForPartition = shouldSkipMetaStoreForPartition; + return this; + } + + @NotNull + public boolean getSkipMetaStoreForPartition() + { + return this.shouldSkipMetaStoreForPartition; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java new file mode 100644 index 0000000000000..401bdf44619c7 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java @@ -0,0 +1,189 @@ +/* + * 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.ImmutableSet; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.log.Logger; +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.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorHandleResolver; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorNodePartitioningProvider; +import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorPageSourceProvider; +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.Optional; +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 static final Logger log = Logger.get(HudiConnector.class); + + private final LifeCycleManager lifeCycleManager; + private final HudiTransactionManager transactionManager; + private final HudiMetadataFactory metadataFactory; + private final ConnectorSplitManager splitManager; + private final ConnectorPageSourceProvider pageSourceProvider; + private final ConnectorPageSinkProvider pageSinkProvider; + private final ConnectorNodePartitioningProvider nodePartitioningProvider; + private final Set systemTables; + private final List> sessionProperties; + private final List> tableProperties; + private final Optional accessControl; + + public HudiConnector( + LifeCycleManager lifeCycleManager, + HudiTransactionManager transactionManager, + HudiMetadataFactory metadataFactory, + ConnectorSplitManager splitManager, + ConnectorPageSourceProvider pageSourceProvider, + ConnectorPageSinkProvider pageSinkProvider, + ConnectorNodePartitioningProvider nodePartitioningProvider, + Set systemTables, + Set sessionPropertiesProviders, + List> tableProperties, + Optional accessControl) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); + this.metadataFactory = requireNonNull(metadataFactory, "metadataFactory is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); + this.pageSinkProvider = requireNonNull(pageSinkProvider, "pageSinkProvider 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")); + this.accessControl = requireNonNull(accessControl, "accessControl is null"); + } + + @Override + public Optional getHandleResolver() + { + return Optional.of(new HudiHandleResolver()); + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) + { + ConnectorMetadata metadata = transactionManager.get(transactionHandle); + return new ClassLoaderSafeConnectorMetadata(metadata, getClass().getClassLoader()); + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return pageSourceProvider; + } + + @Override + public ConnectorPageSinkProvider getPageSinkProvider() + { + return pageSinkProvider; + } + + @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 ConnectorAccessControl getAccessControl() + { + return accessControl.orElseThrow(UnsupportedOperationException::new); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) + { + checkConnectorSupports(SERIALIZABLE, isolationLevel); + ConnectorTransactionHandle transaction = new HiveTransactionHandle(); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(getClass().getClassLoader())) { + transactionManager.put(transaction, metadataFactory.create()); + } + return transaction; + } + + @Override + public void commit(ConnectorTransactionHandle transaction) + { + transactionManager.remove(transaction); + } + + @Override + public void rollback(ConnectorTransactionHandle transaction) + { + HudiMetadata metadata = transactionManager.remove(transaction); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(getClass().getClassLoader())) { + metadata.rollback(); + } + } + + @Override + public final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java new file mode 100644 index 0000000000000..34d077f756a5e --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java @@ -0,0 +1,82 @@ +/* + * 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.Module; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorFactory; + +import java.lang.reflect.InvocationTargetException; +import java.util.Map; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.base.Throwables.throwIfUnchecked; +import static java.util.Objects.requireNonNull; + +public class HudiConnectorFactory + implements ConnectorFactory +{ + private final String name; + private final Class module; + + public HudiConnectorFactory(String name) + { + this(name, EmptyModule.class); + } + + public HudiConnectorFactory(String name, Class module) + { + checkArgument(!isNullOrEmpty(name), "name is null or empty"); + this.name = name; + this.module = requireNonNull(module, "module is null"); + } + + @Override + public String getName() + { + return name; + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + ClassLoader classLoader = context.duplicatePluginClassLoader(); + try { + Object moduleInstance = classLoader.loadClass(module.getName()).getConstructor().newInstance(); + Class moduleClass = classLoader.loadClass(Module.class.getName()); + return (Connector) classLoader.loadClass(InternalHudiConnectorFactory.class.getName()) + .getMethod("createConnector", String.class, Map.class, ConnectorContext.class, moduleClass) + .invoke(null, catalogName, config, context, moduleInstance); + } + catch (InvocationTargetException e) { + Throwable targetException = e.getTargetException(); + throwIfUnchecked(targetException); + throw new RuntimeException(targetException); + } + catch (ReflectiveOperationException e) { + throw new RuntimeException(e); + } + } + + public static class EmptyModule + implements Module + { + @Override + public void configure(Binder binder) {} + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java new file mode 100644 index 0000000000000..d38724451c226 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java @@ -0,0 +1,53 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.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; +import static io.trino.spi.ErrorType.INTERNAL_ERROR; +import static io.trino.spi.ErrorType.USER_ERROR; + +public enum HudiErrorCode + implements ErrorCodeSupplier +{ + HUDI_UNKNOWN_TABLE_TYPE(0, EXTERNAL), + HUDI_INVALID_METADATA(1, EXTERNAL), + HUDI_TOO_MANY_OPEN_PARTITIONS(2, USER_ERROR), + HUDI_INVALID_PARTITION_VALUE(3, EXTERNAL), + HUDI_BAD_DATA(4, EXTERNAL), + HUDI_MISSING_DATA(5, EXTERNAL), + HUDI_CANNOT_OPEN_SPLIT(6, EXTERNAL), + HUDI_WRITER_OPEN_ERROR(7, EXTERNAL), + HUDI_FILESYSTEM_ERROR(8, EXTERNAL), + HUDI_CURSOR_ERROR(9, EXTERNAL), + HUDI_WRITE_VALIDATION_FAILED(10, INTERNAL_ERROR), + HUDI_INVALID_SNAPSHOT_ID(11, USER_ERROR); + + private final ErrorCode errorCode; + + HudiErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0100_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHandleResolver.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHandleResolver.java new file mode 100644 index 0000000000000..fd72934f825ae --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHandleResolver.java @@ -0,0 +1,67 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HiveInsertTableHandle; +import io.trino.plugin.hive.HiveOutputTableHandle; +import io.trino.plugin.hive.HiveTransactionHandle; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorHandleResolver; +import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorOutputTableHandle; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; + +public class HudiHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableHandleClass() + { + return HudiTableHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return HiveColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return HudiSplit.class; + } + + @Override + public Class getOutputTableHandleClass() + { + return HiveOutputTableHandle.class; + } + + @Override + public Class getInsertTableHandleClass() + { + return HiveInsertTableHandle.class; + } + + @Override + public Class getTransactionHandleClass() + { + return HiveTransactionHandle.class; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiInputInfo.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiInputInfo.java new file mode 100644 index 0000000000000..36843d75540ff --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiInputInfo.java @@ -0,0 +1,49 @@ +/* + * 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 java.util.List; + +public class HudiInputInfo +{ + private final List partitionIds; + // Code that serialize HudiInputInfo into log would often need the ability to limit the length of log entries. + // This boolean field allows such code to mark the log entry as length limited. + private final boolean truncated; + + @JsonCreator + public HudiInputInfo( + @JsonProperty("partitionIds") List partitionIds, + @JsonProperty("truncated") boolean truncated) + { + this.partitionIds = partitionIds; + this.truncated = truncated; + } + + @JsonProperty + public List getPartitionIds() + { + return partitionIds; + } + + @JsonProperty + public boolean isTruncated() + { + return truncated; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java new file mode 100644 index 0000000000000..5b7819b28791d --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java @@ -0,0 +1,343 @@ +/* + * 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.Sets; +import io.airlift.log.Logger; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartition; +import io.trino.plugin.hive.acid.AcidSchema; +import io.trino.plugin.hive.authentication.HiveIdentity; +import io.trino.plugin.hive.metastore.Column; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.Table; +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.ConnectorTableProperties; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.ConstraintApplicationResult; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SchemaTablePrefix; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.TypeManager; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.Iterables.concat; +import static io.trino.plugin.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; +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.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static io.trino.plugin.hive.HiveMetadata.TABLE_COMMENT; +import static io.trino.plugin.hive.HiveTableProperties.EXTERNAL_LOCATION_PROPERTY; +import static io.trino.plugin.hive.HiveTableProperties.PARTITIONED_BY_PROPERTY; +import static io.trino.plugin.hive.HiveTimestampPrecision.NANOSECONDS; +import static io.trino.plugin.hive.util.HiveUtil.columnExtraInfo; +import static io.trino.plugin.hive.util.HiveUtil.hiveColumnHandles; +import static io.trino.plugin.hive.util.HiveUtil.isHiveSystemSchema; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_UNKNOWN_TABLE_TYPE; +import static io.trino.plugin.hudi.HudiUtil.splitPredicate; +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.hadoop.hive.metastore.TableType.EXTERNAL_TABLE; +import static org.apache.hadoop.hive.ql.io.AcidUtils.isFullAcidTable; +import static org.apache.hudi.common.fs.FSUtils.getFs; +import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; +import static org.apache.hudi.exception.TableNotFoundException.checkTableValidity; + +public class HudiMetadata + implements ConnectorMetadata +{ + private static final Logger LOG = Logger.get(HudiMetadata.class); + private final HiveMetastore metastore; + private final HdfsEnvironment hdfsEnvironment; + private final TypeManager typeManager; + + public HudiMetadata(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, TypeManager typeManager) + { + this.metastore = requireNonNull(metastore, "metastore is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return metastore.getAllDatabases(); + } + + @Override + public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) + { + requireNonNull(tableName, "tableName is null"); + if (isHiveSystemSchema(tableName.getSchemaName())) { + return null; + } + Optional table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()); + if (table.isEmpty()) { + return null; + } + if (!isHudiTable(session, table.get())) { + throw new TrinoException(HUDI_UNKNOWN_TABLE_TYPE, format("Not a Hudi table: %s", tableName)); + } + return new HudiTableHandle( + tableName.getSchemaName(), + tableName.getTableName(), + table.get().getStorage().getLocation(), + HoodieTableType.COPY_ON_WRITE, + TupleDomain.all(), + Optional.of(getTableMetaClient(session, table.get()))); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + { + HudiTableHandle hudiTableHandle = (HudiTableHandle) table; + return getTableMetadata(session, hudiTableHandle.getSchemaTableName()); + } + + @Override + public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle tableHandle, Constraint constraint) + { + HudiTableHandle handle = (HudiTableHandle) tableHandle; + if (constraint.getSummary().equals(handle.getPredicate())) { + return Optional.empty(); + } + + List> predicate = splitPredicate(constraint.getSummary()); + TupleDomain unenforcedPredicate = predicate.get(1); + HudiTableHandle newHudiTableHandle = handle.withPredicate(constraint.getSummary().transformKeys(HiveColumnHandle.class::cast)); + + return Optional.of(new ConstraintApplicationResult<>( + newHudiTableHandle, + unenforcedPredicate, + false)); + } + + private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName tableName) + { + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + Function metadataGetter = columnMetadataGetter(table); + ImmutableList.Builder columns = ImmutableList.builder(); + for (HiveColumnHandle columnHandle : hiveColumnHandles(table, typeManager, NANOSECONDS)) { + columns.add(metadataGetter.apply(columnHandle)); + } + + // External location property + ImmutableMap.Builder properties = ImmutableMap.builder(); + if (table.getTableType().equals(EXTERNAL_TABLE.name())) { + properties.put(EXTERNAL_LOCATION_PROPERTY, table.getStorage().getLocation()); + } + + // 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.build(), properties.build(), comment); + } + + @Override + public ConnectorTableProperties getTableProperties(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return new ConnectorTableProperties(); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + SchemaTableName tableName = ((HudiTableHandle) tableHandle).getSchemaTableName(); + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + 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 table) + { + return ((HudiTableHandle) table).getPartitions() + .map(partitions -> new HudiInputInfo( + partitions.stream() + .map(HivePartition::getPartitionId) + .collect(toImmutableList()), + false)); + } + + @Override + public List listTables(ConnectorSession session, Optional optionalSchemaName) + { + ImmutableList.Builder tableNames = ImmutableList.builder(); + for (String schemaName : listSchemas(session, optionalSchemaName)) { + for (String tableName : metastore.getAllTables(schemaName)) { + tableNames.add(new SchemaTableName(schemaName, tableName)); + } + } + + tableNames.addAll(listMaterializedViews(session, optionalSchemaName)); + return tableNames.build(); + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + List tables = prefix.getTable() + .map(ignored -> singletonList(prefix.toSchemaTableName())) + .orElseGet(() -> listTables(session, prefix.getSchema())); + + ImmutableMap.Builder> columns = ImmutableMap.builder(); + for (SchemaTableName table : tables) { + try { + columns.put(table, getTableMetadata(session, table).getColumns()); + } + catch (TableNotFoundException e) { + // table disappeared during listing operation + } + } + return columns.build(); + } + + private List listSchemas(ConnectorSession session, Optional schemaName) + { + if (schemaName.isPresent()) { + if (isHiveSystemSchema(schemaName.get())) { + return ImmutableList.of(); + } + return ImmutableList.of(schemaName.get()); + } + return listSchemaNames(session); + } + + @Override + public boolean usesLegacyTableLayouts() + { + return false; + } + + public HiveMetastore getMetastore() + { + return metastore; + } + + public HoodieTableMetaClient getTableMetaClient(ConnectorSession session, Table table) + { + String basePath = table.getStorage().getLocation(); + Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(basePath)); + return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath) + //.setBootstrapIndexClassName(HudiHFileBootstrapIndex.class.getName()) + .build(); + } + + public void rollback() + { + // TODO: cleanup open transaction + } + + private boolean isHudiTable(ConnectorSession session, Table table) + { + String basePath = table.getStorage().getLocation(); + Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(basePath)); + try { + checkTableValidity(getFs(basePath, conf), new Path(basePath), new Path(basePath, METAFOLDER_NAME)); + } + catch (Exception e) { + return false; + } + return true; + } + + private static Function columnMetadataGetter(Table table) + { + ImmutableList.Builder columnNames = ImmutableList.builder(); + table.getPartitionColumns().stream().map(Column::getName).forEach(columnNames::add); + table.getDataColumns().stream().map(Column::getName).forEach(columnNames::add); + List allColumnNames = columnNames.build(); + if (allColumnNames.size() > Sets.newHashSet(allColumnNames).size()) { + throw new TrinoException(HIVE_INVALID_METADATA, + format("Hive metadata for table %s is invalid: Table descriptor contains duplicate columns", table.getTableName())); + } + + List tableColumns = table.getDataColumns(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); + for (Column field : concat(tableColumns, table.getPartitionColumns())) { + if (field.getComment().isPresent() && !field.getComment().get().equals("from deserializer")) { + builder.put(field.getName(), field.getComment()); + } + else { + builder.put(field.getName(), Optional.empty()); + } + } + + // add hidden columns + builder.put(PATH_COLUMN_NAME, Optional.empty()); + if (table.getStorage().getBucketProperty().isPresent()) { + builder.put(BUCKET_COLUMN_NAME, Optional.empty()); + } + builder.put(FILE_SIZE_COLUMN_NAME, Optional.empty()); + builder.put(FILE_MODIFIED_TIME_COLUMN_NAME, Optional.empty()); + if (!table.getPartitionColumns().isEmpty()) { + builder.put(PARTITION_COLUMN_NAME, Optional.empty()); + } + + if (isFullAcidTable(table.getParameters())) { + for (String name : AcidSchema.ACID_COLUMN_NAMES) { + builder.put(name, Optional.empty()); + } + } + + Map> columnComment = builder.build(); + + return handle -> ColumnMetadata.builder() + .setName(handle.getName()) + .setType(handle.getType()) + .setComment(columnComment.get(handle.getName())) + .setExtraInfo(Optional.ofNullable(columnExtraInfo(handle.isPartitionKey()))) + .setHidden(handle.isHidden()) + .build(); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java new file mode 100644 index 0000000000000..c698ca7f26f6c --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.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 io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.spi.type.TypeManager; + +import javax.inject.Inject; + +import static java.util.Objects.requireNonNull; + +public class HudiMetadataFactory +{ + private final HiveMetastore metastore; + private final HdfsEnvironment hdfsEnvironment; + private final TypeManager typeManager; + + @Inject + public HudiMetadataFactory(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, TypeManager typeManager) + { + this.metastore = requireNonNull(metastore, "metastore is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + public HudiMetadata create() + { + return new HudiMetadata(metastore, hdfsEnvironment, typeManager); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetastoreModule.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetastoreModule.java new file mode 100644 index 0000000000000..e9c2a37cb3695 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetastoreModule.java @@ -0,0 +1,79 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.MetastoreTypeConfig; +import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; +import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreModule; +import io.trino.plugin.hive.metastore.cache.ForCachingHiveMetastore; +import io.trino.plugin.hive.metastore.file.FileMetastoreModule; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreModule; + +import javax.inject.Inject; + +import java.util.Optional; + +import static io.airlift.configuration.ConditionalModule.conditionalModule; +import static java.util.Objects.requireNonNull; + +public class HudiMetastoreModule + extends AbstractConfigurationAwareModule +{ + private final Optional metastore; + + public HudiMetastoreModule(Optional metastore) + { + this.metastore = requireNonNull(metastore, "metastore is null"); + } + + @Override + protected void setup(Binder binder) + { + if (metastore.isPresent()) { + binder.bind(HiveMetastore.class).annotatedWith(ForCachingHiveMetastore.class).toInstance(metastore.get()); + install(new CachingHiveMetastoreModule()); + } + else { + bindMetastoreModule("thrift", new ThriftMetastoreModule()); + bindMetastoreModule("file", new FileMetastoreModule()); + // TODO add support for Glue metastore + } + + binder.bind(MetastoreValidator.class).asEagerSingleton(); + } + + public static class MetastoreValidator + { + @Inject + public MetastoreValidator(HiveMetastore metastore) + { + if (metastore instanceof CachingHiveMetastore) { + throw new RuntimeException("Hive metastore caching must not be enabled for Hudi"); + } + } + } + + private void bindMetastoreModule(String name, Module module) + { + install(conditionalModule( + MetastoreTypeConfig.class, + metastore -> name.equalsIgnoreCase(metastore.getMetastoreType()), + module)); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java new file mode 100644 index 0000000000000..7997a7874702d --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java @@ -0,0 +1,79 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.trino.plugin.base.security.AllowAllAccessControl; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.CachingDirectoryLister; +import io.trino.plugin.hive.DirectoryLister; +import io.trino.plugin.hive.FileFormatDataSourceStats; +import io.trino.plugin.hive.HdfsConfiguration; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HiveConfig; +import io.trino.plugin.hive.HiveHdfsConfiguration; +import io.trino.plugin.hive.HiveNodePartitioningProvider; +import io.trino.plugin.hive.HiveTransactionManager; +import io.trino.plugin.hive.metastore.MetastoreConfig; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.plugin.hive.parquet.ParquetWriterConfig; +import io.trino.spi.connector.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorNodePartitioningProvider; +import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSplitManager; + +import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class HudiModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(HdfsConfiguration.class).to(HiveHdfsConfiguration.class).in(Scopes.SINGLETON); + binder.bind(HdfsEnvironment.class).in(Scopes.SINGLETON); + binder.bind(DirectoryLister.class).to(CachingDirectoryLister.class).in(Scopes.SINGLETON); + binder.bind(HudiTransactionManager.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(HiveConfig.class); + configBinder(binder).bindConfig(HudiConfig.class); + configBinder(binder).bindConfig(MetastoreConfig.class); + + 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(ConnectorPageSinkProvider.class).to(HudiPageSinkProvider.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(HiveTransactionManager.class).in(Scopes.SINGLETON); + + binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(FileFormatDataSourceStats.class).withGeneratedName(); + + newOptionalBinder(binder, ConnectorAccessControl.class).setDefault().to(AllowAllAccessControl.class).in(Scopes.SINGLETON); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSinkProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSinkProvider.java new file mode 100644 index 0000000000000..1c54e4deed852 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSinkProvider.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 com.google.common.collect.ImmutableMap; +import io.trino.plugin.hive.HiveInsertTableHandle; +import io.trino.plugin.hive.HiveOutputTableHandle; +import io.trino.plugin.hive.HiveWritableTableHandle; +import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorOutputTableHandle; +import io.trino.spi.connector.ConnectorPageSink; +import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTransactionHandle; + +import java.util.Map; + +public class HudiPageSinkProvider + implements ConnectorPageSinkProvider +{ + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorOutputTableHandle outputTableHandle) + { + HiveOutputTableHandle handle = (HiveOutputTableHandle) outputTableHandle; + return createPageSink(handle, true, session, handle.getAdditionalTableParameters()); + } + + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorInsertTableHandle insertTableHandle) + { + HiveWritableTableHandle handle = (HiveInsertTableHandle) insertTableHandle; + return createPageSink(handle, false, session, ImmutableMap.of() /* for insert properties are taken from metastore */); + } + + private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean isCreateTable, ConnectorSession session, Map additionalTableParameters) + { + return null; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSource.java new file mode 100644 index 0000000000000..e6ed5096baa3e --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSource.java @@ -0,0 +1,127 @@ +/* + * 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.Page; +import io.trino.spi.TrinoException; +import io.trino.spi.block.Block; +import io.trino.spi.block.RunLengthEncodedBlock; +import io.trino.spi.connector.ConnectorPageSource; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; +import static java.util.Objects.requireNonNull; + +public class HudiPageSource + implements ConnectorPageSource +{ + private final List columnHandles; + private final ConnectorPageSource pageSource; + private final Map partitionBlocks; + + public HudiPageSource( + List columnHandles, + Map partitionBlocks, + ConnectorPageSource pageSource) + { + this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); + this.pageSource = requireNonNull(pageSource, "pageSource is null"); + this.partitionBlocks = requireNonNull(partitionBlocks, "partitionBlocks is null"); + } + + @Override + public long getCompletedBytes() + { + return pageSource.getCompletedBytes(); + } + + @Override + public long getReadTimeNanos() + { + return pageSource.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return pageSource.isFinished(); + } + + @Override + public Page getNextPage() + { + try { + Page page = pageSource.getNextPage(); + if (page == null) { + return null; + } + int positionCount = page.getPositionCount(); + + int dataColumnIndex = 0; + int columnIndex = 0; + Block[] blocksWithPartitionColumns = new Block[columnHandles.size()]; + for (HiveColumnHandle columnHandle : columnHandles) { + if (columnHandle.isPartitionKey()) { + Block partitionValue = partitionBlocks.get(columnHandle.getName()); + blocksWithPartitionColumns[columnIndex++] = new RunLengthEncodedBlock(partitionValue, positionCount); + } + else { + blocksWithPartitionColumns[columnIndex++] = (page.getBlock(dataColumnIndex)); + dataColumnIndex++; + } + } + return new Page(positionCount, blocksWithPartitionColumns); + } + catch (TrinoException e) { + closeWithSuppression(e); + throw e; + } + catch (RuntimeException e) { + closeWithSuppression(e); + throw new TrinoException(HUDI_BAD_DATA, e); + } + } + + @Override + public long getSystemMemoryUsage() + { + return pageSource.getSystemMemoryUsage(); + } + + @Override + public void close() + throws IOException + { + pageSource.close(); + } + + private void closeWithSuppression(Throwable throwable) + { + requireNonNull(throwable, "throwable is null"); + try { + close(); + } + catch (Exception e) { + // Self-suppression not permitted + if (e != throwable) { + throwable.addSuppressed(e); + } + } + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java new file mode 100644 index 0000000000000..57cf5b2eff2a4 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -0,0 +1,315 @@ +/* + * 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.parquet.Field; +import io.trino.parquet.ParquetCorruptionException; +import io.trino.parquet.ParquetDataSource; +import io.trino.parquet.ParquetDataSourceId; +import io.trino.parquet.ParquetReaderOptions; +import io.trino.parquet.RichColumnDescriptor; +import io.trino.parquet.predicate.Predicate; +import io.trino.parquet.reader.ParquetReader; +import io.trino.plugin.hive.FileFormatDataSourceStats; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.ReaderColumns; +import io.trino.plugin.hive.parquet.HdfsParquetDataSource; +import io.trino.plugin.hive.parquet.ParquetPageSource; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.spi.TrinoException; +import io.trino.spi.block.Block; +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.predicate.TupleDomain; +import io.trino.spi.predicate.Utils; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.spi.type.Type; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.BlockMissingException; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; +import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.schema.MessageType; +import org.joda.time.DateTimeZone; + +import javax.inject.Inject; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.TimeZone; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkArgument; +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.ParquetTypeUtils.lookupColumnByName; +import static io.trino.parquet.predicate.PredicateUtils.buildPredicate; +import static io.trino.parquet.predicate.PredicateUtils.predicateMatches; +import static io.trino.parquet.reader.MetadataReader.readFooter; +import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; +import static io.trino.plugin.hive.HivePageSourceProvider.projectBaseColumns; +import static io.trino.plugin.hive.HivePageSourceProvider.projectSufficientColumns; +import static io.trino.plugin.hive.parquet.HiveParquetColumnIOConverter.constructField; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.PARQUET_ROW_INDEX_COLUMN; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getColumnIndexStore; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getColumnType; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getParquetTupleDomain; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getParquetType; +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_MISSING_DATA; +import static io.trino.plugin.hudi.HudiUtil.convertPartitionValue; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toList; +import static java.util.stream.Collectors.toMap; +import static java.util.stream.Collectors.toUnmodifiableList; + +public class HudiPageSourceProvider + implements ConnectorPageSourceProvider +{ + private final HdfsEnvironment hdfsEnvironment; + private final FileFormatDataSourceStats fileFormatDataSourceStats; + private final ParquetReaderOptions parquetReaderOptions; + private final DateTimeZone timeZone; + + @Inject + public HudiPageSourceProvider( + HdfsEnvironment hdfsEnvironment, + FileFormatDataSourceStats fileFormatDataSourceStats, + ParquetReaderConfig parquetReaderConfig, + HudiConfig hudiConfig) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.fileFormatDataSourceStats = requireNonNull(fileFormatDataSourceStats, "fileFormatDataSourceStats is null"); + this.parquetReaderOptions = requireNonNull(parquetReaderConfig, "parquetReaderConfig is null").toParquetReaderOptions(); + requireNonNull(hudiConfig, "hudiConfig is null"); + this.timeZone = DateTimeZone.forID(TimeZone.getDefault().getID()); + } + + @Override + public ConnectorPageSource createPageSource( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit connectorSplit, + ConnectorTableHandle connectorTable, + List columns, + DynamicFilter dynamicFilter) + { + HudiSplit split = (HudiSplit) connectorSplit; + Path path = new Path(split.getPath()); + long start = split.getStart(); + long length = split.getLength(); + long estimatedFileSize = split.getFileSize(); + List hiveColumns = columns.stream() + .map(HiveColumnHandle.class::cast) + .collect(toList()); + // just send regular columns to create parquet page source + // for partition columns, separate blocks will be created + List regularColumns = hiveColumns.stream() + .filter(columnHandle -> !columnHandle.isPartitionKey()) + .collect(Collectors.toList()); + Configuration configuration = hdfsEnvironment.getConfiguration(new HdfsContext(session), path); + ConnectorPageSource dataPageSource = createParquetPageSource( + hdfsEnvironment, + configuration, + path, + start, + length, + estimatedFileSize, + regularColumns, + true, + split.getPredicate(), + fileFormatDataSourceStats, + timeZone, + parquetReaderOptions, + session.getIdentity(), + split.getPartitionKeys()); + + return new HudiPageSource( + hiveColumns, + convertPartitionValues(hiveColumns, split.getPartitionKeys()), // create blocks for partition values + dataPageSource); + } + + private static ConnectorPageSource createParquetPageSource( + HdfsEnvironment hdfsEnvironment, + Configuration configuration, + Path path, + long start, + long length, + long estimatedFileSize, + List columns, + boolean useParquetColumnNames, + TupleDomain effectivePredicate, + FileFormatDataSourceStats stats, + DateTimeZone timeZone, + ParquetReaderOptions options, + ConnectorIdentity identity, + List partitionKeys) + { + ParquetDataSource dataSource = null; + try { + FileSystem fileSystem = hdfsEnvironment.getFileSystem(identity, path, configuration); + FSDataInputStream inputStream = hdfsEnvironment.doAs(identity, () -> fileSystem.open(path)); + dataSource = new HdfsParquetDataSource(new ParquetDataSourceId(path.toString()), estimatedFileSize, inputStream, stats, options); + ParquetDataSource parquetDataSource = dataSource; + ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(identity, () -> readFooter(parquetDataSource)); + FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); + MessageType fileSchema = fileMetaData.getSchema(); + + Optional message = projectSufficientColumns(columns) + .map(projection -> projection.get().stream() + .map(HiveColumnHandle.class::cast) + .collect(toUnmodifiableList())) + .orElse(columns).stream() + .filter(column -> column.getColumnType() == REGULAR) + .map(column -> getColumnType(column, fileSchema, useParquetColumnNames)) + .filter(Optional::isPresent) + .map(Optional::get) + .map(type -> new MessageType(fileSchema.getName(), type)) + .reduce(MessageType::union); + + MessageType requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + MessageColumnIO messageColumn = getColumnIO(fileSchema, requestedSchema); + + Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); + TupleDomain parquetTupleDomain = options.isIgnoreStatistics() + ? TupleDomain.all() + : getParquetTupleDomain(descriptorsByPath, effectivePredicate, fileSchema, useParquetColumnNames); + + Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); + + long nextStart = 0; + ImmutableList.Builder blocks = ImmutableList.builder(); + ImmutableList.Builder blockStarts = ImmutableList.builder(); + ImmutableList.Builder> columnIndexes = ImmutableList.builder(); + for (BlockMetaData block : parquetMetadata.getBlocks()) { + long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset(); + Optional columnIndex = getColumnIndexStore(dataSource, block, descriptorsByPath, parquetTupleDomain, options); + if (start <= firstDataPage && firstDataPage < start + length + && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, columnIndex)) { + blocks.add(block); + blockStarts.add(nextStart); + columnIndexes.add(columnIndex); + } + nextStart += block.getRowCount(); + } + + ParquetReader parquetReader = new ParquetReader( + Optional.ofNullable(fileMetaData.getCreatedBy()), + messageColumn, + blocks.build(), + Optional.of(blockStarts.build()), + dataSource, + timeZone, + newSimpleAggregatedMemoryContext(), + options, + parquetPredicate, + columnIndexes.build()); + Optional readerProjections = projectBaseColumns(columns); + List baseColumns = readerProjections.map(projection -> + projection.get().stream() + .map(HiveColumnHandle.class::cast) + .collect(toUnmodifiableList())) + .orElse(columns); + + for (HiveColumnHandle column : baseColumns) { + checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); + } + + ImmutableList.Builder trinoTypes = ImmutableList.builder(); + ImmutableList.Builder> internalFields = ImmutableList.builder(); + ImmutableList.Builder rowIndexColumns = ImmutableList.builder(); + for (HiveColumnHandle column : baseColumns) { + trinoTypes.add(column.getBaseType()); + rowIndexColumns.add(column == PARQUET_ROW_INDEX_COLUMN); + if (column == PARQUET_ROW_INDEX_COLUMN) { + internalFields.add(Optional.empty()); + } + else { + internalFields.add(Optional.ofNullable(getParquetType(column, fileSchema, useParquetColumnNames)) + .flatMap(field -> { + String columnName = useParquetColumnNames ? column.getBaseColumnName() : fileSchema.getFields().get(column.getBaseHiveColumnIndex()).getName(); + return constructField(column.getBaseType(), lookupColumnByName(messageColumn, columnName)); + })); + } + } + + return new ParquetPageSource( + parquetReader, + trinoTypes.build(), + rowIndexColumns.build(), + internalFields.build()); + } + catch (IOException | RuntimeException e) { + try { + if (dataSource != null) { + dataSource.close(); + } + } + catch (IOException ignored) { + } + if (e instanceof TrinoException) { + throw (TrinoException) e; + } + String message = format("Error opening Hudi split %s (offset=%s, length=%s): %s", path, start, length, e.getMessage()); + + if (e instanceof ParquetCorruptionException) { + throw new TrinoException(HUDI_BAD_DATA, message, e); + } + + if (e instanceof BlockMissingException) { + throw new TrinoException(HUDI_MISSING_DATA, message, e); + } + throw new TrinoException(HUDI_CANNOT_OPEN_SPLIT, message, e); + } + } + + private Map convertPartitionValues( + List allColumns, + List partitionKeys) + { + return allColumns.stream() + .filter(HiveColumnHandle::isPartitionKey) + .collect(toMap( + HiveColumnHandle::getName, + columnHandle -> Utils.nativeValueToBlock( + columnHandle.getType(), + convertPartitionValue( + columnHandle.getName(), + partitionKeys.get(0).getValue(), + columnHandle.getType().getTypeSignature()).orElse(null)))); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPlugin.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPlugin.java new file mode 100644 index 0000000000000..565976f638914 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPlugin.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.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("hudi")); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java new file mode 100644 index 0000000000000..fb77e3a279847 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java @@ -0,0 +1,94 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.airlift.units.DataSize; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.session.PropertyMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; + +import javax.inject.Inject; + +import java.util.List; + +import static io.trino.plugin.base.session.PropertyMetadataUtil.dataSizeProperty; +import static io.trino.spi.session.PropertyMetadata.booleanProperty; +import static io.trino.spi.session.PropertyMetadata.enumProperty; + +public class HudiSessionProperties + implements SessionPropertiesProvider +{ + private static final String FILE_FORMAT = "file_format"; + private static final String METADATA_ENABLED = "metadata_enabled"; + private static final String MAX_SPLIT_SIZE = "max_split_size"; + private static final String SKIP_METASTORE_FOR_PARTITION = "skip_metastore_for_partition"; + + private final List> sessionProperties; + + @Inject + public HudiSessionProperties(HudiConfig hudiConfig) + { + sessionProperties = ImmutableList.of( + enumProperty( + FILE_FORMAT, + "Currently, only Parquet is supported", + HoodieFileFormat.class, + hudiConfig.getFileFormat(), + false), + booleanProperty( + METADATA_ENABLED, + "For Hudi tables prefer to fetch the list of files from its metadata", + hudiConfig.isMetadataEnabled(), + false), + booleanProperty( + SKIP_METASTORE_FOR_PARTITION, + "Whether to skip metastore for partition.", + hudiConfig.getSkipMetaStoreForPartition(), + false), + dataSizeProperty( + MAX_SPLIT_SIZE, + "Max split size", + hudiConfig.getMaxSplitSize(), + true)); + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } + + public static HoodieFileFormat getFileFormat(ConnectorSession session) + { + return session.getProperty(FILE_FORMAT, HoodieFileFormat.class); + } + + public static boolean isHudiMetadataEnabled(ConnectorSession session) + { + return session.getProperty(METADATA_ENABLED, Boolean.class); + } + + public static DataSize getMaxSplitSize(ConnectorSession session) + { + return session.getProperty(MAX_SPLIT_SIZE, DataSize.class); + } + + public static boolean shouldSkipMetaStoreForPartition(ConnectorSession session) + { + return session.getProperty(SKIP_METASTORE_FOR_PARTITION, Boolean.class); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java new file mode 100644 index 0000000000000..3a978f2fcd32e --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.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 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.spi.HostAddress; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.predicate.TupleDomain; + +import java.util.List; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class HudiSplit + implements ConnectorSplit +{ + private final String path; + private final long start; + private final long length; + private final long fileSize; + private final List addresses; + private final TupleDomain predicate; + private final List partitionKeys; + + @JsonCreator + public HudiSplit( + @JsonProperty("path") String path, + @JsonProperty("start") long start, + @JsonProperty("length") long length, + @JsonProperty("fileSize") long fileSize, + @JsonProperty("addresses") List addresses, + @JsonProperty("predicate") TupleDomain predicate, + @JsonProperty("partitionKeys") List partitionKeys) + { + checkArgument(start >= 0, "start must be positive"); + checkArgument(length >= 0, "length must be positive"); + checkArgument(fileSize >= 0, "fileSize must be positive"); + + this.path = requireNonNull(path, "path is null"); + this.start = start; + this.length = length; + this.fileSize = fileSize; + this.addresses = ImmutableList.copyOf(requireNonNull(addresses, "addresses is null")); + this.predicate = requireNonNull(predicate, "predicate is null"); + this.partitionKeys = ImmutableList.copyOf(requireNonNull(partitionKeys, "partitionKeys is null")); + } + + @Override + public boolean isRemotelyAccessible() + { + return true; + } + + @JsonProperty + @Override + public List getAddresses() + { + return addresses; + } + + @Override + public Object getInfo() + { + return ImmutableMap.builder() + .put("path", path) + .put("start", start) + .put("length", length) + .put("fileSize", fileSize) + .build(); + } + + @JsonProperty + public String getPath() + { + return path; + } + + @JsonProperty + public long getStart() + { + return start; + } + + @JsonProperty + public long getLength() + { + return length; + } + + @JsonProperty + public long getFileSize() + { + return fileSize; + } + + @JsonProperty + public TupleDomain getPredicate() + { + return predicate; + } + + @JsonProperty + public List getPartitionKeys() + { + return partitionKeys; + } + + @Override + public String toString() + { + return toStringHelper(this) + .addValue(path) + .addValue(start) + .addValue(length) + .addValue(fileSize) + .toString(); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java new file mode 100644 index 0000000000000..b062c81c14c96 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -0,0 +1,102 @@ +/* + * 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.base.classloader.ClassLoaderSafeConnectorSplitSource; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.authentication.HiveIdentity; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.Table; +import io.trino.spi.connector.ColumnHandle; +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.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.table.HoodieTableMetaClient; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.shouldSkipMetaStoreForPartition; +import static io.trino.plugin.hudi.HudiUtil.getMetaClient; +import static java.util.Objects.requireNonNull; + +public class HudiSplitManager + implements ConnectorSplitManager +{ + public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode"); + + private final HudiTransactionManager transactionManager; + private final HdfsEnvironment hdfsEnvironment; + + @Inject + public HudiSplitManager(HudiTransactionManager transactionManager, HdfsEnvironment hdfsEnvironment) + { + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle tableHandle, + SplitSchedulingStrategy splitSchedulingStrategy, + DynamicFilter dynamicFilter, + Constraint constraint) + { + HiveIdentity identity = new HiveIdentity(session); + HudiTableHandle hudiTable = (HudiTableHandle) tableHandle; + SchemaTableName tableName = hudiTable.getSchemaTableName(); + HudiMetadata hudiMetadata = transactionManager.get(transaction); + HiveMetastore metastore = hudiMetadata.getMetastore(); + List partitionColumnHandles = hudiMetadata.getColumnHandles(session, tableHandle) + .values().stream().map(HiveColumnHandle.class::cast) + .filter(HiveColumnHandle::isPartitionKey).collect(Collectors.toList()); + Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + HdfsEnvironment.HdfsContext context = new HdfsEnvironment.HdfsContext(session); + String tablePath = table.getStorage().getLocation(); + Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(tablePath)); + // TODO: Do we need below? + Map valByRegex = conf.getValByRegex(HOODIE_CONSUME_MODE_PATTERN_STRING.pattern()); + HoodieTableMetaClient metaClient = hudiTable.getMetaClient().orElseGet(() -> getMetaClient(conf, hudiTable.getBasePath())); + TupleDomain effectivePredicate = constraint.getSummary(); + HudiSplitSource splitSource = new HudiSplitSource( + identity, + metastore, + hudiTable, + conf, + partitionColumnHandles, + isHudiMetadataEnabled(session), + shouldSkipMetaStoreForPartition(session)); + + return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java new file mode 100644 index 0000000000000..df59a4ec6dea1 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -0,0 +1,340 @@ +/* + * 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.Iterators; +import io.airlift.log.Logger; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartition; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.authentication.HiveIdentity; +import io.trino.plugin.hive.metastore.Column; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.MetastoreUtil; +import io.trino.plugin.hive.metastore.Partition; +import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hive.util.HiveUtil; +import io.trino.spi.connector.ConnectorPartitionHandle; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.Type; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +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.HoodieBaseFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; +import static io.trino.plugin.hudi.HudiUtil.buildPartitionKeys; +import static io.trino.plugin.hudi.HudiUtil.buildPartitionValues; +import static io.trino.plugin.hudi.HudiUtil.getMetaClient; +import static io.trino.plugin.hudi.HudiUtil.getSplits; +import static java.util.Objects.isNull; +import static java.util.Objects.nonNull; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; +import static java.util.stream.Collectors.toList; +import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; + +public class HudiSplitSource + implements ConnectorSplitSource +{ + private static final Logger LOG = Logger.get(HudiSplitSource.class); + private final HiveIdentity identity; + private final HiveMetastore metastore; + private final SchemaTableName tableName; + private final Configuration conf; + private final HudiTableHandle tableHandle; + private final Table table; + private final FileSystem fileSystem; + private final HoodieTableMetaClient metaClient; + private final Map> partitionMap; + private final boolean metadataEnabled; + private final boolean shouldSkipMetastoreForPartition; + private final Map baseFileToPartitionMap; + private final List partitionColumnHandles; + private final ArrayDeque baseFiles = new ArrayDeque<>(); + private final int partitionBatchNum = 32; + + private HoodieTableFileSystemView fileSystemView; + private List partitionColumns; + private Iterator> metastorePartitions; + private Iterator tableHandlePartitions; + + public HudiSplitSource( + HiveIdentity identity, + HiveMetastore metastore, + HudiTableHandle tableHandle, + Configuration conf, + List partitionColumnHandles, + boolean metadataEnabled, + boolean shouldSkipMetastoreForPartition) + { + this.identity = identity; + this.metastore = metastore; + this.tableName = tableHandle.getSchemaTableName(); + this.table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + this.conf = requireNonNull(conf, "conf is null"); + this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); + this.partitionMap = new HashMap<>(); + this.partitionColumnHandles = partitionColumnHandles; + this.metadataEnabled = metadataEnabled; + this.shouldSkipMetastoreForPartition = shouldSkipMetastoreForPartition; + this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); + this.fileSystem = metaClient.getFs(); + this.baseFileToPartitionMap = new HashMap<>(); + } + + @Override + public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + { + try { + List connectorSplits = getSplitsForSnapshotMode(maxSize); + return completedFuture(new ConnectorSplitBatch(connectorSplits, isFinished())); + } + catch (IOException e) { + throw new HoodieIOException("Failed to get next batch of splits", e); + } + } + + @Override + public void close() + { + if (nonNull(fileSystem)) { + fileSystemView.close(); + } + } + + @Override + public boolean isFinished() + { + if (shouldSkipMetastoreForPartition) { + return !tableHandlePartitions.hasNext() && baseFiles.isEmpty(); + } + return !metastorePartitions.hasNext() && baseFiles.isEmpty(); + } + + private List getSplitsForSnapshotMode(int maxSize) + throws IOException + { + if (isNull(this.fileSystemView)) { + // First time calling this + // Load the timeline and file status only once + HoodieEngineContext engineContext = new HoodieLocalEngineContext(conf); + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(metadataEnabled) + .build(); + // Scan the file system to load the instants from timeline + this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); + partitionColumns = table.getPartitionColumns(); + + if (!partitionColumns.isEmpty()) { + HoodieTimer timer = new HoodieTimer().startTimer(); + + Optional> partitions = tableHandle.getPartitions(); + if (partitions.isEmpty() || partitions.get().isEmpty()) { + throw new HoodieIOException("Cannot find partitions in the table"); + } + List partitionTypes = partitionColumnHandles.stream() + .map(HiveColumnHandle::getType) + .collect(toList()); + + LOG.warn("partitionColumnHandles: " + partitionColumnHandles); + + TupleDomain partitionKeyFilters = MetastoreUtil.computePartitionKeyFilter( + partitionColumnHandles, tableHandle.getPredicate()); + LOG.warn("shouldSkipMetastoreForPartition: " + shouldSkipMetastoreForPartition); + LOG.warn("Table handle predicate : " + tableHandle.getPredicate().toString()); + + if (shouldSkipMetastoreForPartition) { + List tablePartitions = tableHandle.getPartitions().orElseGet(ImmutableList::of).stream() + .map(HivePartition::getPartitionId) + .map(partitionName -> HudiUtil.parseValuesAndFilterPartition( + tableName, partitionName, partitionColumnHandles, partitionTypes, tableHandle.getPredicate())) + .filter(Optional::isPresent) + .map(Optional::get) + .collect(toList()); + LOG.warn("full partition names after filtering: " + tablePartitions); + tableHandlePartitions = tablePartitions.iterator(); + } + else { + List fullPartitionNames = metastore.getPartitionNamesByFilter( + identity, + tableName.getSchemaName(), + tableName.getTableName(), + partitionColumns.stream() + .map(Column::getName) + .collect(Collectors.toList()), + partitionKeyFilters) + .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())) + .stream() + // Apply extra filters which may not be done by getPartitionNamesByFilter + .map(partitionName -> HudiUtil.parseValuesAndFilterPartition( + tableName, partitionName, partitionColumnHandles, partitionTypes, tableHandle.getPredicate())) + .filter(Optional::isPresent) + .map(Optional::get) + .collect(toList()); + LOG.warn("full partition names after filtering: " + fullPartitionNames); + List> partitionNameElements = fullPartitionNames + .stream() + .map(HiveUtil::toPartitionValues) + .collect(toImmutableList()); + metastorePartitions = partitionNameElements.iterator(); + } + LOG.warn(String.format("Time to filter partitions: %d ms", timer.endTimer())); + } + else { + if (shouldSkipMetastoreForPartition) { + tableHandlePartitions = Collections.singletonList("").iterator(); + } + else { + metastorePartitions = Collections.singletonList(Collections.singletonList("")).iterator(); + } + } + } + + List batchHudiSplits = new ArrayList<>(); + int remaining = maxSize; + + while (remaining > 0 && !isFinished()) { + if (baseFiles.isEmpty()) { + Map> batchKeyMap; + if (shouldSkipMetastoreForPartition) { + List batchTableHandlePartitions = new ArrayList<>(); + Iterators.limit(tableHandlePartitions, partitionBatchNum).forEachRemaining(batchTableHandlePartitions::add); + batchKeyMap = batchTableHandlePartitions.stream()//.parallel() + .map(p -> Pair.of(p, buildPartitionKeys(partitionColumns, buildPartitionValues(p)))) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + } + else { + List> batchMetastorePartitions = new ArrayList<>(); + Iterators.limit(metastorePartitions, partitionBatchNum).forEachRemaining(batchMetastorePartitions::add); + batchKeyMap = batchMetastorePartitions.stream()//.parallel() + .map(partitionNames -> getPartitionPathToKeyUsingMetastore( + identity, metastore, table, table.getStorage().getLocation(), partitionNames)) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + } + + partitionMap.putAll(batchKeyMap); + List> baseFilesToAdd = batchKeyMap.keySet().stream()//.parallel() + .flatMap(relativePartitionPath -> fileSystemView.getLatestBaseFiles(relativePartitionPath) + .map(baseFile -> new ImmutablePair<>(baseFile, relativePartitionPath)) + .collect(Collectors.toList()).stream()) + .collect(Collectors.toList()); + + baseFilesToAdd.forEach(e -> baseFileToPartitionMap.put(e.getKey(), e.getValue())); + // TODO: skip partitions that are filtered out based on the predicate + baseFiles.addAll(baseFilesToAdd.stream().map(Pair::getKey).collect(Collectors.toList())); + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + List batchBaseFiles = new ArrayList<>(); + while (!baseFiles.isEmpty()) { + HoodieBaseFile baseFile = baseFiles.pollFirst(); + batchBaseFiles.add(baseFile); + } + + List hudiSplitsToAdd = batchBaseFiles.stream()//.parallel() + .flatMap(baseFile -> { + List hudiSplits; + try { + hudiSplits = getSplits( + fileSystem, HoodieInputFormatUtils.getFileStatus(baseFile)) + .stream() + .flatMap(fileSplit -> { + List result = new ArrayList<>(); + try { + result.add(new HudiSplit( + fileSplit.getPath().toString(), + fileSplit.getStart(), + fileSplit.getLength(), + metaClient.getFs().getLength(fileSplit.getPath()), + ImmutableList.of(), + tableHandle.getPredicate(), + partitionMap.get(baseFileToPartitionMap.get(baseFile)))); + } + catch (IOException e) { + throw new HoodieIOException(String.format( + "Unable to get Hudi split for %s, start=%d len=%d", + baseFile.getPath(), fileSplit.getStart(), fileSplit.getLength()), e); + } + return result.stream(); + }) + .collect(Collectors.toList()); + } + catch (IOException e) { + throw new HoodieIOException("Unable to get splits for " + baseFile.getPath(), e); + } + return hudiSplits.stream(); + }) + .collect(Collectors.toList()); + batchHudiSplits.addAll(hudiSplitsToAdd); + remaining -= hudiSplitsToAdd.size(); + if (remaining < hudiSplitsToAdd.size()) { + break; + } + } + + return batchHudiSplits; + } + + private Pair> getPartitionPathToKeyUsingMetastore( + HiveIdentity identity, + HiveMetastore metastore, + Table table, + String tablePath, + List partitionValues) + { + Optional partition; + String relativePartitionPath; + List partitionKeys; + partition = metastore.getPartition(identity, table, partitionValues); + String dataDir1 = partition.isPresent() + ? partition.get().getStorage().getLocation() + : tablePath; + relativePartitionPath = getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); + partitionKeys = getPartitionKeys(table, partition); + + return new ImmutablePair<>(relativePartitionPath, partitionKeys); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java new file mode 100644 index 0000000000000..db5012f9fb124 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +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.HivePartition; +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.TimelineUtils; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkState; +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 TupleDomain predicate; + private final Optional> partitions; + private final Optional metaClient; + + @JsonCreator + public HudiTableHandle( + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("basePath") String basePath, + @JsonProperty("tableType") HoodieTableType tableType, + @JsonProperty("predicate") TupleDomain predicate) + { + this(schemaName, tableName, basePath, tableType, predicate, Optional.empty(), Optional.empty()); + } + + public HudiTableHandle( + String schemaName, + String tableName, + String basePath, + HoodieTableType tableType, + TupleDomain predicate, + Optional metaClient) + { + this(schemaName, tableName, basePath, tableType, predicate, Optional.empty(), metaClient); + } + + public HudiTableHandle( + String schemaName, + String tableName, + String basePath, + HoodieTableType tableType, + TupleDomain predicate, + Optional> partitions, + Optional metaClient) + { + 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.predicate = requireNonNull(predicate, "predicate is null"); + this.partitions = requireNonNull(partitions, "partitions is null").map(ImmutableList::copyOf); + this.metaClient = requireNonNull(metaClient, "metaClient is null"); + } + + @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 getPredicate() + { + return predicate; + } + + @JsonIgnore + public Optional> getPartitions() + { + if (partitions.isEmpty()) { + List partitionIds = TimelineUtils.getPartitionsWritten(metaClient.get().getActiveTimeline()); + List hivePartitions = partitionIds.stream() + .map(p -> new HivePartition(getSchemaTableName(), p, ImmutableMap.of())) + .collect(Collectors.toList()); + return Optional.of(hivePartitions); + } + + return partitions; + } + + @JsonIgnore + public Optional getMetaClient() + { + return metaClient; + } + + public SchemaTableName getSchemaTableName() + { + return new SchemaTableName(schemaName, tableName); + } + + HudiTableHandle withPredicate(TupleDomain predicate) + { + checkState(this.predicate.isAll(), "There is already a predicate."); + return new HudiTableHandle( + schemaName, + tableName, + basePath, + tableType, + predicate, + partitions, + metaClient); + } + + @Override + public String toString() + { + return getSchemaTableName().toString(); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java new file mode 100644 index 0000000000000..7e9960f53bb20 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java @@ -0,0 +1,69 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.session.PropertyMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static io.trino.spi.session.PropertyMetadata.enumProperty; +import static io.trino.spi.session.PropertyMetadata.stringProperty; + +public class HudiTableProperties +{ + public static final String FILE_FORMAT_PROPERTY = "format"; + public static final String LOCATION_PROPERTY = "location"; + + private final List> tableProperties; + + @Inject + public HudiTableProperties(HudiConfig hudiConfig) + { + tableProperties = ImmutableList.>builder() + .add(enumProperty( + FILE_FORMAT_PROPERTY, + "File format for the table", + HoodieFileFormat.class, + hudiConfig.getFileFormat(), + false)) + .add(stringProperty( + LOCATION_PROPERTY, + "File system location URI for the table", + null, + false)) + .build(); + } + + public List> getTableProperties() + { + return tableProperties; + } + + public static HoodieFileFormat getFileFormat(Map tableProperties) + { + return (HoodieFileFormat) tableProperties.get(FILE_FORMAT_PROPERTY); + } + + public static Optional getTableLocation(Map tableProperties) + { + return Optional.ofNullable((String) tableProperties.get(LOCATION_PROPERTY)); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java new file mode 100644 index 0000000000000..7fad31361b2ab --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java @@ -0,0 +1,49 @@ +/* + * 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.ConnectorMetadata; +import io.trino.spi.connector.ConnectorTransactionHandle; + +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; + +public class HudiTransactionManager +{ + private final Map transactions = new ConcurrentHashMap<>(); + + public HudiMetadata get(ConnectorTransactionHandle transaction) + { + HudiMetadata metadata = transactions.get(transaction); + checkArgument(metadata != null, "no such transaction: %s", transaction); + return metadata; + } + + public HudiMetadata remove(ConnectorTransactionHandle transaction) + { + HudiMetadata metadata = transactions.remove(transaction); + checkArgument(metadata != null, "no such transaction: %s", transaction); + return metadata; + } + + public void put(ConnectorTransactionHandle transaction, HudiMetadata metadata) + { + ConnectorMetadata existing = transactions.putIfAbsent(transaction, metadata); + checkState(existing == null, "transaction already exists: %s", existing); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java new file mode 100644 index 0000000000000..4f2f106971e2b --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java @@ -0,0 +1,335 @@ +/* + * 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.log.Logger; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartition; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.metastore.Column; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; +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.Decimals; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeSignature; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.net.NetworkTopology; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.apache.hudi.hadoop.PathWithBootstrapFileStatus; + +import java.io.IOException; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeParseException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.plugin.hive.HivePartitionManager.parsePartition; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_INVALID_PARTITION_VALUE; +import static io.trino.spi.type.StandardTypes.BIGINT; +import static io.trino.spi.type.StandardTypes.BOOLEAN; +import static io.trino.spi.type.StandardTypes.DATE; +import static io.trino.spi.type.StandardTypes.DECIMAL; +import static io.trino.spi.type.StandardTypes.DOUBLE; +import static io.trino.spi.type.StandardTypes.INTEGER; +import static io.trino.spi.type.StandardTypes.REAL; +import static io.trino.spi.type.StandardTypes.SMALLINT; +import static io.trino.spi.type.StandardTypes.TIMESTAMP; +import static io.trino.spi.type.StandardTypes.TINYINT; +import static io.trino.spi.type.StandardTypes.VARBINARY; +import static io.trino.spi.type.StandardTypes.VARCHAR; +import static java.lang.Double.parseDouble; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.Float.parseFloat; +import static java.lang.Long.parseLong; +import static java.lang.String.format; +import static java.util.Objects.isNull; +import static org.apache.hadoop.hive.common.FileUtils.unescapePathName; + +public class HudiUtil +{ + private static final Logger LOG = Logger.get(HudiUtil.class); + private static final double SPLIT_SLOP = 1.1; // 10% slop + + private HudiUtil() {} + + public static HoodieTableMetaClient getMetaClient(Configuration conf, String basePath) + { + return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath) + //.setBootstrapIndexClassName(HudiHFileBootstrapIndex.class.getName()) + .build(); + } + + public static boolean isHudiParquetInputFormat(InputFormat inputFormat) + { + return inputFormat instanceof HoodieParquetInputFormat; + } + + public static List> splitPredicate( + 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(key, value); + } + else { + regularColumnPredicates.put(key, value); + } + })); + + return ImmutableList.of( + TupleDomain.withColumnDomains(partitionColumnPredicates), + TupleDomain.withColumnDomains(regularColumnPredicates)); + } + + public static Optional parseValuesAndFilterPartition( + SchemaTableName tableName, + String partitionId, + List partitionColumns, + List partitionColumnTypes, + TupleDomain constraintSummary) + { + LOG.warn(String.format("partitionId: %s partition Columns: %s", partitionId, partitionColumns)); + HivePartition partition = parsePartition(tableName, partitionId, partitionColumns, partitionColumnTypes); + + if (partitionMatches(partitionColumns, constraintSummary, partition)) { + LOG.warn(String.format("Match %s", partitionId)); + return Optional.of(partitionId); + } + LOG.warn(String.format("No match %s", partitionId)); + return Optional.empty(); + } + + public static boolean partitionMatches(List partitionColumns, TupleDomain constraintSummary, HivePartition partition) + { + if (constraintSummary.isNone()) { + LOG.warn("constraintSummary is none"); + return false; + } + Map domains = constraintSummary.getDomains().get(); + for (HiveColumnHandle column : partitionColumns) { + NullableValue value = partition.getKeys().get(column); + Domain allowedDomain = domains.get(column); + if (allowedDomain != null && !allowedDomain.includesNullableValue(value.getValue())) { + LOG.warn(String.format("Does not match: %s %s", allowedDomain, value)); + return false; + } + } + return true; + } + + public static Optional convertPartitionValue( + String partitionColumnName, + String partitionValue, + TypeSignature partitionDataType) + { + if (isNull(partitionValue)) { + return Optional.empty(); + } + + String baseType = partitionDataType.getBase(); + try { + switch (baseType) { + case TINYINT: + case SMALLINT: + case INTEGER: + case BIGINT: + return Optional.of(parseLong(partitionValue)); + case REAL: + return Optional.of((long) floatToRawIntBits(parseFloat(partitionValue))); + case DOUBLE: + return Optional.of(parseDouble(partitionValue)); + case VARCHAR: + case VARBINARY: + return Optional.of(utf8Slice(partitionValue)); + case DATE: + return Optional.of(LocalDate.parse(partitionValue, DateTimeFormatter.ISO_LOCAL_DATE).toEpochDay()); + case TIMESTAMP: + return Optional.of(Timestamp.valueOf(partitionValue).toLocalDateTime().toEpochSecond(ZoneOffset.UTC) * 1_000); + case BOOLEAN: + checkArgument(partitionValue.equalsIgnoreCase("true") || partitionValue.equalsIgnoreCase("false")); + return Optional.of(Boolean.valueOf(partitionValue)); + case DECIMAL: + return Optional.of(Decimals.parse(partitionValue).getObject()); + default: + throw new TrinoException(HUDI_INVALID_PARTITION_VALUE, + format("Unsupported data type '%s' for partition column %s", partitionDataType, partitionColumnName)); + } + } + catch (IllegalArgumentException | DateTimeParseException e) { + throw new TrinoException(HUDI_INVALID_PARTITION_VALUE, + format("Can not parse partition value '%s' of type '%s' for partition column '%s'", + partitionValue, partitionDataType, partitionColumnName)); + } + } + + public static List getSplits(FileSystem fs, FileStatus fileStatus) + throws IOException + { + if (fileStatus.isDirectory()) { + throw new IOException("Not a file: " + fileStatus.getPath()); + } + + Path path = fileStatus.getPath(); + long length = fileStatus.getLen(); + + // generate splits + List splits = new ArrayList<>(); + NetworkTopology clusterMap = new NetworkTopology(); + if (length != 0) { + BlockLocation[] blkLocations; + if (fileStatus instanceof LocatedFileStatus) { + blkLocations = ((LocatedFileStatus) fileStatus).getBlockLocations(); + } + else { + blkLocations = fs.getFileBlockLocations(fileStatus, 0, length); + } + if (isSplitable(fs, path)) { + long splitSize = fileStatus.getBlockSize(); + + long bytesRemaining = length; + while (((double) bytesRemaining) / splitSize > SPLIT_SLOP) { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, + length - bytesRemaining, splitSize, clusterMap); + splits.add(makeSplit(path, length - bytesRemaining, splitSize, + splitHosts[0], splitHosts[1])); + bytesRemaining -= splitSize; + } + + if (bytesRemaining != 0) { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, length + - bytesRemaining, bytesRemaining, clusterMap); + splits.add(makeSplit(path, length - bytesRemaining, bytesRemaining, + splitHosts[0], splitHosts[1])); + } + } + else { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, 0, length, clusterMap); + splits.add(makeSplit(path, 0, length, splitHosts[0], splitHosts[1])); + } + } + else { + //Create empty hosts array for zero length files + splits.add(makeSplit(path, 0, length, new String[0])); + } + return splits; + } + + private static boolean isSplitable(FileSystem fs, Path filename) + { + return !(filename instanceof PathWithBootstrapFileStatus); + } + + private static long computeSplitSize(long goalSize, long minSize, long blockSize) + { + return Math.max(minSize, Math.min(goalSize, blockSize)); + } + + private static FileSplit makeSplit(Path file, long start, long length, String[] hosts) + { + return new FileSplit(file, start, length, hosts); + } + + private static FileSplit makeSplit(Path file, long start, long length, String[] hosts, String[] inMemoryHosts) + { + return new FileSplit(file, start, length, hosts, inMemoryHosts); + } + + private static String[][] getSplitHostsAndCachedHosts(BlockLocation[] blkLocations, long offset, long splitSize, NetworkTopology clusterMap) + throws IOException + { + int startIndex = getBlockIndex(blkLocations, offset); + + return new String[][] {blkLocations[startIndex].getHosts(), + blkLocations[startIndex].getCachedHosts()}; + } + + private static int getBlockIndex(BlockLocation[] blkLocations, long offset) + { + for (int i = 0; i < blkLocations.length; i++) { + // is the offset inside this block? + if ((blkLocations[i].getOffset() <= offset) && + (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())) { + return i; + } + } + BlockLocation last = blkLocations[blkLocations.length - 1]; + long fileLength = last.getOffset() + last.getLength() - 1; + throw new IllegalArgumentException("Offset " + offset + + " is outside of file (0.." + + fileLength + ")"); + } + + public static List buildPartitionKeys(List keys, List 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 List buildPartitionValues(String partitionNames) + { + ImmutableList.Builder values = ImmutableList.builder(); + String[] parts = partitionNames.split("="); + if (parts.length == 1) { + values.add(unescapePathName(partitionNames)); + return values.build(); + } + if (parts.length == 2) { + values.add(unescapePathName(parts[1])); + return values.build(); + } + for (int i = 1; i < parts.length; i++) { + String val = parts[i]; + int j = val.lastIndexOf('/'); + if (j == -1) { + values.add(unescapePathName(val)); + } + else { + values.add(unescapePathName(val.substring(0, j))); + } + } + return values.build(); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java new file mode 100644 index 0000000000000..7105509cfefe9 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.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.ImmutableSet; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.TypeLiteral; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.event.client.EventModule; +import io.airlift.json.JsonModule; +import io.trino.plugin.base.CatalogName; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSinkProvider; +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.security.AllowAllAccessControl; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.HiveHdfsModule; +import io.trino.plugin.hive.NodeVersion; +import io.trino.plugin.hive.authentication.HdfsAuthenticationModule; +import io.trino.plugin.hive.azure.HiveAzureModule; +import io.trino.plugin.hive.gcs.HiveGcsModule; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreModule; +import io.trino.plugin.hive.s3.HiveS3Module; +import io.trino.spi.NodeManager; +import io.trino.spi.PageIndexerFactory; +import io.trino.spi.classloader.ThreadContextClassLoader; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorNodePartitioningProvider; +import io.trino.spi.connector.ConnectorPageSinkProvider; +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 java.util.Objects.requireNonNull; + +public final class InternalHudiConnectorFactory +{ + private InternalHudiConnectorFactory() {} + + public static Connector createConnector(String catalogName, Map config, ConnectorContext context, Module module) + { + return createConnector(catalogName, config, context, module, Optional.empty()); + } + + public static Connector createConnector(String catalogName, Map config, ConnectorContext context, Module module, Optional metastore) + { + requireNonNull(config, "config is null"); + ClassLoader classLoader = InternalHudiConnectorFactory.class.getClassLoader(); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + Bootstrap app = new Bootstrap( + new EventModule(), + new MBeanModule(), + new JsonModule(), + new HudiModule(), + new HiveMetastoreModule(metastore), + new HiveHdfsModule(), + new HiveS3Module(), + new HiveGcsModule(), + new HiveAzureModule(), + new HdfsAuthenticationModule(), + new MBeanServerModule(), + binder -> { + 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(PageIndexerFactory.class).toInstance(context.getPageIndexerFactory()); + binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); + }, + module); + + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + LifeCycleManager lifeCycleManager = injector.getInstance(LifeCycleManager.class); + HudiTransactionManager transactionManager = injector.getInstance(HudiTransactionManager.class); + HudiMetadataFactory metadataFactory = injector.getInstance(HudiMetadataFactory.class); + ConnectorSplitManager splitManager = injector.getInstance(ConnectorSplitManager.class); + ConnectorPageSourceProvider connectorPageSource = injector.getInstance(ConnectorPageSourceProvider.class); + ConnectorPageSinkProvider pageSinkProvider = injector.getInstance(ConnectorPageSinkProvider.class); + ConnectorNodePartitioningProvider connectorDistributionProvider = injector.getInstance(ConnectorNodePartitioningProvider.class); + Set sessionPropertiesProviders = injector.getInstance(Key.get(new TypeLiteral>() {})); + HudiTableProperties hudiTableProperties = injector.getInstance(HudiTableProperties.class); + Optional accessControl = Optional.of(new AllowAllAccessControl()); + + return new HudiConnector( + lifeCycleManager, + transactionManager, + metadataFactory, + new ClassLoaderSafeConnectorSplitManager(splitManager, classLoader), + new ClassLoaderSafeConnectorPageSourceProvider(connectorPageSource, classLoader), + new ClassLoaderSafeConnectorPageSinkProvider(pageSinkProvider, classLoader), + new ClassLoaderSafeNodePartitioningProvider(connectorDistributionProvider, classLoader), + ImmutableSet.of(), + sessionPropertiesProviders, + hudiTableProperties.getTableProperties(), + accessControl); + } + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java new file mode 100644 index 0000000000000..04a0899e2f507 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java @@ -0,0 +1,180 @@ +/* + * 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.Session; +import io.trino.plugin.hive.HivePlugin; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.spi.type.TimeZoneKey; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; + +import java.nio.file.Path; +import java.util.Locale; +import java.util.Map; + +import static io.trino.testing.TestingSession.testSessionBuilder; +import static java.lang.String.format; + +public class AbstractHudiTestQueryFramework + extends AbstractTestQueryFramework +{ + public static final String HUDI_CATALOG = "hudi"; + public static final String HIVE_CATALOG = "hive"; + public static final String HUDI_SCHEMA = "default"; + + static final String NON_PARTITIONED_TABLE_NAME = "hudi_non_part_cow"; + static final String PARTITIONED_COW_TABLE_NAME = "stock_ticks_cow"; + static final String PARTITIONED_MOR_TABLE_NAME = "stock_ticks_mor"; + + private static final String CREATE_NON_PARTITIONED_TABLE_STATEMENT = "CREATE TABLE %s.\"%s\".\"%s\" (\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" + + " rowid varchar,\n" + + " partitionid varchar,\n" + + " precomb bigint,\n" + + " name varchar,\n" + + " versionid varchar,\n" + + " tobedeletedstr varchar,\n" + + " inttolong integer,\n" + + " longtoint bigint\n" + + " )\n" + + " WITH (\n" + + " external_location = '%s',\n" + + " format = 'PARQUET'\n" + + " )"; + + private static final String CREATE_PARTITIONED_TABLE_STATEMENT = "CREATE TABLE %s.\"%s\".\"%s\" (\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" + + " external_location = '%s',\n" + + " format = 'PARQUET',\n" + + " partitioned_by = ARRAY['dt']\n" + + " )"; + + private static final Map TABLE_NAME_TO_CREATE_STATEMENT = new ImmutableMap.Builder() + .put(NON_PARTITIONED_TABLE_NAME, CREATE_NON_PARTITIONED_TABLE_STATEMENT) + .put(PARTITIONED_COW_TABLE_NAME, CREATE_PARTITIONED_TABLE_STATEMENT) + .put(PARTITIONED_MOR_TABLE_NAME, CREATE_PARTITIONED_TABLE_STATEMENT) + .build(); + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return createHudiQueryRunner(ImmutableMap.of()); + } + + protected void assertHudiQuery(String table, String testQuery, String expResults, boolean fail) + { + try { + syncHudiTableInMetastore(table); + if (!fail) { + assertQuery(testQuery, expResults); + } + else { + assertQueryFails(testQuery, expResults); + } + } + finally { + dropHudiTableFromMetastore(table); + } + } + + protected static String getTableBasePath(String tableName) + { + return AbstractHudiTestQueryFramework.class.getClassLoader().getResource(tableName).toString(); + } + + private static DistributedQueryRunner createHudiQueryRunner(Map extraProperties) + throws Exception + { + Session session = testSessionBuilder() + .setCatalog(HUDI_CATALOG) + .setSchema(HUDI_SCHEMA.toLowerCase(Locale.ROOT)) + .setTimeZoneKey(TimeZoneKey.UTC_KEY) + .build(); + + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session) + .setExtraProperties(extraProperties) + .build(); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hudi_metadata"); + Path catalogDir = dataDir.getParent().resolve("catalog"); + + // Install Hudi connector + queryRunner.installPlugin(new HudiPlugin()); + Map hudiProperties = ImmutableMap.builder() + .put("hive.metastore", "file") + .put("hive.metastore.catalog.dir", catalogDir.toFile().toURI().toString()) + .build(); + queryRunner.createCatalog(HUDI_CATALOG, "hudi", hudiProperties); + + // Install Hive connector + queryRunner.installPlugin(new HivePlugin()); + Map hiveProperties = ImmutableMap.builder() + .put("hive.metastore", "file") + .put("hive.metastore.catalog.dir", catalogDir.toFile().toURI().toString()) + .put("hive.allow-drop-table", "true") + .put("hive.security", "legacy") + .build(); + queryRunner.createCatalog(HIVE_CATALOG, "hive", hiveProperties); + queryRunner.execute(format("CREATE SCHEMA %s.%s", HIVE_CATALOG, HUDI_SCHEMA)); + + return queryRunner; + } + + protected void syncHudiTableInMetastore(String tableName) + { + getQueryRunner().execute(format( + TABLE_NAME_TO_CREATE_STATEMENT.get(tableName), + HIVE_CATALOG, + HUDI_SCHEMA, + tableName, + getTableBasePath(tableName))); + } + + protected void dropHudiTableFromMetastore(String tableName) + { + getQueryRunner().execute( + format("DROP TABLE IF EXISTS %s.\"%s\".\"%s\"", HIVE_CATALOG, HUDI_SCHEMA, tableName)); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java new file mode 100644 index 0000000000000..779ef22a04077 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.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 com.google.common.collect.ImmutableMap; +import io.airlift.units.DataSize; +import org.testng.annotations.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 org.apache.hudi.common.model.HoodieFileFormat.ORC; +import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; + +public class TestHudiConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(HudiConfig.class) + .setFileFormat(PARQUET) + .setMetadataEnabled(false) + .setMaxSplitSize(DataSize.ofBytes(128 * 1024 * 1024)) + .setSkipMetaStoreForPartition(true)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("hudi.file-format", "ORC") + .put("hudi.max-split-size", "256MB") + .put("hudi.metadata-enabled", "true") + .put("hudi.skip-metastore-for-partition", "false") + .build(); + + HudiConfig expected = new HudiConfig() + .setFileFormat(ORC) + .setMaxSplitSize(DataSize.of(256, DataSize.Unit.MEGABYTE)) + .setMetadataEnabled(true) + .setSkipMetaStoreForPartition(false); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java new file mode 100644 index 0000000000000..1fd113dbd7e4f --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.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 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.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.testing.Assertions.assertInstanceOf; +import static io.trino.plugin.hudi.AbstractHudiTestQueryFramework.HUDI_CATALOG; +import static io.trino.spi.transaction.IsolationLevel.READ_UNCOMMITTED; +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.builder() + .put("hive.metastore.uri", metastoreUri) + .build(); + + Connector connector = new HudiConnectorFactory(HUDI_CATALOG).create("test", config, new TestingConnectorContext()); + ConnectorTransactionHandle transaction = connector.beginTransaction(READ_UNCOMMITTED, true, true); + assertInstanceOf(connector.getMetadata(transaction), ClassLoaderSafeConnectorMetadata.class); + assertInstanceOf(connector.getSplitManager(), ClassLoaderSafeConnectorSplitManager.class); + assertInstanceOf(connector.getPageSourceProvider(), ConnectorPageSourceProvider.class); + connector.commit(transaction); + } + + private static void assertCreateConnectorFails(String metastoreUri, String exceptionString) + { + assertThatThrownBy(() -> assertCreateConnector(metastoreUri)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining(exceptionString); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java new file mode 100644 index 0000000000000..bb548e85703c1 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java @@ -0,0 +1,55 @@ +/* + * 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 org.testng.annotations.Test; + +import static java.lang.String.format; + +public class TestHudiSanity + extends AbstractHudiTestQueryFramework +{ + @Test + public void readNonPartitionedTable() + { + String testQuery = format("SELECT rowid, name FROM \"%s\"", NON_PARTITIONED_TABLE_NAME); + String expResults = "SELECT * FROM VALUES('row_1', 'bob'),('row_2', 'john'),('row_3', 'tom')"; + assertHudiQuery(NON_PARTITIONED_TABLE_NAME, testQuery, expResults, false); + } + + @Test + public void readPartitionedCowTable() + { + String testQuery = format("SELECT symbol, max(ts) FROM \"%s\" group by symbol HAVING symbol = 'GOOG'", PARTITIONED_COW_TABLE_NAME); + String expResults = "SELECT * FROM VALUES('GOOG', '2018-08-31 10:59:00')"; + assertHudiQuery(PARTITIONED_COW_TABLE_NAME, testQuery, expResults, false); + } + + @Test + public void readPartitionedMorTable() + { + String testQuery = format("SELECT symbol, max(ts) FROM \"%s\" group by symbol HAVING symbol = 'GOOG'", PARTITIONED_MOR_TABLE_NAME); + String expResults = "SELECT * FROM VALUES('GOOG', '2018-08-31 10:59:00')"; + assertHudiQuery(PARTITIONED_MOR_TABLE_NAME, testQuery, expResults, false); + } + + @Test + public void readPartitionedColumn() + { + String testQuery = format("SELECT dt, count(1) FROM \"%s\" group by dt", PARTITIONED_COW_TABLE_NAME); + String expResults = "SELECT * FROM VALUES('2018/08/31', '99')"; + assertHudiQuery(PARTITIONED_COW_TABLE_NAME, testQuery, expResults, false); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSplit.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSplit.java new file mode 100644 index 0000000000000..ecc6ae014ebdb --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSplit.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 com.google.common.collect.ImmutableList; +import io.airlift.json.JsonCodec; +import io.trino.spi.predicate.TupleDomain; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + +public class TestHudiSplit +{ + private final JsonCodec codec = JsonCodec.jsonCodec(HudiSplit.class); + + @Test + public void testJsonRoundTrip() + { + HudiSplit expectedSplit = new HudiSplit( + "/user/hive/warehouse/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet", + 0L, + 440747L, + 440747L, + ImmutableList.of(), + TupleDomain.all(), + ImmutableList.of()); + + String json = codec.toJson(expectedSplit); + HudiSplit actualSplit = codec.fromJson(json); + + assertEquals(actualSplit.getPath(), expectedSplit.getPath()); + assertEquals(actualSplit.getAddresses(), expectedSplit.getAddresses()); + assertEquals(actualSplit.getPartitionKeys(), expectedSplit.getPartitionKeys()); + assertEquals(actualSplit.getStart(), expectedSplit.getStart()); + assertEquals(actualSplit.getLength(), expectedSplit.getLength()); + assertEquals(actualSplit.getFileSize(), expectedSplit.getFileSize()); + assertEquals(actualSplit.getAddresses(), expectedSplit.getAddresses()); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiTableHandle.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiTableHandle.java new file mode 100644 index 0000000000000..3541b081436ed --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiTableHandle.java @@ -0,0 +1,49 @@ +/* + * 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.airlift.json.JsonCodec; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.model.HoodieTableType; +import org.testng.annotations.Test; + +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; +import static org.testng.Assert.assertEquals; + +public class TestHudiTableHandle +{ + private final JsonCodec codec = JsonCodec.jsonCodec(HudiTableHandle.class); + + @Test + public void testJsonRoundTrip() + { + HudiTableHandle expectedHandle = new HudiTableHandle( + "schema", + "table", + "/tmp/hudi_trips", + HoodieTableType.valueOf(COPY_ON_WRITE.name()), + TupleDomain.all()); + + String json = codec.toJson(expectedHandle); + HudiTableHandle actualHandle = codec.fromJson(json); + + assertEquals(actualHandle.getSchemaName(), expectedHandle.getSchemaName()); + assertEquals(actualHandle.getTableName(), expectedHandle.getTableName()); + assertEquals(actualHandle.getPredicate(), expectedHandle.getPredicate()); + assertEquals(actualHandle.getTableType(), expectedHandle.getTableType()); + assertEquals(actualHandle.getBasePath(), expectedHandle.getBasePath()); + assertEquals(actualHandle.getTableType(), expectedHandle.getTableType()); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiUtil.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiUtil.java new file mode 100644 index 0000000000000..fa00b76e1fc00 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiUtil.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.ImmutableList; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.Properties; + +import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; +import static io.trino.plugin.hive.util.HiveUtil.getInputFormat; +import static io.trino.plugin.hudi.HudiUtil.buildPartitionValues; +import static io.trino.plugin.hudi.HudiUtil.isHudiParquetInputFormat; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT; +import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class TestHudiUtil +{ + @Test + public void testIsHudiParquetInputFormat() + { + Properties schema = new Properties(); + schema.setProperty(FILE_INPUT_FORMAT, HoodieParquetInputFormat.class.getName()); + schema.setProperty(SERIALIZATION_LIB, PARQUET.getSerDe()); + + assertTrue(isHudiParquetInputFormat(getInputFormat(new Configuration(false), schema, false))); + } + + @Test + public void testBuildPartitionValues() + { + assertToPartitionValues("partitionColumn1=01/01/2020", ImmutableList.of("01/01/2020")); + assertToPartitionValues("partitionColumn1=01/01/2020/partitioncolumn2=abc", ImmutableList.of("01/01/2020", "abc")); + assertToPartitionValues("ds=2015-12-30/event_type=QueryCompletion", ImmutableList.of("2015-12-30", "QueryCompletion")); + assertToPartitionValues("ds=2015-12-30", ImmutableList.of("2015-12-30")); + assertToPartitionValues("a=1", ImmutableList.of("1")); + assertToPartitionValues("a=1/b=2/c=3", ImmutableList.of("1", "2", "3")); + assertToPartitionValues("pk=!@%23$%25%5E&%2A()%2F%3D", ImmutableList.of("!@#$%^&*()/=")); + assertToPartitionValues("pk=__HIVE_DEFAULT_PARTITION__", ImmutableList.of("__HIVE_DEFAULT_PARTITION__")); + } + + private static void assertToPartitionValues(String partitionName, List expected) + { + List actual = buildPartitionValues(partitionName); + assertEquals(actual, expected); + } +} diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit new file mode 100644 index 0000000000000..f77eeb137f026 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit @@ -0,0 +1,50 @@ +{ + "partitionToWriteStats" : { + "" : [ { + "fileId" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0", + "path" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet", + "prevCommit" : "null", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 436273, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 436273, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"hudi_non_part_cow_record\",\"namespace\":\"hoodie.hudi_non_part_cow\",\"fields\":[{\"name\":\"rowId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"partitionId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preComb\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"versionId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"toBeDeletedStr\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"intToLong\",\"type\":[\"null\",\"int\"],\"default\":null},{\"name\":\"longToInt\",\"type\":[\"null\",\"long\"],\"default\":null}]}" + }, + "operationType" : "INSERT", + "writePartitionPaths" : [ "" ], + "fileIdAndRelativePaths" : { + "d0875d00-483d-4e8b-bbbe-c520366c47a0-0" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet" + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 1743, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + } +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit.requested b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit.requested new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.inflight b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.inflight new file mode 100644 index 0000000000000..6605bcaf9b36c --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.inflight @@ -0,0 +1,48 @@ +{ + "partitionToWriteStats" : { + "" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "INSERT", + "writePartitionPaths" : [ "" ], + "fileIdAndRelativePaths" : { + "" : null + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + } +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/hoodie.properties new file mode 100644 index 0000000000000..3d03fa7915c39 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/hoodie.properties @@ -0,0 +1,14 @@ +#Properties saved on Fri Dec 17 11:05:14 UTC 2021 +#Fri Dec 17 11:05:14 UTC 2021 +hoodie.table.precombine.field=preComb +hoodie.table.partition.fields= +hoodie.table.type=COPY_ON_WRITE +hoodie.archivelog.folder=archived +hoodie.populate.meta.fields=true +hoodie.timeline.layout.version=1 +hoodie.table.version=3 +hoodie.table.recordkey.fields=rowId +hoodie.table.base.file.format=PARQUET +hoodie.table.keygenerator.class=org.apache.hudi.keygen.NonpartitionedKeyGenerator +hoodie.table.name=hudi_non_part_cow +hoodie.datasource.write.hive_style_partitioning=false diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie_partition_metadata new file mode 100644 index 0000000000000..f2149eb6cd5a3 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Fri Dec 17 11:05:23 UTC 2021 +commitTime=20211217110514527 +partitionDepth=0 diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet new file mode 100644 index 0000000000000..52de8719bf62d Binary files /dev/null and b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet differ diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit new file mode 100644 index 0000000000000..18cf55cc1bfd3 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit @@ -0,0 +1,51 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "871677fb-e0e3-46f8-9cc1-fe497e317216-0", + "path" : "2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet", + "prevCommit" : "null", + "numWrites" : 99, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 99, + "totalWriteBytes" : 440747, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "2018/08/31", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 440747, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", + "deltastreamer.checkpoint.key" : "stock_ticks,0:1668" + }, + "operationType" : "UPSERT", + "fileIdAndRelativePaths" : { + "871677fb-e0e3-46f8-9cc1-fe497e317216-0" : "2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet" + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 750, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit.requested b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit.requested new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.inflight b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.inflight new file mode 100644 index 0000000000000..6dc689a285d97 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.inflight @@ -0,0 +1,48 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 99, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "fileIdAndRelativePaths" : { + "" : null + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/hoodie.properties new file mode 100644 index 0000000000000..4754c1c23eb2e --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/hoodie.properties @@ -0,0 +1,13 @@ +#Properties saved on Thu Dec 16 07:14:51 UTC 2021 +#Thu Dec 16 07:14:51 UTC 2021 +hoodie.table.precombine.field=ts +hoodie.table.partition.fields=date +hoodie.table.type=COPY_ON_WRITE +hoodie.archivelog.folder=archived +hoodie.populate.meta.fields=true +hoodie.timeline.layout.version=1 +hoodie.table.version=3 +hoodie.table.recordkey.fields=key +hoodie.table.base.file.format=PARQUET +hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator +hoodie.table.name=stock_ticks_cow diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata new file mode 100644 index 0000000000000..1aaf9e64d9332 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Thu Dec 16 07:14:56 UTC 2021 +commitTime=20211216071453747 +partitionDepth=3 diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet new file mode 100644 index 0000000000000..b97391697e624 Binary files /dev/null and b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet differ diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit new file mode 100644 index 0000000000000..f9e28873d5244 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit @@ -0,0 +1,51 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", + "path" : "2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet", + "prevCommit" : "null", + "numWrites" : 99, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 99, + "totalWriteBytes" : 440746, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "2018/08/31", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 440746, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", + "deltastreamer.checkpoint.key" : "stock_ticks,0:1668" + }, + "operationType" : "UPSERT", + "fileIdAndRelativePaths" : { + "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : "2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet" + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 1402, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight new file mode 100644 index 0000000000000..6dc689a285d97 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight @@ -0,0 +1,48 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 99, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "fileIdAndRelativePaths" : { + "" : null + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.requested b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.requested new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit new file mode 100644 index 0000000000000..f1cc26fecc7b8 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit @@ -0,0 +1,55 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", + "path" : "2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29", + "prevCommit" : "20211221030120532", + "numWrites" : 99, + "numDeletes" : 0, + "numUpdateWrites" : 99, + "numInserts" : 0, + "totalWriteBytes" : 22220, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "2018/08/31", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 22220, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet", + "logFiles" : [ ".167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29" ] + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", + "deltastreamer.checkpoint.key" : "stock_ticks,0:3336" + }, + "operationType" : "UPSERT", + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 187, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ], + "fileIdAndRelativePaths" : { + "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : "2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29" + } +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight new file mode 100644 index 0000000000000..724ce56ff0d6d --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight @@ -0,0 +1,71 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", + "path" : null, + "prevCommit" : "20211221030120532", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 99, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ], + "fileIdAndRelativePaths" : { + "" : null, + "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : null + } +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.requested b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.requested new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/hoodie.properties new file mode 100644 index 0000000000000..33392aa182f21 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/hoodie.properties @@ -0,0 +1,14 @@ +#Properties saved on Tue Dec 21 03:01:13 UTC 2021 +#Tue Dec 21 03:01:13 UTC 2021 +hoodie.table.precombine.field=ts +hoodie.table.partition.fields=date +hoodie.table.type=MERGE_ON_READ +hoodie.archivelog.folder=archived +hoodie.populate.meta.fields=true +hoodie.compaction.payload.class=org.apache.hudi.common.model.OverwriteWithLatestAvroPayload +hoodie.timeline.layout.version=1 +hoodie.table.version=3 +hoodie.table.recordkey.fields=key +hoodie.table.base.file.format=PARQUET +hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator +hoodie.table.name=stock_ticks_mor diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 new file mode 100644 index 0000000000000..da3c7bc07ee11 Binary files /dev/null and b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 differ diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata new file mode 100644 index 0000000000000..340533d6e6809 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Dec 21 03:01:25 UTC 2021 +commitTime=20211221030120532 +partitionDepth=3 diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet new file mode 100644 index 0000000000000..9fe2112d09bb4 Binary files /dev/null and b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet differ diff --git a/pom.xml b/pom.xml index 887dcedd6d90d..16917a281a633 100644 --- a/pom.xml +++ b/pom.xml @@ -120,6 +120,7 @@ plugin/trino-hive plugin/trino-hive-hadoop2 plugin/trino-http-event-listener + plugin/trino-hudi plugin/trino-iceberg plugin/trino-jmx plugin/trino-kafka @@ -271,6 +272,12 @@ ${project.version} + + io.trino + trino-hudi + ${project.version} + + io.trino trino-iceberg diff --git a/testing/trino-server-dev/etc/catalog/hudi.properties b/testing/trino-server-dev/etc/catalog/hudi.properties new file mode 100644 index 0000000000000..23bc4e0573fc9 --- /dev/null +++ b/testing/trino-server-dev/etc/catalog/hudi.properties @@ -0,0 +1,17 @@ +# +# 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. +# + +connector.name=hudi +hive.metastore.uri=thrift://localhost:9083 +#hive.config.resources=/path/to/core-site.xml,/path/to/hdfs-site.xml diff --git a/testing/trino-server-dev/etc/config.properties b/testing/trino-server-dev/etc/config.properties index e1e4530f74213..fd2beb31dbaae 100644 --- a/testing/trino-server-dev/etc/config.properties +++ b/testing/trino-server-dev/etc/config.properties @@ -36,6 +36,7 @@ plugin.bundles=\ ../../plugin/trino-jmx/pom.xml,\ ../../plugin/trino-raptor-legacy/pom.xml,\ ../../plugin/trino-hive-hadoop2/pom.xml,\ + ../../plugin/trino-hudi/pom.xml,\ ../../plugin/trino-example-http/pom.xml,\ ../../plugin/trino-kafka/pom.xml, \ ../../plugin/trino-tpch/pom.xml, \