From d7fae54cb75895dceb901eea459a40af68c946a8 Mon Sep 17 00:00:00 2001 From: Vamsi Date: Wed, 20 Aug 2025 15:28:50 +0530 Subject: [PATCH 1/8] Upgrade Hudi version in Presto-hudi module to 1.0.2 Batch calls to metastore for fetching partitions --- presto-hudi/pom.xml | 25 +++++++++ .../facebook/presto/hudi/HudiErrorCode.java | 1 + .../facebook/presto/hudi/HudiMetadata.java | 3 +- .../presto/hudi/HudiPartitionManager.java | 46 +++++++++++----- .../presto/hudi/HudiSplitManager.java | 52 +++--------------- .../facebook/presto/hudi/HudiSplitSource.java | 7 +-- .../facebook/presto/hudi/HudiTableHandle.java | 26 +++++++++ .../hudi/split/HudiBackgroundSplitLoader.java | 14 ++--- .../split/HudiPartitionSplitGenerator.java | 53 +++++++++++++++++-- 9 files changed, 155 insertions(+), 72 deletions(-) diff --git a/presto-hudi/pom.xml b/presto-hudi/pom.xml index 63c0ced3bd811..b64177f57b3c3 100644 --- a/presto-hudi/pom.xml +++ b/presto-hudi/pom.xml @@ -148,6 +148,7 @@ org.apache.hudi hudi-presto-bundle + 1.0.2 @@ -263,6 +264,30 @@ module-info META-INF.versions.9.module-info + + + io.airlift + aircompressor + + + org.apache.hudi + hudi-presto-bundle + + + org.apache.hive + hive-storage-api + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + io.airlift:aircompressor:jar + diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiErrorCode.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiErrorCode.java index c240e47446055..ae162de8e1650 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiErrorCode.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiErrorCode.java @@ -31,6 +31,7 @@ public enum HudiErrorCode HUDI_CANNOT_OPEN_SPLIT(0x41, EXTERNAL), HUDI_CURSOR_ERROR(0x42, EXTERNAL), HUDI_CANNOT_GENERATE_SPLIT(0x43, EXTERNAL), + HUDI_PARTITION_NOT_FOUND(0x44, EXTERNAL), /**/; private final ErrorCode errorCode; diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java index 4e9c4e0248c5a..fae49cf97b01d 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java @@ -101,6 +101,7 @@ public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTable } return new HudiTableHandle( + Optional.of(table), table.getDatabaseName(), table.getTableName(), table.getStorage().getLocation(), @@ -241,7 +242,7 @@ static List fromPartitionColumns(List partitionColumns return builder.build(); } - static List fromDataColumns(List dataColumns) + public static List fromDataColumns(List dataColumns) { ImmutableList.Builder builder = ImmutableList.builder(); int id = 0; diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java index ba80fd2f516c7..dddffb52e0644 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java @@ -23,10 +23,12 @@ import com.facebook.presto.hive.metastore.Column; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; import com.facebook.presto.hive.metastore.MetastoreContext; +import com.facebook.presto.hive.metastore.Partition; +import com.facebook.presto.hive.metastore.StorageFormat; import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSession; -import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.PrestoException; import com.google.common.base.Verify; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -42,10 +44,11 @@ import static com.facebook.presto.hive.HiveUtil.parsePartitionValue; import static com.facebook.presto.hive.metastore.MetastoreUtil.extractPartitionValues; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_PARTITION_NOT_FOUND; import static com.facebook.presto.hudi.HudiMetadata.fromPartitionColumns; import static com.facebook.presto.hudi.HudiMetadata.toMetastoreContext; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.toList; public class HudiPartitionManager { @@ -57,18 +60,27 @@ public HudiPartitionManager(TypeManager typeManager) this.typeManager = requireNonNull(typeManager, "typeManager is null"); } - public List getEffectivePartitions( + public Map getEffectivePartitions( ConnectorSession connectorSession, ExtendedHiveMetastore metastore, - SchemaTableName schemaTableName, + HudiTableHandle tableHandle, TupleDomain constraintSummary) { MetastoreContext metastoreContext = toMetastoreContext(connectorSession); - Optional table = metastore.getTable(metastoreContext, schemaTableName.getSchemaName(), schemaTableName.getTableName()); + Optional
table = metastore.getTable(metastoreContext, tableHandle.getSchemaName(), tableHandle.getTableName()); Verify.verify(table.isPresent()); List partitionColumns = table.get().getPartitionColumns(); if (partitionColumns.isEmpty()) { - return ImmutableList.of(""); + return ImmutableMap.of( + "", Partition.builder() + .setDatabaseName(tableHandle.getSchemaName()) + .setTableName(tableHandle.getTableName()) + .withStorage(storageBuilder -> + storageBuilder.setLocation(tableHandle.getPath()) + .setStorageFormat(StorageFormat.VIEW_STORAGE_FORMAT)) + .setColumns(ImmutableList.of()) + .setValues(ImmutableList.of()) + .build()); } Map partitionPredicate = new HashMap<>(); @@ -84,20 +96,28 @@ public List getEffectivePartitions( partitionPredicate.put(partitionColumn, Domain.all(column.getHiveType().getType(typeManager))); } } - List partitionNames = metastore.getPartitionNamesByFilter(metastoreContext, schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionPredicate); + List partitionNames = metastore.getPartitionNamesByFilter(metastoreContext, tableHandle.getSchemaName(), tableHandle.getTableName(), partitionPredicate); List partitionTypes = partitionColumns.stream() .map(column -> typeManager.getType(column.getType().getTypeSignature())) - .collect(toList()); + .toList(); - return partitionNames.stream() - .map(PartitionNameWithVersion::getPartitionName) + List filteredPartitionNames = partitionNames.stream() // Apply extra filters which could not be done by getPartitionNamesByFilter, similar to filtering in HivePartitionManager#getPartitionsIterator - .filter(partitionName -> parseValuesAndFilterPartition( - partitionName, + .filter(partitionNameWithVersion -> parseValuesAndFilterPartition( + partitionNameWithVersion.getPartitionName(), hudiColumnHandles, partitionTypes, constraintSummary)) - .collect(toList()); + .toList(); + Map> partitionsByNames = metastore.getPartitionsByNames(metastoreContext, tableHandle.getSchemaName(), tableHandle.getTableName(), filteredPartitionNames); + List partitionsNotFound = partitionsByNames.entrySet().stream().filter(e -> e.getValue().isEmpty()).map(Map.Entry::getKey).toList(); + if (!partitionsNotFound.isEmpty()) { + throw new PrestoException(HUDI_PARTITION_NOT_FOUND, format("Cannot find partitions in metastore: %s", partitionsNotFound)); + } + return partitionsByNames + .entrySet().stream() + .filter(e -> e.getValue().isPresent()) + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().get())); } private boolean parseValuesAndFilterPartition( diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java index e1f38c291aa88..8228be8a45c86 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java @@ -19,9 +19,7 @@ import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.filesystem.ExtendedFileSystem; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; -import com.facebook.presto.hive.metastore.MetastoreContext; import com.facebook.presto.hive.metastore.Partition; -import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.hudi.split.ForHudiBackgroundSplitLoader; import com.facebook.presto.hudi.split.ForHudiSplitAsyncQueue; import com.facebook.presto.hudi.split.ForHudiSplitSource; @@ -33,8 +31,6 @@ import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Streams; import jakarta.inject.Inject; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -45,21 +41,17 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.hadoop.fs.HadoopFSUtils; +import org.apache.hudi.storage.StorageConfiguration; import java.io.IOException; -import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; -import static com.facebook.presto.hive.metastore.MetastoreUtil.extractPartitionValues; import static com.facebook.presto.hudi.HudiErrorCode.HUDI_FILESYSTEM_ERROR; -import static com.facebook.presto.hudi.HudiErrorCode.HUDI_INVALID_METADATA; -import static com.facebook.presto.hudi.HudiMetadata.fromDataColumns; import static com.facebook.presto.hudi.HudiSessionProperties.getMaxOutstandingSplits; import static com.facebook.presto.hudi.HudiSessionProperties.isHudiMetadataTableEnabled; -import static com.google.common.base.Preconditions.checkArgument; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static org.apache.hudi.common.table.view.FileSystemViewManager.createInMemoryFileSystemViewWithTimeline; @@ -104,8 +96,8 @@ public ConnectorSplitSource getSplits( HudiTableHandle table = layout.getTable(); // Retrieve and prune partitions - HoodieTimer timer = new HoodieTimer().startTimer(); - List partitions = hudiPartitionManager.getEffectivePartitions(session, metastore, table.getSchemaTableName(), layout.getTupleDomain()); + HoodieTimer timer = HoodieTimer.start(); + Map partitions = hudiPartitionManager.getEffectivePartitions(session, metastore, table, layout.getTupleDomain()); log.debug("Took %d ms to get %d partitions", timer.endTimer(), partitions.size()); if (partitions.isEmpty()) { return new FixedSplitSource(ImmutableList.of()); @@ -114,10 +106,10 @@ public ConnectorSplitSource getSplits( // Load Hudi metadata ExtendedFileSystem fs = getFileSystem(session, table); HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().enable(isHudiMetadataTableEnabled(session)).build(); - Configuration conf = fs.getConf(); + StorageConfiguration conf = HadoopFSUtils.getStorageConfWithCopy(fs.getConf()); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(conf).setBasePath(table.getPath()).build(); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - String timestamp = timeline.lastInstant().map(HoodieInstant::getTimestamp).orElse(null); + String timestamp = timeline.lastInstant().map(HoodieInstant::requestedTime).orElse(null); if (timestamp == null) { // no completed instant for current table return new FixedSplitSource(ImmutableList.of()); @@ -153,36 +145,4 @@ private ExtendedFileSystem getFileSystem(ConnectorSession session, HudiTableHand throw new PrestoException(HUDI_FILESYSTEM_ERROR, "Could not open file system for " + table, e); } } - - public static HudiPartition getHudiPartition(ExtendedHiveMetastore metastore, MetastoreContext context, HudiTableLayoutHandle tableLayout, String partitionName) - { - String databaseName = tableLayout.getTable().getSchemaName(); - String tableName = tableLayout.getTable().getTableName(); - List partitionColumns = tableLayout.getPartitionColumns(); - - if (partitionColumns.isEmpty()) { - // non-partitioned tableLayout - Table table = metastore.getTable(context, databaseName, tableName) - .orElseThrow(() -> new PrestoException(HUDI_INVALID_METADATA, format("Table %s.%s expected but not found", databaseName, tableName))); - return new HudiPartition(partitionName, ImmutableList.of(), ImmutableMap.of(), table.getStorage(), tableLayout.getDataColumns()); - } - else { - // partitioned tableLayout - List partitionValues = extractPartitionValues(partitionName); - checkArgument(partitionColumns.size() == partitionValues.size(), - format("Invalid partition name %s for partition columns %s", partitionName, partitionColumns)); - Partition partition = metastore.getPartition(context, databaseName, tableName, partitionValues) - .orElseThrow(() -> new PrestoException(HUDI_INVALID_METADATA, format("Partition %s expected but not found", partitionName))); - Map keyValues = zipPartitionKeyValues(partitionColumns, partitionValues); - return new HudiPartition(partitionName, partitionValues, keyValues, partition.getStorage(), fromDataColumns(partition.getColumns())); - } - } - - private static Map zipPartitionKeyValues(List partitionColumns, List partitionValues) - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - Streams.forEachPair(partitionColumns.stream(), partitionValues.stream(), - (column, value) -> builder.put(column.getName(), value)); - return builder.build(); - } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java index 989870b2bee7b..6d74c46d4583c 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java @@ -15,6 +15,7 @@ package com.facebook.presto.hudi; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; +import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.util.AsyncQueue; import com.facebook.presto.hudi.split.HudiBackgroundSplitLoader; import com.facebook.presto.spi.ConnectorSession; @@ -24,7 +25,7 @@ import com.google.common.util.concurrent.Futures; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; -import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; @@ -46,7 +47,7 @@ public HudiSplitSource( ExtendedHiveMetastore metastore, HudiTableLayoutHandle layout, HoodieTableFileSystemView fsView, - List partitions, + Map partitionMap, String latestInstant, ExecutorService asyncQueueExecutor, ScheduledExecutorService splitLoaderExecutorService, @@ -61,7 +62,7 @@ public HudiSplitSource( layout, fsView, queue, - partitions, + partitionMap, latestInstant); this.splitLoaderFuture = splitLoaderExecutorService.schedule( this.splitLoader, 0, TimeUnit.MILLISECONDS); diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java index 686267d2d1c8a..6b58408b09088 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java @@ -14,13 +14,16 @@ package com.facebook.presto.hudi; +import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.spi.ConnectorTableHandle; import com.facebook.presto.spi.SchemaTableName; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Objects; +import java.util.Optional; +import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; public class HudiTableHandle @@ -30,6 +33,7 @@ public class HudiTableHandle private final String tableName; private final String path; private final HudiTableType hudiTableType; + private Optional
table; @JsonCreator public HudiTableHandle( @@ -44,6 +48,28 @@ public HudiTableHandle( this.hudiTableType = requireNonNull(hudiTableType, "tableType is null"); } + public HudiTableHandle( + Optional
table, + String schemaName, + String tableName, + String path, + HudiTableType hudiTableType) + { + this.table = requireNonNull(table, "table is null"); + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.path = requireNonNull(path, "path is null"); + this.hudiTableType = requireNonNull(hudiTableType, "tableType is null"); + } + + public Table getTable() + { + checkArgument(table.isPresent(), + "getTable() called on a table handle that has no metastore table object; " + + "this is likely because it is called on the worker."); + return table.get(); + } + @JsonProperty public String getSchemaName() { diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java index 14894adf26318..b832271297dbd 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java @@ -16,6 +16,7 @@ import com.facebook.airlift.log.Logger; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; +import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.util.AsyncQueue; import com.facebook.presto.hudi.HudiTableLayoutHandle; import com.facebook.presto.spi.ConnectorSession; @@ -26,6 +27,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -48,7 +50,7 @@ public class HudiBackgroundSplitLoader private final HudiTableLayoutHandle layout; private final HoodieTableFileSystemView fsView; private final AsyncQueue asyncQueue; - private final List partitions; + private final Map partitionMap; private final String latestInstant; private final int splitGeneratorNumThreads; private final ExecutorService splitGeneratorExecutorService; @@ -60,7 +62,7 @@ public HudiBackgroundSplitLoader( HudiTableLayoutHandle layout, HoodieTableFileSystemView fsView, AsyncQueue asyncQueue, - List partitions, + Map partitionMap, String latestInstant) { this.session = requireNonNull(session, "session is null"); @@ -68,7 +70,7 @@ public HudiBackgroundSplitLoader( this.layout = requireNonNull(layout, "layout is null"); this.fsView = requireNonNull(fsView, "fsView is null"); this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); - this.partitions = requireNonNull(partitions, "partitions is null"); + this.partitionMap = requireNonNull(partitionMap, "partitions is null"); this.latestInstant = requireNonNull(latestInstant, "latestInstant is null"); this.splitGeneratorNumThreads = getSplitGeneratorParallelism(session); @@ -78,15 +80,15 @@ public HudiBackgroundSplitLoader( @Override public void run() { - HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieTimer timer = HoodieTimer.start(); List splitGeneratorList = new ArrayList<>(); List splitGeneratorFutures = new ArrayList<>(); - ConcurrentLinkedQueue concurrentPartitionQueue = new ConcurrentLinkedQueue<>(partitions); + ConcurrentLinkedQueue concurrentPartitionQueue = new ConcurrentLinkedQueue<>(partitionMap.keySet()); // Start a number of partition split generators to generate the splits in parallel for (int i = 0; i < splitGeneratorNumThreads; i++) { HudiPartitionSplitGenerator generator = new HudiPartitionSplitGenerator( - session, metastore, layout, fsView, asyncQueue, concurrentPartitionQueue, latestInstant); + session, metastore, layout, fsView, partitionMap, asyncQueue, concurrentPartitionQueue, latestInstant); splitGeneratorList.add(generator); splitGeneratorFutures.add(splitGeneratorExecutorService.submit(generator)); } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java index 54fd0f4bda8a0..f7d22bc1f1716 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java @@ -18,7 +18,10 @@ import com.facebook.airlift.units.DataSize; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; import com.facebook.presto.hive.metastore.MetastoreContext; +import com.facebook.presto.hive.metastore.Partition; +import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.hive.util.AsyncQueue; +import com.facebook.presto.hudi.HudiColumnHandle; import com.facebook.presto.hudi.HudiFile; import com.facebook.presto.hudi.HudiPartition; import com.facebook.presto.hudi.HudiSplit; @@ -27,25 +30,34 @@ import com.facebook.presto.hudi.HudiTableType; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.schedule.NodeSelectionStrategy; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Streams; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.storage.StoragePath; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Queue; import java.util.stream.Stream; +import static com.facebook.presto.hive.metastore.MetastoreUtil.extractPartitionValues; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_INVALID_METADATA; +import static com.facebook.presto.hudi.HudiMetadata.fromDataColumns; import static com.facebook.presto.hudi.HudiMetadata.toMetastoreContext; import static com.facebook.presto.hudi.HudiSessionProperties.getMinimumAssignedSplitWeight; import static com.facebook.presto.hudi.HudiSessionProperties.getStandardSplitWeightSize; import static com.facebook.presto.hudi.HudiSessionProperties.isSizeBasedSplitWeightsEnabled; -import static com.facebook.presto.hudi.HudiSplitManager.getHudiPartition; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; /** @@ -68,12 +80,14 @@ public class HudiPartitionSplitGenerator private final Queue concurrentPartitionQueue; private final String latestInstant; private final HudiSplitWeightProvider splitWeightProvider; + private final Map partitionMap; public HudiPartitionSplitGenerator( ConnectorSession session, ExtendedHiveMetastore metastore, HudiTableLayoutHandle layout, HoodieTableFileSystemView fsView, + Map partitionMap, AsyncQueue asyncQueue, Queue concurrentPartitionQueue, String latestInstant) @@ -84,6 +98,7 @@ public HudiPartitionSplitGenerator( this.table = layout.getTable(); this.tablePath = new Path(table.getPath()); this.fsView = requireNonNull(fsView, "fsView is null"); + this.partitionMap = requireNonNull(partitionMap, "partitionMap is null"); this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); this.concurrentPartitionQueue = requireNonNull(concurrentPartitionQueue, "concurrentPartitionQueue is null"); this.latestInstant = requireNonNull(latestInstant, "latestInstant is null"); @@ -93,7 +108,7 @@ public HudiPartitionSplitGenerator( @Override public void run() { - HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieTimer timer = HoodieTimer.start(); while (!concurrentPartitionQueue.isEmpty()) { String partitionName = concurrentPartitionQueue.poll(); if (partitionName != null) { @@ -107,7 +122,7 @@ private void generateSplitsFromPartition(String partitionName) { HudiPartition hudiPartition = getHudiPartition(metastore, metastoreContext, layout, partitionName); Path partitionPath = new Path(hudiPartition.getStorage().getLocation()); - String relativePartitionPath = FSUtils.getRelativePartitionPath(tablePath, partitionPath); + String relativePartitionPath = FSUtils.getRelativePartitionPath(new StoragePath(tablePath.toUri()), new StoragePath(partitionPath.toUri())); Stream fileSlices = HudiTableType.MOR.equals(table.getTableType()) ? fsView.getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, latestInstant) : fsView.getLatestFileSlicesBeforeOrOn(relativePartitionPath, latestInstant, false); @@ -117,6 +132,38 @@ private void generateSplitsFromPartition(String partitionName) .forEach(asyncQueue::offer); } + private HudiPartition getHudiPartition(ExtendedHiveMetastore metastore, MetastoreContext context, HudiTableLayoutHandle tableLayout, String partitionName) + { + String databaseName = tableLayout.getTable().getSchemaName(); + String tableName = tableLayout.getTable().getTableName(); + List partitionColumns = tableLayout.getPartitionColumns(); + + if (partitionColumns.isEmpty()) { + // non-partitioned tableLayout + Table metastoreTable = Optional.ofNullable(table.getTable()) + .orElseThrow(() -> new PrestoException(HUDI_INVALID_METADATA, format("Table %s.%s expected but not found", databaseName, tableName))); + return new HudiPartition(partitionName, ImmutableList.of(), ImmutableMap.of(), metastoreTable.getStorage(), tableLayout.getDataColumns()); + } + else { + // partitioned tableLayout + List partitionValues = extractPartitionValues(partitionName); + checkArgument(partitionColumns.size() == partitionValues.size(), + format("Invalid partition name %s for partition columns %s", partitionName, partitionColumns)); + Partition partition = Optional.ofNullable(partitionMap.get(partitionName)) + .orElseThrow(() -> new PrestoException(HUDI_INVALID_METADATA, format("Partition %s expected but not found", partitionName))); + Map keyValues = zipPartitionKeyValues(partitionColumns, partitionValues); + return new HudiPartition(partitionName, partitionValues, keyValues, partition.getStorage(), fromDataColumns(partition.getColumns())); + } + } + + private Map zipPartitionKeyValues(List partitionColumns, List partitionValues) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + Streams.forEachPair(partitionColumns.stream(), partitionValues.stream(), + (column, value) -> builder.put(column.getName(), value)); + return builder.build(); + } + private Optional createHudiSplit( HudiTableHandle table, FileSlice slice, From 1f946fde620e8399690dcd50adbc0ebd84cfb1ae Mon Sep 17 00:00:00 2001 From: Vamsi Date: Thu, 21 Aug 2025 21:46:53 +0530 Subject: [PATCH 2/8] Fix tests --- .../presto/hudi/TestHudiPartitionManager.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java index ba445bc1d2054..3869826670d92 100644 --- a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java +++ b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java @@ -23,18 +23,20 @@ import com.facebook.presto.hive.OrcFileWriterConfig; import com.facebook.presto.hive.ParquetFileWriterConfig; import com.facebook.presto.hive.metastore.Column; +import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.metastore.PrestoTableType; import com.facebook.presto.hive.metastore.Storage; import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSession; -import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.testing.TestingConnectorSession; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import org.testng.annotations.Test; import java.util.List; +import java.util.Map; import java.util.Optional; import static com.facebook.presto.common.type.VarcharType.VARCHAR; @@ -100,11 +102,12 @@ public void testParseValuesAndFilterPartition() Optional.empty(), HudiColumnHandle.ColumnType.PARTITION_KEY), Domain.singleValue(VARCHAR, utf8Slice("2019-07-23")))); - List actualPartitions = hudiPartitionManager.getEffectivePartitions( + HudiTableHandle tableHandle = new HudiTableHandle(Optional.of(TABLE), SCHEMA_NAME, TABLE_NAME, LOCATION, HudiTableType.COW); + Map actualPartitions = hudiPartitionManager.getEffectivePartitions( session, metastore, - new SchemaTableName(SCHEMA_NAME, TABLE_NAME), + tableHandle, constraintSummary); - assertEquals(actualPartitions, ImmutableList.of("ds=2019-07-23")); + assertEquals(actualPartitions.keySet(), ImmutableSet.of("ds=2019-07-23")); } } From cf99f8ae3e11d2eca5d5690ebbbc4095893cc76e Mon Sep 17 00:00:00 2001 From: Vamsi Date: Fri, 22 Aug 2025 17:40:56 +0530 Subject: [PATCH 3/8] fix test failures --- .../presto/hudi/HudiPartitionManager.java | 1 + .../presto/hudi/HudiRecordCursors.java | 3 ++ .../presto/hudi/TestHudiPartitionManager.java | 31 +++++++++++++++++-- .../hudi/TestingExtendedHiveMetastore.java | 15 +++++++-- 4 files changed, 44 insertions(+), 6 deletions(-) diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java index dddffb52e0644..3e3440fa96e32 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java @@ -73,6 +73,7 @@ public Map getEffectivePartitions( if (partitionColumns.isEmpty()) { return ImmutableMap.of( "", Partition.builder() + .setCatalogName(Optional.empty()) .setDatabaseName(tableHandle.getSchemaName()) .setTableName(tableHandle.getTableName()) .withStorage(storageBuilder -> diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiRecordCursors.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiRecordCursors.java index dd73e684fc370..7fd43f7258f96 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiRecordCursors.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiRecordCursors.java @@ -55,6 +55,7 @@ import static org.apache.hadoop.hive.serde2.ColumnProjectionUtils.READ_ALL_COLUMNS; import static org.apache.hadoop.hive.serde2.ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR; import static org.apache.hadoop.hive.serde2.ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR; +import static org.apache.hudi.common.config.HoodieReaderConfig.FILE_GROUP_READER_ENABLED; class HudiRecordCursors { @@ -105,6 +106,8 @@ public static RecordCursor createRealtimeRecordCursor( jobConf.setBoolean(READ_ALL_COLUMNS, false); jobConf.set(READ_COLUMN_IDS_CONF_STR, join(dataColumns, HudiColumnHandle::getId)); jobConf.set(READ_COLUMN_NAMES_CONF_STR, join(dataColumns, HudiColumnHandle::getName)); + // Disable file group reader since enabling it makes record cursor fail + jobConf.setBoolean(FILE_GROUP_READER_ENABLED.key(), false); schema.stringPropertyNames() .forEach(name -> jobConf.set(name, schema.getProperty(name))); refineCompressionCodecs(jobConf); diff --git a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java index 3869826670d92..1d354d8f2a653 100644 --- a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java +++ b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java @@ -26,6 +26,7 @@ import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.metastore.PrestoTableType; import com.facebook.presto.hive.metastore.Storage; +import com.facebook.presto.hive.metastore.StorageFormat; import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSession; @@ -35,7 +36,7 @@ import com.google.common.collect.ImmutableSet; import org.testng.annotations.Test; -import java.util.List; +import java.util.Collections; import java.util.Map; import java.util.Optional; @@ -80,9 +81,33 @@ public class TestHudiPartitionManager Optional.empty(), Optional.empty()); - private static final List PARTITIONS = ImmutableList.of("ds=2019-07-23", "ds=2019-08-23"); + private static final Map> PARTITION_MAP = ImmutableMap.of( + "ds=2019-07-23", + Optional.of( + Partition.builder() + .setCatalogName(Optional.empty()) + .setDatabaseName(SCHEMA_NAME) + .setTableName(TABLE_NAME) + .withStorage(storageBuilder -> + storageBuilder.setLocation(LOCATION) + .setStorageFormat(StorageFormat.VIEW_STORAGE_FORMAT)) + .setColumns(ImmutableList.of(PARTITION_COLUMN)) + .setValues(Collections.singletonList("2019-07-23")) + .build()), + "ds=2019-08-23", + Optional.of( + Partition.builder() + .setCatalogName(Optional.empty()) + .setDatabaseName(SCHEMA_NAME) + .setTableName(TABLE_NAME) + .withStorage(storageBuilder -> + storageBuilder.setLocation(LOCATION) + .setStorageFormat(StorageFormat.VIEW_STORAGE_FORMAT)) + .setColumns(ImmutableList.of(PARTITION_COLUMN)) + .setValues(Collections.singletonList("2019-08-23")) + .build())); private final HudiPartitionManager hudiPartitionManager = new HudiPartitionManager(new TestingTypeManager()); - private final TestingExtendedHiveMetastore metastore = new TestingExtendedHiveMetastore(TABLE, PARTITIONS); + private final TestingExtendedHiveMetastore metastore = new TestingExtendedHiveMetastore(TABLE, PARTITION_MAP); @Test public void testParseValuesAndFilterPartition() diff --git a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestingExtendedHiveMetastore.java b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestingExtendedHiveMetastore.java index 40f7fe9425475..ac34d1d70d37f 100644 --- a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestingExtendedHiveMetastore.java +++ b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestingExtendedHiveMetastore.java @@ -18,6 +18,7 @@ import com.facebook.presto.hive.PartitionNameWithVersion; import com.facebook.presto.hive.metastore.Column; import com.facebook.presto.hive.metastore.MetastoreContext; +import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.hive.metastore.UnimplementedHiveMetastore; @@ -26,15 +27,16 @@ import java.util.Optional; import static com.facebook.presto.hive.metastore.MetastoreUtil.getPartitionNamesWithEmptyVersion; +import static com.google.common.collect.ImmutableMap.toImmutableMap; import static java.util.Objects.requireNonNull; public class TestingExtendedHiveMetastore extends UnimplementedHiveMetastore { private final Table table; - private final List partitions; + private final Map> partitions; - public TestingExtendedHiveMetastore(Table table, List partitions) + public TestingExtendedHiveMetastore(Table table, Map> partitions) { this.table = requireNonNull(table, "table is null"); this.partitions = requireNonNull(partitions, "partitions is null"); @@ -53,6 +55,13 @@ public List getPartitionNamesByFilter( String tableName, Map partitionPredicates) { - return getPartitionNamesWithEmptyVersion(partitions); + return getPartitionNamesWithEmptyVersion(partitions.keySet()); + } + + @Override + public Map> getPartitionsByNames(MetastoreContext metastoreContext, String databaseName, String tableName, List partitionNames) + { + return partitionNames.stream().filter(partitionName -> partitions.containsKey(partitionName.getPartitionName())) + .collect(toImmutableMap(PartitionNameWithVersion::getPartitionName, partitionName -> partitions.get(partitionName.getPartitionName()))); } } From 0844719497b8126ff470ff5f1d7aef8929fcac70 Mon Sep 17 00:00:00 2001 From: Vamsi Date: Fri, 22 Aug 2025 19:36:15 +0530 Subject: [PATCH 4/8] Minor fix --- .../java/com/facebook/presto/hudi/HudiTableHandle.java | 7 ++----- .../com/facebook/presto/hudi/TestHudiPartitionManager.java | 2 +- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java index 6b58408b09088..d8a21b27f37e3 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java @@ -33,7 +33,7 @@ public class HudiTableHandle private final String tableName; private final String path; private final HudiTableType hudiTableType; - private Optional
table; + private final transient Optional
table; @JsonCreator public HudiTableHandle( @@ -42,10 +42,7 @@ public HudiTableHandle( @JsonProperty("path") String path, @JsonProperty("tableType") HudiTableType hudiTableType) { - this.schemaName = requireNonNull(schemaName, "schemaName is null"); - this.tableName = requireNonNull(tableName, "tableName is null"); - this.path = requireNonNull(path, "path is null"); - this.hudiTableType = requireNonNull(hudiTableType, "tableType is null"); + this(Optional.empty(), schemaName, tableName, path, hudiTableType); } public HudiTableHandle( diff --git a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java index 1d354d8f2a653..86a5bf187fe70 100644 --- a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java +++ b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java @@ -127,7 +127,7 @@ public void testParseValuesAndFilterPartition() Optional.empty(), HudiColumnHandle.ColumnType.PARTITION_KEY), Domain.singleValue(VARCHAR, utf8Slice("2019-07-23")))); - HudiTableHandle tableHandle = new HudiTableHandle(Optional.of(TABLE), SCHEMA_NAME, TABLE_NAME, LOCATION, HudiTableType.COW); + HudiTableHandle tableHandle = new HudiTableHandle(SCHEMA_NAME, TABLE_NAME, LOCATION, HudiTableType.COW); Map actualPartitions = hudiPartitionManager.getEffectivePartitions( session, metastore, From be82ef1c8459fb7633a8f1847e9e9b5c8678760c Mon Sep 17 00:00:00 2001 From: Vamsi Date: Mon, 25 Aug 2025 17:44:38 +0530 Subject: [PATCH 5/8] clean up --- .../com/facebook/presto/hudi/HudiSplitManager.java | 1 - .../com/facebook/presto/hudi/HudiSplitSource.java | 3 --- .../presto/hudi/split/HudiBackgroundSplitLoader.java | 6 +----- .../hudi/split/HudiPartitionSplitGenerator.java | 12 ++---------- 4 files changed, 3 insertions(+), 19 deletions(-) diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java index 8228be8a45c86..e6e725aefcfcf 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java @@ -119,7 +119,6 @@ public ConnectorSplitSource getSplits( return new HudiSplitSource( session, - metastore, layout, fsView, partitions, diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java index 6d74c46d4583c..466410d6954c8 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java @@ -14,7 +14,6 @@ package com.facebook.presto.hudi; -import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.util.AsyncQueue; import com.facebook.presto.hudi.split.HudiBackgroundSplitLoader; @@ -44,7 +43,6 @@ public class HudiSplitSource public HudiSplitSource( ConnectorSession session, - ExtendedHiveMetastore metastore, HudiTableLayoutHandle layout, HoodieTableFileSystemView fsView, Map partitionMap, @@ -57,7 +55,6 @@ public HudiSplitSource( this.queue = new AsyncQueue<>(maxOutstandingSplits, asyncQueueExecutor); this.splitLoader = new HudiBackgroundSplitLoader( session, - metastore, splitGeneratorExecutorService, layout, fsView, diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java index b832271297dbd..0a228cd70c458 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java @@ -15,7 +15,6 @@ package com.facebook.presto.hudi.split; import com.facebook.airlift.log.Logger; -import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.util.AsyncQueue; import com.facebook.presto.hudi.HudiTableLayoutHandle; @@ -46,7 +45,6 @@ public class HudiBackgroundSplitLoader private static final Logger log = Logger.get(HudiBackgroundSplitLoader.class); private final ConnectorSession session; - private final ExtendedHiveMetastore metastore; private final HudiTableLayoutHandle layout; private final HoodieTableFileSystemView fsView; private final AsyncQueue asyncQueue; @@ -57,7 +55,6 @@ public class HudiBackgroundSplitLoader public HudiBackgroundSplitLoader( ConnectorSession session, - ExtendedHiveMetastore metastore, ExecutorService splitGeneratorExecutorService, HudiTableLayoutHandle layout, HoodieTableFileSystemView fsView, @@ -66,7 +63,6 @@ public HudiBackgroundSplitLoader( String latestInstant) { this.session = requireNonNull(session, "session is null"); - this.metastore = requireNonNull(metastore, "metastore is null"); this.layout = requireNonNull(layout, "layout is null"); this.fsView = requireNonNull(fsView, "fsView is null"); this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); @@ -88,7 +84,7 @@ public void run() // Start a number of partition split generators to generate the splits in parallel for (int i = 0; i < splitGeneratorNumThreads; i++) { HudiPartitionSplitGenerator generator = new HudiPartitionSplitGenerator( - session, metastore, layout, fsView, partitionMap, asyncQueue, concurrentPartitionQueue, latestInstant); + session, layout, fsView, partitionMap, asyncQueue, concurrentPartitionQueue, latestInstant); splitGeneratorList.add(generator); splitGeneratorFutures.add(splitGeneratorExecutorService.submit(generator)); } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java index f7d22bc1f1716..1fb3cc9227250 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java @@ -16,8 +16,6 @@ import com.facebook.airlift.log.Logger; import com.facebook.airlift.units.DataSize; -import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; -import com.facebook.presto.hive.metastore.MetastoreContext; import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.hive.util.AsyncQueue; @@ -51,7 +49,6 @@ import static com.facebook.presto.hive.metastore.MetastoreUtil.extractPartitionValues; import static com.facebook.presto.hudi.HudiErrorCode.HUDI_INVALID_METADATA; import static com.facebook.presto.hudi.HudiMetadata.fromDataColumns; -import static com.facebook.presto.hudi.HudiMetadata.toMetastoreContext; import static com.facebook.presto.hudi.HudiSessionProperties.getMinimumAssignedSplitWeight; import static com.facebook.presto.hudi.HudiSessionProperties.getStandardSplitWeightSize; import static com.facebook.presto.hudi.HudiSessionProperties.isSizeBasedSplitWeightsEnabled; @@ -70,8 +67,6 @@ public class HudiPartitionSplitGenerator { private static final Logger log = Logger.get(HudiPartitionSplitGenerator.class); - private final ExtendedHiveMetastore metastore; - private final MetastoreContext metastoreContext; private final HudiTableLayoutHandle layout; private final HudiTableHandle table; private final Path tablePath; @@ -84,7 +79,6 @@ public class HudiPartitionSplitGenerator public HudiPartitionSplitGenerator( ConnectorSession session, - ExtendedHiveMetastore metastore, HudiTableLayoutHandle layout, HoodieTableFileSystemView fsView, Map partitionMap, @@ -92,8 +86,6 @@ public HudiPartitionSplitGenerator( Queue concurrentPartitionQueue, String latestInstant) { - this.metastore = requireNonNull(metastore, "metastore is null"); - this.metastoreContext = toMetastoreContext(requireNonNull(session, "session is null")); this.layout = requireNonNull(layout, "layout is null"); this.table = layout.getTable(); this.tablePath = new Path(table.getPath()); @@ -120,7 +112,7 @@ public void run() private void generateSplitsFromPartition(String partitionName) { - HudiPartition hudiPartition = getHudiPartition(metastore, metastoreContext, layout, partitionName); + HudiPartition hudiPartition = getHudiPartition(layout, partitionName); Path partitionPath = new Path(hudiPartition.getStorage().getLocation()); String relativePartitionPath = FSUtils.getRelativePartitionPath(new StoragePath(tablePath.toUri()), new StoragePath(partitionPath.toUri())); Stream fileSlices = HudiTableType.MOR.equals(table.getTableType()) ? @@ -132,7 +124,7 @@ private void generateSplitsFromPartition(String partitionName) .forEach(asyncQueue::offer); } - private HudiPartition getHudiPartition(ExtendedHiveMetastore metastore, MetastoreContext context, HudiTableLayoutHandle tableLayout, String partitionName) + private HudiPartition getHudiPartition(HudiTableLayoutHandle tableLayout, String partitionName) { String databaseName = tableLayout.getTable().getSchemaName(); String tableName = tableLayout.getTable().getTableName(); From 2931f9e9152bc8241d6a7d352e563ab6a461f1b8 Mon Sep 17 00:00:00 2001 From: Vamsi Date: Fri, 22 Aug 2025 19:34:22 +0530 Subject: [PATCH 6/8] Lazy load metaclient and fsview --- .../facebook/presto/hudi/HudiMetadata.java | 43 +++++++++++- .../presto/hudi/HudiSplitManager.java | 68 +++---------------- .../facebook/presto/hudi/HudiSplitSource.java | 34 ++++++++-- .../facebook/presto/hudi/HudiTableHandle.java | 64 +++++++++++++++-- .../hudi/split/HudiBackgroundSplitLoader.java | 20 +++--- .../split/HudiPartitionSplitGenerator.java | 18 +++-- .../presto/hudi/TestHudiPartitionManager.java | 2 +- 7 files changed, 152 insertions(+), 97 deletions(-) diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java index fae49cf97b01d..0315b01dd0261 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java @@ -16,9 +16,11 @@ import com.facebook.airlift.log.Logger; import com.facebook.presto.common.type.TypeManager; +import com.facebook.presto.hive.HdfsContext; import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveColumnConverterProvider; import com.facebook.presto.hive.HiveType; +import com.facebook.presto.hive.filesystem.ExtendedFileSystem; import com.facebook.presto.hive.metastore.Column; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; import com.facebook.presto.hive.metastore.MetastoreContext; @@ -40,7 +42,14 @@ import com.facebook.presto.spi.connector.ConnectorMetadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.hadoop.fs.HadoopFSUtils; +import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.util.Lazy; + +import java.io.IOException; import java.util.List; import java.util.Map; import java.util.Optional; @@ -51,6 +60,7 @@ import static com.facebook.presto.hive.HiveColumnHandle.MAX_PARTITION_KEY_COLUMN_INDEX; import static com.facebook.presto.hudi.HudiColumnHandle.ColumnType.PARTITION_KEY; import static com.facebook.presto.hudi.HudiColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_FILESYSTEM_ERROR; import static com.facebook.presto.hudi.HudiErrorCode.HUDI_UNKNOWN_TABLE_TYPE; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.google.common.base.Preconditions.checkArgument; @@ -100,14 +110,41 @@ public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTable throw new PrestoException(HUDI_UNKNOWN_TABLE_TYPE, "Unknown table type " + inputFormat); } + String basePath = table.getStorage().getLocation(); + ExtendedFileSystem fs = getFileSystem(session, tableName, basePath); return new HudiTableHandle( - Optional.of(table), + table, + Lazy.lazily(() -> buildTableMetaClient(fs, basePath)), table.getDatabaseName(), table.getTableName(), - table.getStorage().getLocation(), + basePath, hudiTableType); } + public static HoodieTableMetaClient buildTableMetaClient( + ExtendedFileSystem fs, + String basePath) + { + StorageConfiguration conf = HadoopFSUtils.getStorageConfWithCopy(fs.getConf()); + return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build(); + } + + private ExtendedFileSystem getFileSystem(ConnectorSession session, SchemaTableName table, String basePath) + { + HdfsContext hdfsContext = new HdfsContext( + session, + table.getSchemaName(), + table.getTableName(), + basePath, + false); + try { + return hdfsEnvironment.getFileSystem(hdfsContext, new Path(basePath)); + } + catch (IOException e) { + throw new PrestoException(HUDI_FILESYSTEM_ERROR, "Could not open file system for " + table, e); + } + } + @Override public Optional getSystemTable(ConnectorSession session, SchemaTableName tableName) { diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java index 8228be8a45c86..27fc5bd1d763f 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java @@ -15,9 +15,7 @@ package com.facebook.presto.hudi; import com.facebook.airlift.log.Logger; -import com.facebook.presto.hive.HdfsContext; import com.facebook.presto.hive.HdfsEnvironment; -import com.facebook.presto.hive.filesystem.ExtendedFileSystem; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hudi.split.ForHudiBackgroundSplitLoader; @@ -26,34 +24,18 @@ import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSplitSource; import com.facebook.presto.spi.ConnectorTableLayoutHandle; -import com.facebook.presto.spi.FixedSplitSource; -import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; -import com.google.common.collect.ImmutableList; import jakarta.inject.Inject; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; -import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.util.Lazy; -import java.io.IOException; import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; -import static com.facebook.presto.hudi.HudiErrorCode.HUDI_FILESYSTEM_ERROR; import static com.facebook.presto.hudi.HudiSessionProperties.getMaxOutstandingSplits; -import static com.facebook.presto.hudi.HudiSessionProperties.isHudiMetadataTableEnabled; import static java.util.Objects.requireNonNull; -import static org.apache.hudi.common.table.view.FileSystemViewManager.createInMemoryFileSystemViewWithTimeline; public class HudiSplitManager implements ConnectorSplitManager @@ -95,54 +77,22 @@ public ConnectorSplitSource getSplits( HudiTableLayoutHandle layout = (HudiTableLayoutHandle) layoutHandle; HudiTableHandle table = layout.getTable(); - // Retrieve and prune partitions - HoodieTimer timer = HoodieTimer.start(); - Map partitions = hudiPartitionManager.getEffectivePartitions(session, metastore, table, layout.getTupleDomain()); - log.debug("Took %d ms to get %d partitions", timer.endTimer(), partitions.size()); - if (partitions.isEmpty()) { - return new FixedSplitSource(ImmutableList.of()); - } - - // Load Hudi metadata - ExtendedFileSystem fs = getFileSystem(session, table); - HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().enable(isHudiMetadataTableEnabled(session)).build(); - StorageConfiguration conf = HadoopFSUtils.getStorageConfWithCopy(fs.getConf()); - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(conf).setBasePath(table.getPath()).build(); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - String timestamp = timeline.lastInstant().map(HoodieInstant::requestedTime).orElse(null); - if (timestamp == null) { - // no completed instant for current table - return new FixedSplitSource(ImmutableList.of()); - } - HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(conf); - HoodieTableFileSystemView fsView = createInMemoryFileSystemViewWithTimeline(engineContext, metaClient, metadataConfig, timeline); + Lazy> lazyPartitionMap = Lazy.lazily(() -> { + // Retrieve and prune partitions + HoodieTimer timer = HoodieTimer.start(); + Map partitions = hudiPartitionManager.getEffectivePartitions(session, metastore, table, layout.getTupleDomain()); + log.debug("Took %d ms to get %d partitions", timer.endTimer(), partitions.size()); + return partitions; + }); return new HudiSplitSource( session, metastore, layout, - fsView, - partitions, - timestamp, + lazyPartitionMap, asyncQueueExecutor, splitLoaderExecutorService, splitGeneratorExecutorService, getMaxOutstandingSplits(session)); } - - private ExtendedFileSystem getFileSystem(ConnectorSession session, HudiTableHandle table) - { - HdfsContext hdfsContext = new HdfsContext( - session, - table.getSchemaName(), - table.getTableName(), - table.getPath(), - false); - try { - return hdfsEnvironment.getFileSystem(hdfsContext, new Path(table.getPath())); - } - catch (IOException e) { - throw new PrestoException(HUDI_FILESYSTEM_ERROR, "Could not open file system for " + table, e); - } - } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java index 6d74c46d4583c..d79df6c6e35e6 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java @@ -14,6 +14,7 @@ package com.facebook.presto.hudi; +import com.facebook.airlift.log.Logger; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.util.AsyncQueue; @@ -21,9 +22,16 @@ import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSplit; import com.facebook.presto.spi.ConnectorSplitSource; +import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.connector.ConnectorPartitionHandle; import com.google.common.util.concurrent.Futures; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.util.Lazy; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -33,11 +41,15 @@ import java.util.concurrent.TimeUnit; import static com.facebook.airlift.concurrent.MoreFutures.toCompletableFuture; +import static com.facebook.presto.hudi.HudiSessionProperties.isHudiMetadataTableEnabled; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static org.apache.hudi.common.table.view.FileSystemViewManager.createInMemoryFileSystemViewWithTimeline; public class HudiSplitSource implements ConnectorSplitSource { + private static final Logger log = Logger.get(HudiSplitManager.class); + private final AsyncQueue queue; private final HudiBackgroundSplitLoader splitLoader; private final ScheduledFuture splitLoaderFuture; @@ -46,24 +58,34 @@ public HudiSplitSource( ConnectorSession session, ExtendedHiveMetastore metastore, HudiTableLayoutHandle layout, - HoodieTableFileSystemView fsView, - Map partitionMap, - String latestInstant, + Lazy> lazyPartitionMap, ExecutorService asyncQueueExecutor, ScheduledExecutorService splitLoaderExecutorService, ExecutorService splitGeneratorExecutorService, int maxOutstandingSplits) { this.queue = new AsyncQueue<>(maxOutstandingSplits, asyncQueueExecutor); + + SchemaTableName schemaTableName = layout.getTable().getSchemaTableName(); + Lazy lazyFsView = Lazy.lazily(() -> { + HoodieTimer timer = HoodieTimer.start(); + HoodieTableMetaClient metaClient = layout.getTable().getMetaClient(); + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().enable(isHudiMetadataTableEnabled(session)).build(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getStorageConf()); + HoodieTableFileSystemView fsView = createInMemoryFileSystemViewWithTimeline(engineContext, metaClient, metadataConfig, timeline); + log.info("Created file system view of table %s in %s ms", schemaTableName, timer.endTimer()); + return fsView; + }); + this.splitLoader = new HudiBackgroundSplitLoader( session, metastore, splitGeneratorExecutorService, layout, - fsView, + lazyFsView, queue, - partitionMap, - latestInstant); + lazyPartitionMap); this.splitLoaderFuture = splitLoaderExecutorService.schedule( this.splitLoader, 0, TimeUnit.MILLISECONDS); } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java index 6b58408b09088..d486ddcda1e3f 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableHandle.java @@ -16,12 +16,17 @@ import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaTableName; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.util.Lazy; import java.util.Objects; import java.util.Optional; +import java.util.function.Supplier; import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; @@ -33,33 +38,64 @@ public class HudiTableHandle private final String tableName; private final String path; private final HudiTableType hudiTableType; - private Optional
table; + private final transient Optional
table; + private final transient Optional> lazyMetaClient; + private final transient Lazy lazyLatestCommitTime; @JsonCreator public HudiTableHandle( @JsonProperty("schemaName") String schemaName, @JsonProperty("tableName") String tableName, @JsonProperty("path") String path, - @JsonProperty("tableType") HudiTableType hudiTableType) + @JsonProperty("tableType") HudiTableType hudiTableType, + @JsonProperty("latestCommitTime") String latestCommitTime) { - this.schemaName = requireNonNull(schemaName, "schemaName is null"); - this.tableName = requireNonNull(tableName, "tableName is null"); - this.path = requireNonNull(path, "path is null"); - this.hudiTableType = requireNonNull(hudiTableType, "tableType is null"); + this(Optional.empty(), Optional.empty(), schemaName, tableName, path, hudiTableType, () -> latestCommitTime); } public HudiTableHandle( - Optional
table, + Table table, + Lazy lazyMetaClient, String schemaName, String tableName, String path, HudiTableType hudiTableType) + { + this( + Optional.of(table), + Optional.of(lazyMetaClient), + schemaName, + tableName, + path, + hudiTableType, + () -> lazyMetaClient + .get() + .getActiveTimeline() + .getCommitsTimeline() + .filterCompletedInstants() + .lastInstant() + .map(HoodieInstant::requestedTime) + .orElseThrow(() -> new PrestoException( + HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT, + "Table has no valid commits"))); + } + + HudiTableHandle( + Optional
table, + Optional> lazyMetaClient, + String schemaName, + String tableName, + String path, + HudiTableType hudiTableType, + Supplier latestCommitTimeSupplier) { this.table = requireNonNull(table, "table is null"); this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); this.path = requireNonNull(path, "path is null"); this.hudiTableType = requireNonNull(hudiTableType, "tableType is null"); + this.lazyMetaClient = requireNonNull(lazyMetaClient, "lazyMetaClient is null"); + this.lazyLatestCommitTime = Lazy.lazily(latestCommitTimeSupplier); } public Table getTable() @@ -70,6 +106,20 @@ public Table getTable() return table.get(); } + public HoodieTableMetaClient getMetaClient() + { + checkArgument(lazyMetaClient.isPresent(), + "getMetaClient() called on a table handle that has no Hudi meta-client; " + + "this is likely because it is called on the worker."); + return lazyMetaClient.get().get(); + } + + @JsonProperty + public String getLatestCommitTime() + { + return lazyLatestCommitTime.get(); + } + @JsonProperty public String getSchemaName() { diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java index b832271297dbd..fa4413cb5430a 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java @@ -24,6 +24,7 @@ import com.facebook.presto.spi.PrestoException; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.util.Lazy; import java.util.ArrayList; import java.util.List; @@ -48,10 +49,9 @@ public class HudiBackgroundSplitLoader private final ConnectorSession session; private final ExtendedHiveMetastore metastore; private final HudiTableLayoutHandle layout; - private final HoodieTableFileSystemView fsView; + private final Lazy lazyFsView; private final AsyncQueue asyncQueue; - private final Map partitionMap; - private final String latestInstant; + private final Lazy> lazyPartitionMap; private final int splitGeneratorNumThreads; private final ExecutorService splitGeneratorExecutorService; @@ -60,18 +60,16 @@ public HudiBackgroundSplitLoader( ExtendedHiveMetastore metastore, ExecutorService splitGeneratorExecutorService, HudiTableLayoutHandle layout, - HoodieTableFileSystemView fsView, + Lazy lazyFsView, AsyncQueue asyncQueue, - Map partitionMap, - String latestInstant) + Lazy> lazyPartitionMap) { this.session = requireNonNull(session, "session is null"); this.metastore = requireNonNull(metastore, "metastore is null"); this.layout = requireNonNull(layout, "layout is null"); - this.fsView = requireNonNull(fsView, "fsView is null"); + this.lazyFsView = requireNonNull(lazyFsView, "fsView is null"); this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); - this.partitionMap = requireNonNull(partitionMap, "partitions is null"); - this.latestInstant = requireNonNull(latestInstant, "latestInstant is null"); + this.lazyPartitionMap = requireNonNull(lazyPartitionMap, "partitions is null"); this.splitGeneratorNumThreads = getSplitGeneratorParallelism(session); this.splitGeneratorExecutorService = requireNonNull(splitGeneratorExecutorService, "splitGeneratorExecutorService is null"); @@ -83,12 +81,12 @@ public void run() HoodieTimer timer = HoodieTimer.start(); List splitGeneratorList = new ArrayList<>(); List splitGeneratorFutures = new ArrayList<>(); - ConcurrentLinkedQueue concurrentPartitionQueue = new ConcurrentLinkedQueue<>(partitionMap.keySet()); + ConcurrentLinkedQueue concurrentPartitionQueue = new ConcurrentLinkedQueue<>(lazyPartitionMap.get().keySet()); // Start a number of partition split generators to generate the splits in parallel for (int i = 0; i < splitGeneratorNumThreads; i++) { HudiPartitionSplitGenerator generator = new HudiPartitionSplitGenerator( - session, metastore, layout, fsView, partitionMap, asyncQueue, concurrentPartitionQueue, latestInstant); + session, metastore, layout, lazyFsView, lazyPartitionMap.get(), asyncQueue, concurrentPartitionQueue); splitGeneratorList.add(generator); splitGeneratorFutures.add(splitGeneratorExecutorService.submit(generator)); } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java index f7d22bc1f1716..ef0c45c8a91e6 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java @@ -41,6 +41,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.util.Lazy; import java.util.List; import java.util.Map; @@ -75,10 +76,9 @@ public class HudiPartitionSplitGenerator private final HudiTableLayoutHandle layout; private final HudiTableHandle table; private final Path tablePath; - private final HoodieTableFileSystemView fsView; + private final Lazy lazyFsView; private final AsyncQueue asyncQueue; private final Queue concurrentPartitionQueue; - private final String latestInstant; private final HudiSplitWeightProvider splitWeightProvider; private final Map partitionMap; @@ -86,22 +86,20 @@ public HudiPartitionSplitGenerator( ConnectorSession session, ExtendedHiveMetastore metastore, HudiTableLayoutHandle layout, - HoodieTableFileSystemView fsView, + Lazy lazyFsView, Map partitionMap, AsyncQueue asyncQueue, - Queue concurrentPartitionQueue, - String latestInstant) + Queue concurrentPartitionQueue) { this.metastore = requireNonNull(metastore, "metastore is null"); this.metastoreContext = toMetastoreContext(requireNonNull(session, "session is null")); this.layout = requireNonNull(layout, "layout is null"); this.table = layout.getTable(); this.tablePath = new Path(table.getPath()); - this.fsView = requireNonNull(fsView, "fsView is null"); + this.lazyFsView = requireNonNull(lazyFsView, "fsView is null"); this.partitionMap = requireNonNull(partitionMap, "partitionMap is null"); this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); this.concurrentPartitionQueue = requireNonNull(concurrentPartitionQueue, "concurrentPartitionQueue is null"); - this.latestInstant = requireNonNull(latestInstant, "latestInstant is null"); this.splitWeightProvider = createSplitWeightProvider(requireNonNull(session, "session is null")); } @@ -124,9 +122,9 @@ private void generateSplitsFromPartition(String partitionName) Path partitionPath = new Path(hudiPartition.getStorage().getLocation()); String relativePartitionPath = FSUtils.getRelativePartitionPath(new StoragePath(tablePath.toUri()), new StoragePath(partitionPath.toUri())); Stream fileSlices = HudiTableType.MOR.equals(table.getTableType()) ? - fsView.getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, latestInstant) : - fsView.getLatestFileSlicesBeforeOrOn(relativePartitionPath, latestInstant, false); - fileSlices.map(fileSlice -> createHudiSplit(table, fileSlice, latestInstant, hudiPartition, splitWeightProvider)) + lazyFsView.get().getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, table.getLatestCommitTime()) : + lazyFsView.get().getLatestFileSlicesBeforeOrOn(relativePartitionPath, table.getLatestCommitTime(), false); + fileSlices.map(fileSlice -> createHudiSplit(table, fileSlice, table.getLatestCommitTime(), hudiPartition, splitWeightProvider)) .filter(Optional::isPresent) .map(Optional::get) .forEach(asyncQueue::offer); diff --git a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java index 1d354d8f2a653..1cd7fca6cbb30 100644 --- a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java +++ b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiPartitionManager.java @@ -127,7 +127,7 @@ public void testParseValuesAndFilterPartition() Optional.empty(), HudiColumnHandle.ColumnType.PARTITION_KEY), Domain.singleValue(VARCHAR, utf8Slice("2019-07-23")))); - HudiTableHandle tableHandle = new HudiTableHandle(Optional.of(TABLE), SCHEMA_NAME, TABLE_NAME, LOCATION, HudiTableType.COW); + HudiTableHandle tableHandle = new HudiTableHandle(SCHEMA_NAME, TABLE_NAME, LOCATION, HudiTableType.COW, "101"); Map actualPartitions = hudiPartitionManager.getEffectivePartitions( session, metastore, From fbbb1dbc929b9f17b008f1a09207a73d8665eb05 Mon Sep 17 00:00:00 2001 From: Vamsi Karnika Date: Tue, 2 Sep 2025 22:03:17 +0530 Subject: [PATCH 7/8] Add Support for Column Stats --- .../com/facebook/presto/hudi/HudiConfig.java | 32 ++ .../presto/hudi/HudiSessionProperties.java | 25 ++ .../query/index/HudiBaseIndexSupport.java | 53 +++ .../index/HudiColumnStatsIndexSupport.java | 324 ++++++++++++++++++ .../hudi/query/index/HudiIndexSupport.java | 14 + .../query/index/HudiNoOpIndexSupport.java | 27 ++ .../hudi/query/index/IndexSupportFactory.java | 110 ++++++ .../presto/hudi/util/TupleDomainUtils.java | 38 ++ 8 files changed, 623 insertions(+) create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiBaseIndexSupport.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiColumnStatsIndexSupport.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiIndexSupport.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiNoOpIndexSupport.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/IndexSupportFactory.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/util/TupleDomainUtils.java diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiConfig.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiConfig.java index 98e5154a108c3..a4c32094277ec 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiConfig.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiConfig.java @@ -17,12 +17,14 @@ import com.facebook.airlift.configuration.Config; import com.facebook.airlift.configuration.ConfigDescription; import com.facebook.airlift.units.DataSize; +import com.facebook.airlift.units.Duration; import jakarta.validation.constraints.DecimalMax; import jakarta.validation.constraints.DecimalMin; import jakarta.validation.constraints.Min; import jakarta.validation.constraints.NotNull; import static com.facebook.airlift.units.DataSize.Unit.MEGABYTE; +import static java.util.concurrent.TimeUnit.SECONDS; public class HudiConfig { @@ -34,6 +36,9 @@ public class HudiConfig private int splitLoaderParallelism = 4; private int splitGeneratorParallelism = 4; + private boolean isColumnStatsIndexEnabled = true; + private Duration columnStatsWaitTimeout = new Duration(1, SECONDS); + public boolean isMetadataTableEnabled() { return metadataTableEnabled; @@ -132,4 +137,31 @@ public HudiConfig setSplitLoaderParallelism(int splitLoaderParallelism) this.splitLoaderParallelism = splitLoaderParallelism; return this; } + + @Config("hudi.index.column-stats-index-enabled") + @ConfigDescription("Internal configuration to control whether column stats index is enabled for debugging/testing.") + public HudiConfig setColumnStatsIndexEnabled(boolean isColumnStatsIndexEnabled) + { + this.isColumnStatsIndexEnabled = isColumnStatsIndexEnabled; + return this; + } + + public boolean isColumnStatsIndexEnabled() + { + return isColumnStatsIndexEnabled; + } + + @Config("hudi.index.column-stats.wait-timeout") + @ConfigDescription("Maximum timeout to wait for loading column stats, e.g. 1000ms, 20s") + public HudiConfig setColumnStatsWaitTimeout(Duration columnStatusWaitTimeout) + { + this.columnStatsWaitTimeout = columnStatusWaitTimeout; + return this; + } + + @NotNull + public Duration getColumnStatsWaitTimeout() + { + return columnStatsWaitTimeout; + } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSessionProperties.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSessionProperties.java index b559e55c1cee9..07251f4da913e 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSessionProperties.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSessionProperties.java @@ -15,6 +15,7 @@ package com.facebook.presto.hudi; import com.facebook.airlift.units.DataSize; +import com.facebook.airlift.units.Duration; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.session.PropertyMetadata; @@ -28,6 +29,7 @@ import static com.facebook.presto.spi.StandardErrorCode.INVALID_SESSION_PROPERTY; import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty; import static com.facebook.presto.spi.session.PropertyMetadata.dataSizeProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.durationProperty; import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty; import static java.lang.String.format; @@ -42,6 +44,9 @@ public class HudiSessionProperties private static final String MAX_OUTSTANDING_SPLITS = "max_outstanding_splits"; private static final String SPLIT_GENERATOR_PARALLELISM = "split_generator_parallelism"; + static final String COLUMN_STATS_INDEX_ENABLED = "column_stats_index_enabled"; + static final String COLUMN_STATS_WAIT_TIMEOUT = "column_stats_wait_timeout"; + @Inject public HudiSessionProperties(HudiConfig hudiConfig) { @@ -91,6 +96,16 @@ public HudiSessionProperties(HudiConfig hudiConfig) SPLIT_GENERATOR_PARALLELISM, "Number of threads used to generate splits from partitions", hudiConfig.getSplitGeneratorParallelism(), + false), + booleanProperty( + COLUMN_STATS_INDEX_ENABLED, + "Enable column stats index for file skipping", + hudiConfig.isColumnStatsIndexEnabled(), + true), + durationProperty( + COLUMN_STATS_WAIT_TIMEOUT, + "Maximum timeout to wait for loading column stats", + hudiConfig.getColumnStatsWaitTimeout(), false)); } @@ -128,4 +143,14 @@ public static int getSplitGeneratorParallelism(ConnectorSession session) { return session.getProperty(SPLIT_GENERATOR_PARALLELISM, Integer.class); } + + public static boolean isColumnStatsIndexEnabled(ConnectorSession session) + { + return session.getProperty(COLUMN_STATS_INDEX_ENABLED, Boolean.class); + } + + public static Duration getColumnStatsWaitTimeout(ConnectorSession session) + { + return session.getProperty(COLUMN_STATS_WAIT_TIMEOUT, Duration.class); + } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiBaseIndexSupport.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiBaseIndexSupport.java new file mode 100644 index 0000000000000..3c2738996fc91 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiBaseIndexSupport.java @@ -0,0 +1,53 @@ +package com.facebook.presto.hudi.query.index; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.spi.SchemaTableName; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public abstract class HudiBaseIndexSupport + implements HudiIndexSupport +{ + private final Logger log; + protected final SchemaTableName schemaTableName; + protected final Lazy lazyMetaClient; + + public HudiBaseIndexSupport(Logger log, SchemaTableName schemaTableName, Lazy lazyMetaClient) + { + this.log = requireNonNull(log, "log is null"); + this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); + this.lazyMetaClient = requireNonNull(lazyMetaClient, "metaClient is null"); + } + + public void printDebugMessage(Map> candidateFileSlices, Map> inputFileSlices, long lookupDurationMs) + { + if (log.isDebugEnabled()) { + int candidateFileSize = candidateFileSlices.values().stream().mapToInt(List::size).sum(); + int totalFiles = inputFileSlices.values().stream().mapToInt(List::size).sum(); + double skippingPercent = totalFiles == 0 ? 0.0d : (totalFiles - candidateFileSize) / (totalFiles * 1.0d); + + log.info("Total files: %s; files after data skipping: %s; skipping percent %s; time taken: %s ms; table name: %s", + totalFiles, + candidateFileSize, + skippingPercent, + lookupDurationMs, + schemaTableName); + } + } + + protected Map getAllIndexDefinitions() + { + if (lazyMetaClient.get().getIndexMetadata().isEmpty()) { + return Map.of(); + } + + return lazyMetaClient.get().getIndexMetadata().get().getIndexDefinitions(); + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiColumnStatsIndexSupport.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiColumnStatsIndexSupport.java new file mode 100644 index 0000000000000..b702772e20142 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiColumnStatsIndexSupport.java @@ -0,0 +1,324 @@ +package com.facebook.presto.hudi.query.index; + +import com.facebook.airlift.log.Logger; +import com.facebook.airlift.units.Duration; +import com.facebook.presto.common.predicate.Domain; +import com.facebook.presto.common.predicate.Range; +import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.common.predicate.ValueSet; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.VarcharType; +import com.facebook.presto.hudi.util.TupleDomainUtils; +import com.facebook.presto.parquet.predicate.TupleDomainParquetPredicate; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.hash.ColumnIndexID; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.util.Lazy; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.DateType.DATE; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.RealType.REAL; +import static com.facebook.presto.common.type.SmallintType.SMALLINT; +import static com.facebook.presto.common.type.TinyintType.TINYINT; +import static com.facebook.presto.hudi.HudiSessionProperties.getColumnStatsWaitTimeout; +import static com.facebook.presto.parquet.predicate.PredicateUtils.isStatisticsOverflow; +import static java.lang.Float.floatToRawIntBits; + +public class HudiColumnStatsIndexSupport + extends HudiBaseIndexSupport +{ + private static final Logger log = Logger.get(HudiColumnStatsIndexSupport.class); + // file name -> column name -> domain with column stats + private final CompletableFuture>>> domainsWithStatsFuture; + protected final TupleDomain regularColumnPredicates; + private final List regularColumns; + private final Duration columnStatsWaitTimeout; + private final long futureStartTimeMs; + + public HudiColumnStatsIndexSupport(ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + this(log, session, schemaTableName, lazyMetaClient, lazyTableMetadata, regularColumnPredicates); + } + + public HudiColumnStatsIndexSupport(Logger log, ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + super(log, schemaTableName, lazyMetaClient); + this.columnStatsWaitTimeout = getColumnStatsWaitTimeout(session); + this.regularColumnPredicates = regularColumnPredicates; + this.regularColumns = regularColumnPredicates.getDomains() + .map(domains -> new ArrayList<>(domains.keySet())) + .orElse(new ArrayList<>()); + if (regularColumnPredicates.isAll() || regularColumnPredicates.getDomains().isEmpty()) { + this.domainsWithStatsFuture = CompletableFuture.completedFuture(Optional.empty()); + } + else { + // Get filter columns + List encodedTargetColumnNames = regularColumns + .stream() + .map(col -> new ColumnIndexID(col).asBase64EncodedString()).collect(Collectors.toList()); + + Map columnTypes = regularColumnPredicates.getDomains().get().entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getType())); + + domainsWithStatsFuture = CompletableFuture.supplyAsync(() -> { + HoodieTimer timer = HoodieTimer.start(); + if (!lazyMetaClient.get().getTableConfig().getMetadataPartitions() + .contains(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS)) { + return Optional.empty(); + } + + Map> domainsWithStats = + lazyTableMetadata.get().getRecordsByKeyPrefixes(encodedTargetColumnNames, + HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS, true) + .collectAsList() + .stream() + .filter(f -> f.getData().getColumnStatMetadata().isPresent()) + .map(f -> f.getData().getColumnStatMetadata().get()) + .collect(Collectors.groupingBy( + HoodieMetadataColumnStats::getFileName, + Collectors.toMap( + HoodieMetadataColumnStats::getColumnName, + // Pre-compute the Domain object for each HoodieMetadataColumnStats + stats -> getDomainFromColumnStats(stats.getColumnName(), columnTypes.get(stats.getColumnName()), stats)))); + + log.debug("Column stats lookup took %s ms and identified %d relevant file IDs.", timer.endTimer(), domainsWithStats.size()); + + return Optional.of(domainsWithStats); + }); + } + this.futureStartTimeMs = System.currentTimeMillis(); + } + + @Override + public boolean shouldSkipFileSlice(FileSlice slice) + { + try { + if (domainsWithStatsFuture.isDone()) { + Optional>> domainsWithStatsOpt = domainsWithStatsFuture.get(); + return domainsWithStatsOpt + .map(domainsWithStats -> shouldSkipFileSlice(slice, domainsWithStats, regularColumnPredicates, regularColumns)) + .orElse(false); + } + + long elapsedMs = System.currentTimeMillis() - futureStartTimeMs; + if (elapsedMs > columnStatsWaitTimeout.toMillis()) { + // Took too long; skip decision + return false; + } + + // If still within the timeout window, wait up to the remaining time + long remainingMs = Math.max(0, columnStatsWaitTimeout.toMillis() - elapsedMs); + Optional>> statsOpt = + domainsWithStatsFuture.get(remainingMs, TimeUnit.MILLISECONDS); + + return statsOpt + .map(stats -> shouldSkipFileSlice(slice, stats, regularColumnPredicates, regularColumns)) + .orElse(false); + } + catch (TimeoutException | InterruptedException | ExecutionException e) { + return false; + } + } + + @Override + public boolean canApply(TupleDomain tupleDomain) + { + boolean isIndexSupported = isIndexSupportAvailable(); + // indexDefinition is only available after table version EIGHT + // For tables that have versions < EIGHT, column stats index is available as long as partition in metadata is available + if (!isIndexSupported || lazyMetaClient.get().getTableConfig().getTableVersion().lesserThan(HoodieTableVersion.EIGHT)) { + log.debug("Column Stats Index partition is not enabled in metadata."); + return isIndexSupported; + } + + Map indexDefinitions = getAllIndexDefinitions(); + HoodieIndexDefinition colStatsDefinition = indexDefinitions.get(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS); + if (colStatsDefinition == null || colStatsDefinition.getSourceFields() == null || colStatsDefinition.getSourceFields().isEmpty()) { + log.warn("Column stats index definition is missing or has no source fields defined"); + return false; + } + + // Optimization applied: Only consider applicable if predicates reference indexed columns + List sourceFields = colStatsDefinition.getSourceFields(); + boolean applicable = TupleDomainUtils.areSomeFieldsReferenced(tupleDomain, sourceFields); + + if (applicable) { + log.debug("Column Stats Index is available and applicable (predicates reference indexed columns)."); + } + else { + log.debug("Column Stats Index is available, but predicates do not reference any indexed columns."); + } + return applicable; + } + + public boolean isIndexSupportAvailable() + { + return lazyMetaClient.get().getTableConfig().getMetadataPartitions() + .contains(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS); + } + + // TODO: Move helper functions below to TupleDomain/DomainUtils + private static boolean shouldSkipFileSlice( + FileSlice fileSlice, + Map> domainsWithStats, + TupleDomain regularColumnPredicates, + List regularColumns) + { + List filesToLookUp = new ArrayList<>(); + fileSlice.getBaseFile() + .map(BaseFile::getFileName) + .ifPresent(filesToLookUp::add); + + if (fileSlice.hasLogFiles()) { + fileSlice.getLogFiles().forEach(logFile -> filesToLookUp.add(logFile.getFileName())); + } + + // if any log or base file in the file slice matches the predicate, all files in the file slice needs to be read + return filesToLookUp.stream().allMatch(fileName -> { + // If no stats exist for this specific file, we cannot prune it. + if (!domainsWithStats.containsKey(fileName)) { + return false; + } + Map fileDomainsWithStats = domainsWithStats.get(fileName); + return !evaluateStatisticPredicate(regularColumnPredicates, fileDomainsWithStats, regularColumns); + }); + } + + protected static boolean evaluateStatisticPredicate( + TupleDomain regularColumnPredicates, + Map domainsWithStats, + List regularColumns) + { + if (regularColumnPredicates.isNone() || !regularColumnPredicates.getDomains().isPresent()) { + return true; + } + for (String regularColumn : regularColumns) { + Domain columnPredicate = regularColumnPredicates.getDomains().get().get(regularColumn); + Optional currentColumnStats = Optional.ofNullable(domainsWithStats.get(regularColumn)); + if (currentColumnStats.isEmpty()) { + // No stats for column + } + else { + Domain domain = currentColumnStats.get(); + if (columnPredicate.intersect(domain).isNone()) { + return false; + } + } + } + return true; + } + + static Domain getDomainFromColumnStats(String colName, Type type, HoodieMetadataColumnStats statistics) + { + if (statistics == null) { + return Domain.all(type); + } + boolean hasNullValue = statistics.getNullCount() != 0L; + boolean hasNonNullValue = statistics.getValueCount() - statistics.getNullCount() > 0; + if (!hasNonNullValue || statistics.getMaxValue() == null || statistics.getMinValue() == null) { + return Domain.create(ValueSet.all(type), hasNullValue); + } + if (!(statistics.getMinValue() instanceof GenericRecord) || + !(statistics.getMaxValue() instanceof GenericRecord)) { + return Domain.all(type); + } + return getDomainFromColumnStats(colName, type, ((GenericRecord) statistics.getMinValue()).get(0), + ((GenericRecord) statistics.getMaxValue()).get(0), hasNullValue); + } + + /** + * Get a domain for the ranges defined by each pair of elements from {@code minimums} and {@code maximums}. + * Both arrays must have the same length. + */ + private static Domain getDomainFromColumnStats(String colName, Type type, Object minimum, Object maximum, boolean hasNullValue) + { + try { + if (type.equals(BOOLEAN)) { + boolean hasTrueValue = (boolean) minimum || (boolean) maximum; + boolean hasFalseValue = !(boolean) minimum || !(boolean) maximum; + if (hasTrueValue && hasFalseValue) { + return Domain.all(type); + } + if (hasTrueValue) { + return Domain.create(ValueSet.of(type, true), hasNullValue); + } + if (hasFalseValue) { + return Domain.create(ValueSet.of(type, false), hasNullValue); + } + // No other case, since all null case is handled earlier. + } + + if ((type.equals(BIGINT) || type.equals(TINYINT) || type.equals(SMALLINT) + || type.equals(INTEGER) || type.equals(DATE))) { + long minValue = TupleDomainParquetPredicate.asLong(minimum); + long maxValue = TupleDomainParquetPredicate.asLong(maximum); + if (isStatisticsOverflow(type, minValue, maxValue)) { + return Domain.create(ValueSet.all(type), hasNullValue); + } + return ofMinMax(type, minValue, maxValue, hasNullValue); + } + + if (type.equals(REAL)) { + Float minValue = (Float) minimum; + Float maxValue = (Float) maximum; + if (minValue.isNaN() || maxValue.isNaN()) { + return Domain.create(ValueSet.all(type), hasNullValue); + } + return ofMinMax(type, (long) floatToRawIntBits(minValue), (long) floatToRawIntBits(maxValue), hasNullValue); + } + + if (type.equals(DOUBLE)) { + Double minValue = (Double) minimum; + Double maxValue = (Double) maximum; + if (minValue.isNaN() || maxValue.isNaN()) { + return Domain.create(ValueSet.all(type), hasNullValue); + } + return ofMinMax(type, minValue, maxValue, hasNullValue); + } + + if (type.equals(VarcharType.VARCHAR)) { + Slice min = Slices.utf8Slice((String) minimum); + Slice max = Slices.utf8Slice((String) maximum); + return ofMinMax(type, min, max, hasNullValue); + } + return Domain.create(ValueSet.all(type), hasNullValue); + } + catch (Exception e) { + log.warn("failed to create Domain for column: %s which type is: %s", colName, type.toString()); + return Domain.create(ValueSet.all(type), hasNullValue); + } + } + + private static Domain ofMinMax(Type type, Object min, Object max, boolean hasNullValue) + { + Range range = Range.range(type, min, true, max, true); + ValueSet vs = ValueSet.ofRanges(ImmutableList.of(range)); + return Domain.create(vs, hasNullValue); + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiIndexSupport.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiIndexSupport.java new file mode 100644 index 0000000000000..2fc1c9c6a4657 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiIndexSupport.java @@ -0,0 +1,14 @@ +package com.facebook.presto.hudi.query.index; + +import com.facebook.presto.common.predicate.TupleDomain; +import org.apache.hudi.common.model.FileSlice; + +public interface HudiIndexSupport +{ + boolean canApply(TupleDomain tupleDomain); + + default boolean shouldSkipFileSlice(FileSlice slice) + { + return false; + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiNoOpIndexSupport.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiNoOpIndexSupport.java new file mode 100644 index 0000000000000..af059c45bb5b4 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiNoOpIndexSupport.java @@ -0,0 +1,27 @@ +package com.facebook.presto.hudi.query.index; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.spi.SchemaTableName; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.util.Lazy; + +/** + * Noop index support to ensure that MDT enabled split generation is entered. + */ +public class HudiNoOpIndexSupport + extends HudiBaseIndexSupport +{ + private static final Logger log = Logger.get(HudiNoOpIndexSupport.class); + + public HudiNoOpIndexSupport(SchemaTableName schemaTableName, Lazy lazyMetaClient) + { + super(log, schemaTableName, lazyMetaClient); + } + + @Override + public boolean canApply(TupleDomain tupleDomain) + { + return true; + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/IndexSupportFactory.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/IndexSupportFactory.java new file mode 100644 index 0000000000000..6daee2023df86 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/IndexSupportFactory.java @@ -0,0 +1,110 @@ +package com.facebook.presto.hudi.query.index; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.hive.HiveColumnHandle; +import com.facebook.presto.hudi.HudiTableHandle; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.SchemaTableName; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.util.Lazy; + +import java.util.List; +import java.util.Optional; +import java.util.function.Supplier; + +import static com.facebook.presto.hudi.HudiSessionProperties.isColumnStatsIndexEnabled; +import static com.facebook.presto.hudi.HudiSessionProperties.isNoOpIndexEnabled; +import static java.util.Objects.requireNonNull; + +/** + * Factory to create the appropriate HudiIndexSupport strategy based on: + * 1. Query predicates + * 2. Available table indexes + * 3. Configuration flags + */ +public class IndexSupportFactory +{ + private static final Logger log = Logger.get(IndexSupportFactory.class); + + private IndexSupportFactory() {} + + /** + * Creates the most suitable HudiIndexSupport strategy, considering configuration. + * Uses Supplier-based lazy instantiation combined with config checks. + * + * @param hudiTableHandle The hudi table handle + * @param lazyMetaClient The Hudi table metadata client that is lazily instantiated. + * @param tupleDomain The query predicates. + * @param session Session containing session properties, which is required to control index behaviours for testing/debugging + * @return An Optional containing the chosen HudiIndexSupport strategy, or empty if none are applicable or enabled. + */ + public static Optional createIndexSupport( + HudiTableHandle hudiTableHandle, + Lazy lazyMetaClient, + Lazy lazyTableMetadata, + TupleDomain tupleDomain, + ConnectorSession session) + { + TupleDomain transformedTupleDomain = tupleDomain.transform(HiveColumnHandle::getName);; + SchemaTableName schemaTableName = hudiTableHandle.getSchemaTableName(); + // Define strategies as Suppliers paired with their config (isEnabled) flag + // IMPORTANT: Order of strategy here determines which index implementation is preferred first + List strategyProviders = List.of( + new StrategyProvider( + () -> isColumnStatsIndexEnabled(session), + () -> new HudiColumnStatsIndexSupport(session, schemaTableName, lazyMetaClient, lazyTableMetadata, transformedTupleDomain)), + new StrategyProvider( + () -> isNoOpIndexEnabled(session), + () -> new HudiNoOpIndexSupport(schemaTableName, lazyMetaClient))); + + for (StrategyProvider provider : strategyProviders) { + // Check if the strategy is enabled via config before instantiating + if (provider.isEnabled()) { + HudiIndexSupport strategy = provider.getStrategy(); + String strategyName = strategy.getClass().getSimpleName(); // Get name for logging + + // Check if the instantiated strategy is applicable + if (strategy.canApply(transformedTupleDomain)) { + log.debug(String.format("Selected %s strategy (Enabled & Applicable).", strategyName)); + return Optional.of(strategy); + } + else { + log.debug(String.format("%s is enabled but not applicable for this query.", strategyName)); + // Strategy object becomes eligible for GC here, acceptable penalty as the object is lightweight + } + } + else { + log.debug(String.format("Strategy associated with supplier %s is disabled by configuration.", provider.supplier.get().getClass().getSimpleName())); + } + } + + log.debug("No suitable and enabled index support strategy found to be applicable."); + return Optional.empty(); + } + /** + * Helper class to pair the configuration check with the strategy supplier to allow for lazy initialization. + */ + private static class StrategyProvider + { + private final Supplier isEnabled; + private final Supplier supplier; + + StrategyProvider(Supplier isEnabled, Supplier supplier) + { + this.isEnabled = requireNonNull(isEnabled); + this.supplier = requireNonNull(supplier); + } + + boolean isEnabled() + { + return isEnabled.get(); + } + + HudiIndexSupport getStrategy() + { + return supplier.get(); + } + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/util/TupleDomainUtils.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/util/TupleDomainUtils.java new file mode 100644 index 0000000000000..1361aa7637e40 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/util/TupleDomainUtils.java @@ -0,0 +1,38 @@ +package com.facebook.presto.hudi.util; + +import com.facebook.presto.common.predicate.TupleDomain; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class TupleDomainUtils +{ + // Utility classes should not have a public or default constructor. + private TupleDomainUtils() {} + + /** + * Get all columns that are referenced in the provided tupleDomain predicates. + */ + public static List getReferencedColumns(TupleDomain tupleDomain) + { + if (tupleDomain.getDomains().isEmpty()) { + return List.of(); + } + return tupleDomain.getDomains().get().keySet().stream().toList(); + } + + /** + * Check if at least one of the provided source field is referenced in the tupleDomain predicates. + */ + public static boolean areSomeFieldsReferenced(TupleDomain tupleDomain, List sourceFields) + { + Set referenceColSet = new HashSet<>(TupleDomainUtils.getReferencedColumns(tupleDomain)); + for (String sourceField : sourceFields) { + if (referenceColSet.contains(sourceField)) { + return true; + } + } + return false; + } +} \ No newline at end of file From 4ccbe2871092445eebdb45bdb61133069e04a836 Mon Sep 17 00:00:00 2001 From: Vamsi Karnika Date: Fri, 5 Sep 2025 00:23:40 +0530 Subject: [PATCH 8/8] Uber PR --- .../com/facebook/presto/hudi/HudiConfig.java | 101 ++ .../facebook/presto/hudi/HudiMetadata.java | 30 +- .../facebook/presto/hudi/HudiPartition.java | 13 +- .../presto/hudi/HudiPartitionManager.java | 11 +- .../facebook/presto/hudi/HudiPredicates.java | 67 + .../presto/hudi/HudiSessionProperties.java | 84 +- .../presto/hudi/HudiSplitManager.java | 8 +- .../facebook/presto/hudi/HudiSplitSource.java | 52 +- .../presto/hudi/HudiTableLayoutHandle.java | 98 +- .../HiveStylePartitionValueExtractor.java | 24 + .../MultiPartKeysValueExtractor.java | 32 + .../partition/NonPartitionedExtractor.java | 15 + .../partition/PartitionValueExtractor.java | 16 + .../SinglePartPartitionValueExtractor.java | 19 + .../hudi/query/HudiDirectoryLister.java | 13 + .../query/HudiSnapshotDirectoryLister.java | 88 ++ .../hudi/query/index/HudiDirectoryLister.java | 2 + .../index/HudiPartitionStatsIndexSupport.java | 138 ++ .../index/HudiRecordLevelIndexSupport.java | 317 ++++ .../index/HudiSecondaryIndexSupport.java | 221 +++ .../hudi/query/index/IndexSupportFactory.java | 52 +- .../hudi/split/HudiBackgroundSplitLoader.java | 327 +++- .../split/HudiPartitionSplitGenerator.java | 96 +- .../facebook/presto/hudi/util/HudiUtil.java | 121 ++ .../presto/hudi/util/TupleDomainUtils.java | 211 +++ .../presto/hudi/TestHudiSmokeTest.java | 1390 +++++++++++++++++ .../hudi/testing/HudiTablesInitializer.java | 10 + .../ResourceHudiTablesInitializer.java | 768 +++++++++ 28 files changed, 4168 insertions(+), 156 deletions(-) create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPredicates.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/partition/HiveStylePartitionValueExtractor.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/partition/MultiPartKeysValueExtractor.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/partition/NonPartitionedExtractor.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/partition/PartitionValueExtractor.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/partition/SinglePartPartitionValueExtractor.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/HudiDirectoryLister.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/HudiSnapshotDirectoryLister.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiDirectoryLister.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiPartitionStatsIndexSupport.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiRecordLevelIndexSupport.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiSecondaryIndexSupport.java create mode 100644 presto-hudi/src/main/java/com/facebook/presto/hudi/util/HudiUtil.java create mode 100644 presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiSmokeTest.java create mode 100644 presto-hudi/src/test/java/com/facebook/presto/hudi/testing/HudiTablesInitializer.java create mode 100644 presto-hudi/src/test/java/com/facebook/presto/hudi/testing/ResourceHudiTablesInitializer.java diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiConfig.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiConfig.java index a4c32094277ec..cb9860cd5c394 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiConfig.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiConfig.java @@ -37,7 +37,14 @@ public class HudiConfig private int splitGeneratorParallelism = 4; private boolean isColumnStatsIndexEnabled = true; + private boolean isRecordLevelIndexEnabled = true; + private boolean isSecondaryIndexEnabled = true; + private boolean isPartitionStatsIndexEnabled = true; private Duration columnStatsWaitTimeout = new Duration(1, SECONDS); + private Duration recordIndexWaitTimeout = new Duration(2, SECONDS); + private Duration secondaryIndexWaitTimeout = new Duration(2, SECONDS); + private boolean metadataPartitionListingEnabled = true; + private boolean resolveColumnNameCasingEnabled; public boolean isMetadataTableEnabled() { @@ -151,6 +158,53 @@ public boolean isColumnStatsIndexEnabled() return isColumnStatsIndexEnabled; } + @Config("hudi.index.record-level-index-enabled") + @ConfigDescription("Internal configuration to control whether record level index is enabled for debugging/testing.") + public HudiConfig setRecordLevelIndexEnabled(boolean isRecordLevelIndexEnabled) + { + this.isRecordLevelIndexEnabled = isRecordLevelIndexEnabled; + return this; + } + + public boolean isRecordLevelIndexEnabled() + { + return isRecordLevelIndexEnabled; + } + + @Config("hudi.index.secondary-index-enabled") + @ConfigDescription("Internal configuration to control whether secondary index is enabled for debugging/testing.") + public HudiConfig setSecondaryIndexEnabled(boolean isSecondaryIndexEnabled) + { + this.isSecondaryIndexEnabled = isSecondaryIndexEnabled; + return this; + } + + public boolean isSecondaryIndexEnabled() + { + return isSecondaryIndexEnabled; + } + + @Config("hudi.index.partition-stats-index-enabled") + @ConfigDescription("Internal configuration to control whether partition stats index is enabled for debugging/testing.") + public HudiConfig setPartitionStatsIndexEnabled(boolean isPartitionStatsIndexEnabled) + { + this.isPartitionStatsIndexEnabled = isPartitionStatsIndexEnabled; + return this; + } + + public boolean isPartitionStatsIndexEnabled() + { + return isPartitionStatsIndexEnabled; + } + + @Config("hudi.index.record-index.wait-timeout") + @ConfigDescription("Maximum timeout to wait for loading record index, e.g. 1000ms, 20s") + public HudiConfig setRecordIndexWaitTimeout(Duration recordIndexWaitTimeout) + { + this.recordIndexWaitTimeout = recordIndexWaitTimeout; + return this; + } + @Config("hudi.index.column-stats.wait-timeout") @ConfigDescription("Maximum timeout to wait for loading column stats, e.g. 1000ms, 20s") public HudiConfig setColumnStatsWaitTimeout(Duration columnStatusWaitTimeout) @@ -164,4 +218,51 @@ public Duration getColumnStatsWaitTimeout() { return columnStatsWaitTimeout; } + + @NotNull + public Duration getRecordIndexWaitTimeout() + { + return recordIndexWaitTimeout; + } + + @Config("hudi.index.secondary-index.wait-timeout") + @ConfigDescription("Maximum timeout to wait for loading secondary index, e.g. 1000ms, 20s") + public HudiConfig setSecondaryIndexWaitTimeout(Duration secondaryIndexWaitTimeout) + { + this.secondaryIndexWaitTimeout = secondaryIndexWaitTimeout; + return this; + } + + @NotNull + public Duration getSecondaryIndexWaitTimeout() + { + return secondaryIndexWaitTimeout; + } + + public boolean isMetadataPartitionListingEnabled() + { + return metadataPartitionListingEnabled; + } + + @Config("hudi.metadata.partition-listing.enabled") + @ConfigDescription("Enables listing table partitions through the metadata table.") + public HudiConfig setMetadataPartitionListingEnabled(boolean metadataPartitionListingEnabled) + { + this.metadataPartitionListingEnabled = metadataPartitionListingEnabled; + return this; + } + + + public boolean isResolveColumnNameCasingEnabled() + { + return resolveColumnNameCasingEnabled; + } + + @Config("hudi.table.resolve-column-name-casing.enabled") + @ConfigDescription("Reconcile column names between the catalog schema and the Hudi table to handle case differences") + public HudiConfig setResolveColumnNameCasingEnabled(boolean resolveColumnNameCasingEnabled) + { + this.resolveColumnNameCasingEnabled = resolveColumnNameCasingEnabled; + return this; + } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java index 0315b01dd0261..657d8388845cc 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java @@ -42,11 +42,8 @@ import com.facebook.presto.spi.connector.ConnectorMetadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.org.apache.avro.Schema; import org.apache.hudi.util.Lazy; import java.io.IOException; @@ -62,6 +59,9 @@ import static com.facebook.presto.hudi.HudiColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hudi.HudiErrorCode.HUDI_FILESYSTEM_ERROR; import static com.facebook.presto.hudi.HudiErrorCode.HUDI_UNKNOWN_TABLE_TYPE; +import static com.facebook.presto.hudi.HudiSessionProperties.isResolveColumnNameCasingEnabled; +import static com.facebook.presto.hudi.util.HudiUtil.buildTableMetaClient; +import static com.facebook.presto.hudi.util.HudiUtil.getLatestTableSchema; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableMap.toImmutableMap; @@ -114,21 +114,13 @@ public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTable ExtendedFileSystem fs = getFileSystem(session, tableName, basePath); return new HudiTableHandle( table, - Lazy.lazily(() -> buildTableMetaClient(fs, basePath)), + Lazy.lazily(() -> buildTableMetaClient(fs, tableName.getTableName(), basePath)), table.getDatabaseName(), table.getTableName(), basePath, hudiTableType); } - public static HoodieTableMetaClient buildTableMetaClient( - ExtendedFileSystem fs, - String basePath) - { - StorageConfiguration conf = HadoopFSUtils.getStorageConfWithCopy(fs.getConf()); - return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build(); - } - private ExtendedFileSystem getFileSystem(ConnectorSession session, SchemaTableName table, String basePath) { HdfsContext hdfsContext = new HdfsContext( @@ -155,16 +147,22 @@ public Optional getSystemTable(ConnectorSession session, SchemaTabl @Override public List getTableLayouts(ConnectorSession session, ConnectorTableHandle tableHandle, Constraint constraint, Optional> desiredColumns) { - HudiTableHandle handle = (HudiTableHandle) tableHandle; + HudiTableHandle hudiTableHandle = (HudiTableHandle) tableHandle; Table table = getTable(session, tableHandle); List partitionColumns = getPartitionColumnHandles(table); List dataColumns = getDataColumnHandles(table); + HudiPredicates predicates = HudiPredicates.from(constraint.getSummary()); + Optional> hudiTableSchema = isResolveColumnNameCasingEnabled(session) ? + Optional.of(Lazy.lazily(() -> getLatestTableSchema(hudiTableHandle.getMetaClient(), hudiTableHandle.getTableName()))) : Optional.empty(); + ConnectorTableLayout layout = new ConnectorTableLayout(new HudiTableLayoutHandle( - handle, + hudiTableHandle, dataColumns, partitionColumns, table.getParameters(), - constraint.getSummary())); + predicates.getRegularColumnPredicates(), + predicates.getPartitionColumnPredicates(), + hudiTableSchema)); return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartition.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartition.java index ccc4adb7ae2de..906dc51d063ce 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartition.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartition.java @@ -17,6 +17,8 @@ import com.facebook.presto.hive.metastore.Storage; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.storage.StoragePath; import java.util.List; import java.util.Map; @@ -32,6 +34,7 @@ public class HudiPartition // TODO: storage and dataColumns is required from MOR record cursor, might be able to remove later private final Storage storage; private final List dataColumns; + private final String relativePartitionPath; @JsonCreator public HudiPartition( @@ -39,13 +42,15 @@ public HudiPartition( @JsonProperty("values") List values, @JsonProperty("keyValues") Map keyValues, @JsonProperty("storage") Storage storage, - @JsonProperty("dataColumns") List dataColumns) + @JsonProperty("dataColumns") List dataColumns, + @JsonProperty("relativePartitionPath") String relativePartitionPath) { this.name = requireNonNull(name, "name is null"); this.values = requireNonNull(values, "values is null"); this.keyValues = requireNonNull(keyValues, "keyValues is null"); this.storage = requireNonNull(storage, "storage is null"); this.dataColumns = requireNonNull(dataColumns, "dataColumns is null"); + this.relativePartitionPath = requireNonNull(relativePartitionPath, "relativePartitionPath is null"); } @JsonProperty @@ -78,6 +83,12 @@ public List getDataColumns() return dataColumns; } + @JsonProperty + public String getRelativePartitionPath() + { + return relativePartitionPath; + } + @Override public boolean equals(Object o) { diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java index 3e3440fa96e32..7ce0a5a2a3005 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPartitionManager.java @@ -26,7 +26,6 @@ import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.metastore.StorageFormat; import com.facebook.presto.hive.metastore.Table; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.PrestoException; import com.google.common.base.Verify; @@ -64,7 +63,7 @@ public Map getEffectivePartitions( ConnectorSession connectorSession, ExtendedHiveMetastore metastore, HudiTableHandle tableHandle, - TupleDomain constraintSummary) + TupleDomain partitionPredicates) { MetastoreContext metastoreContext = toMetastoreContext(connectorSession); Optional
table = metastore.getTable(metastoreContext, tableHandle.getSchemaName(), tableHandle.getTableName()); @@ -85,7 +84,7 @@ public Map getEffectivePartitions( } Map partitionPredicate = new HashMap<>(); - Map domains = constraintSummary.getDomains().orElseGet(ImmutableMap::of); + Map domains = partitionPredicates.getDomains().orElseGet(ImmutableMap::of); List hudiColumnHandles = fromPartitionColumns(partitionColumns); for (int i = 0; i < hudiColumnHandles.size(); i++) { HudiColumnHandle column = hudiColumnHandles.get(i); @@ -108,7 +107,7 @@ public Map getEffectivePartitions( partitionNameWithVersion.getPartitionName(), hudiColumnHandles, partitionTypes, - constraintSummary)) + partitionPredicates)) .toList(); Map> partitionsByNames = metastore.getPartitionsByNames(metastoreContext, tableHandle.getSchemaName(), tableHandle.getTableName(), filteredPartitionNames); List partitionsNotFound = partitionsByNames.entrySet().stream().filter(e -> e.getValue().isEmpty()).map(Map.Entry::getKey).toList(); @@ -125,13 +124,13 @@ private boolean parseValuesAndFilterPartition( String partitionName, List partitionColumns, List partitionColumnTypes, - TupleDomain constraintSummary) + TupleDomain constraintSummary) { if (constraintSummary.isNone()) { return false; } - Map domains = constraintSummary.getDomains().orElseGet(ImmutableMap::of); + Map domains = constraintSummary.getDomains().orElseGet(ImmutableMap::of); Map partitionValues = parsePartition(partitionName, partitionColumns, partitionColumnTypes); for (HudiColumnHandle column : partitionColumns) { NullableValue value = partitionValues.get(column); diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPredicates.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPredicates.java new file mode 100644 index 0000000000000..a965f22890ac2 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiPredicates.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 com.facebook.presto.hudi; + +import com.facebook.presto.common.predicate.Domain; +import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.spi.ColumnHandle; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class HudiPredicates +{ + private final TupleDomain partitionColumnPredicates; + private final TupleDomain regularColumnPredicates; + + public static HudiPredicates from(TupleDomain predicate) + { + Map partitionColumnPredicates = new HashMap<>(); + Map regularColumnPredicates = new HashMap<>(); + + Optional> domains = predicate.getDomains(); + domains.ifPresent(columnHandleDomainMap -> columnHandleDomainMap.forEach((key, value) -> { + HudiColumnHandle columnHandle = (HudiColumnHandle) key; + if (columnHandle.isPartitionKey()) { + partitionColumnPredicates.put(columnHandle, value); + } + else { + regularColumnPredicates.put(columnHandle, value); + } + })); + + return new HudiPredicates( + TupleDomain.withColumnDomains(partitionColumnPredicates), + TupleDomain.withColumnDomains(regularColumnPredicates)); + } + + private HudiPredicates( + TupleDomain partitionColumnPredicates, + TupleDomain regularColumnPredicates) + { + this.partitionColumnPredicates = partitionColumnPredicates; + this.regularColumnPredicates = regularColumnPredicates; + } + + public TupleDomain getPartitionColumnPredicates() + { + return partitionColumnPredicates; + } + + public TupleDomain getRegularColumnPredicates() + { + return regularColumnPredicates; + } +} \ No newline at end of file diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSessionProperties.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSessionProperties.java index 07251f4da913e..61870a85fd348 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSessionProperties.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSessionProperties.java @@ -45,7 +45,14 @@ public class HudiSessionProperties private static final String SPLIT_GENERATOR_PARALLELISM = "split_generator_parallelism"; static final String COLUMN_STATS_INDEX_ENABLED = "column_stats_index_enabled"; + static final String RECORD_LEVEL_INDEX_ENABLED = "record_level_index_enabled"; + static final String SECONDARY_INDEX_ENABLED = "secondary_index_enabled"; + static final String PARTITION_STATS_INDEX_ENABLED = "partition_stats_index_enabled"; static final String COLUMN_STATS_WAIT_TIMEOUT = "column_stats_wait_timeout"; + static final String RECORD_INDEX_WAIT_TIMEOUT = "record_index_wait_timeout"; + static final String SECONDARY_INDEX_WAIT_TIMEOUT = "secondary_index_wait_timeout"; + static final String METADATA_PARTITION_LISTING_ENABLED = "metadata_partition_listing_enabled"; + static final String RESOLVE_COLUMN_NAME_CASING_ENABLED = "resolve_column_name_casing_enabled"; @Inject public HudiSessionProperties(HudiConfig hudiConfig) @@ -97,16 +104,51 @@ public HudiSessionProperties(HudiConfig hudiConfig) "Number of threads used to generate splits from partitions", hudiConfig.getSplitGeneratorParallelism(), false), + booleanProperty( + RECORD_LEVEL_INDEX_ENABLED, + "Enable record level index for file skipping", + hudiConfig.isRecordLevelIndexEnabled(), + true), + booleanProperty( + SECONDARY_INDEX_ENABLED, + "Enable secondary index for file skipping", + hudiConfig.isSecondaryIndexEnabled(), + true), booleanProperty( COLUMN_STATS_INDEX_ENABLED, "Enable column stats index for file skipping", hudiConfig.isColumnStatsIndexEnabled(), true), + booleanProperty( + PARTITION_STATS_INDEX_ENABLED, + "Enable partition stats index for file skipping", + hudiConfig.isPartitionStatsIndexEnabled(), + true), durationProperty( COLUMN_STATS_WAIT_TIMEOUT, "Maximum timeout to wait for loading column stats", hudiConfig.getColumnStatsWaitTimeout(), - false)); + false), + durationProperty( + RECORD_INDEX_WAIT_TIMEOUT, + "Maximum timeout to wait for loading record index", + hudiConfig.getRecordIndexWaitTimeout(), + false), + durationProperty( + SECONDARY_INDEX_WAIT_TIMEOUT, + "Maximum timeout to wait for loading secondary index", + hudiConfig.getSecondaryIndexWaitTimeout(), + false), + booleanProperty( + METADATA_PARTITION_LISTING_ENABLED, + "Enable metadata table based partition listing", + hudiConfig.isMetadataPartitionListingEnabled(), + false), + booleanProperty( + RESOLVE_COLUMN_NAME_CASING_ENABLED, + "Enable resolve column name casing", + hudiConfig.isResolveColumnNameCasingEnabled(), + true)); } public List> getSessionProperties() @@ -144,13 +186,53 @@ public static int getSplitGeneratorParallelism(ConnectorSession session) return session.getProperty(SPLIT_GENERATOR_PARALLELISM, Integer.class); } + public static boolean isRecordLevelIndexEnabled(ConnectorSession session) + { + return session.getProperty(RECORD_LEVEL_INDEX_ENABLED, Boolean.class); + } + + public static boolean isSecondaryIndexEnabled(ConnectorSession session) + { + return session.getProperty(SECONDARY_INDEX_ENABLED, Boolean.class); + } + public static boolean isColumnStatsIndexEnabled(ConnectorSession session) { return session.getProperty(COLUMN_STATS_INDEX_ENABLED, Boolean.class); } + public static boolean isPartitionStatsIndexEnabled(ConnectorSession session) + { + return session.getProperty(PARTITION_STATS_INDEX_ENABLED, Boolean.class); + } + + public static boolean isNoOpIndexEnabled(ConnectorSession session) + { + return !isRecordLevelIndexEnabled(session) && !isSecondaryIndexEnabled(session) && !isColumnStatsIndexEnabled(session); + } + public static Duration getColumnStatsWaitTimeout(ConnectorSession session) { return session.getProperty(COLUMN_STATS_WAIT_TIMEOUT, Duration.class); } + + public static Duration getRecordIndexWaitTimeout(ConnectorSession session) + { + return session.getProperty(RECORD_INDEX_WAIT_TIMEOUT, Duration.class); + } + + public static Duration getSecondaryIndexWaitTimeout(ConnectorSession session) + { + return session.getProperty(SECONDARY_INDEX_WAIT_TIMEOUT, Duration.class); + } + + public static boolean isMetadataPartitionListingEnabled(ConnectorSession session) + { + return session.getProperty(METADATA_PARTITION_LISTING_ENABLED, Boolean.class); + } + + public static boolean isResolveColumnNameCasingEnabled(ConnectorSession session) + { + return session.getProperty(RESOLVE_COLUMN_NAME_CASING_ENABLED, Boolean.class); + } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java index b7ea5dc4c7f93..d4d71e0803565 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitManager.java @@ -15,7 +15,6 @@ package com.facebook.presto.hudi; import com.facebook.airlift.log.Logger; -import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hudi.split.ForHudiBackgroundSplitLoader; @@ -42,7 +41,6 @@ public class HudiSplitManager { private static final Logger log = Logger.get(HudiSplitManager.class); - private final HdfsEnvironment hdfsEnvironment; private final HudiTransactionManager hudiTransactionManager; private final HudiPartitionManager hudiPartitionManager; private final ExecutorService asyncQueueExecutor; @@ -51,14 +49,12 @@ public class HudiSplitManager @Inject public HudiSplitManager( - HdfsEnvironment hdfsEnvironment, HudiTransactionManager hudiTransactionManager, HudiPartitionManager hudiPartitionManager, @ForHudiSplitAsyncQueue ExecutorService asyncQueueExecutor, @ForHudiSplitSource ScheduledExecutorService splitLoaderExecutorService, @ForHudiBackgroundSplitLoader ExecutorService splitGeneratorExecutorService) { - this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.hudiTransactionManager = requireNonNull(hudiTransactionManager, "hudiTransactionManager is null"); this.hudiPartitionManager = requireNonNull(hudiPartitionManager, "hudiPartitionManager is null"); this.asyncQueueExecutor = requireNonNull(asyncQueueExecutor, "asyncQueueExecutor is null"); @@ -75,12 +71,12 @@ public ConnectorSplitSource getSplits( { ExtendedHiveMetastore metastore = ((HudiMetadata) hudiTransactionManager.get(transaction)).getMetastore(); HudiTableLayoutHandle layout = (HudiTableLayoutHandle) layoutHandle; - HudiTableHandle table = layout.getTable(); + HudiTableHandle table = layout.getTableHandle(); Lazy> lazyPartitionMap = Lazy.lazily(() -> { // Retrieve and prune partitions HoodieTimer timer = HoodieTimer.start(); - Map partitions = hudiPartitionManager.getEffectivePartitions(session, metastore, table, layout.getTupleDomain()); + Map partitions = hudiPartitionManager.getEffectivePartitions(session, metastore, table, layout.getPartitionPredicates()); log.debug("Took %d ms to get %d partitions", timer.endTimer(), partitions.size()); return partitions; }); diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java index 54404ff16de30..575a586cb4f79 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiSplitSource.java @@ -17,19 +17,23 @@ import com.facebook.airlift.log.Logger; import com.facebook.presto.hive.metastore.Partition; import com.facebook.presto.hive.util.AsyncQueue; +import com.facebook.presto.hudi.query.HudiDirectoryLister; +import com.facebook.presto.hudi.query.HudiSnapshotDirectoryLister; import com.facebook.presto.hudi.split.HudiBackgroundSplitLoader; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSplit; import com.facebook.presto.spi.ConnectorSplitSource; +import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.connector.ConnectorPartitionHandle; import com.google.common.util.concurrent.Futures; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.util.Lazy; import java.util.Map; @@ -38,11 +42,12 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import static com.facebook.airlift.concurrent.MoreFutures.toCompletableFuture; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT; import static com.facebook.presto.hudi.HudiSessionProperties.isHudiMetadataTableEnabled; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static org.apache.hudi.common.table.view.FileSystemViewManager.createInMemoryFileSystemViewWithTimeline; public class HudiSplitSource implements ConnectorSplitSource @@ -52,6 +57,7 @@ public class HudiSplitSource private final AsyncQueue queue; private final HudiBackgroundSplitLoader splitLoader; private final ScheduledFuture splitLoaderFuture; + private final AtomicReference prestoExceptionReference = new AtomicReference<>(); public HudiSplitSource( ConnectorSession session, @@ -62,27 +68,45 @@ public HudiSplitSource( ExecutorService splitGeneratorExecutorService, int maxOutstandingSplits) { + boolean enableMetadataTable = isHudiMetadataTableEnabled(session); this.queue = new AsyncQueue<>(maxOutstandingSplits, asyncQueueExecutor); - SchemaTableName schemaTableName = layout.getTable().getSchemaTableName(); - Lazy lazyFsView = Lazy.lazily(() -> { + SchemaTableName schemaTableName = layout.getTableHandle().getSchemaTableName(); + Lazy lazyTableMetadata = Lazy.lazily(() -> { HoodieTimer timer = HoodieTimer.start(); - HoodieTableMetaClient metaClient = layout.getTable().getMetaClient(); - HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().enable(isHudiMetadataTableEnabled(session)).build(); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getStorageConf()); - HoodieTableFileSystemView fsView = createInMemoryFileSystemViewWithTimeline(engineContext, metaClient, metadataConfig, timeline); - log.info("Created file system view of table %s in %s ms", schemaTableName, timer.endTimer()); - return fsView; + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(enableMetadataTable) + .build(); + HoodieTableMetaClient metaClient = layout.getTableHandle().getMetaClient(); + HoodieEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getStorage().getConf()); + + HoodieTableMetadata tableMetadata = new HoodieBackedTableMetadata( + engineContext, + layout.getTableHandle().getMetaClient().getStorage(), metadataConfig, metaClient.getBasePath().toString(), true); + log.info("Loaded table metadata for table: %s in %s ms", schemaTableName, timer.endTimer()); + return tableMetadata; }); + HudiDirectoryLister hudiDirectoryLister = new HudiSnapshotDirectoryLister( + session, + layout, + enableMetadataTable, + lazyTableMetadata); + this.splitLoader = new HudiBackgroundSplitLoader( session, splitGeneratorExecutorService, layout, - lazyFsView, + hudiDirectoryLister, queue, - lazyPartitionMap); + lazyPartitionMap, + enableMetadataTable, + lazyTableMetadata, + throwable -> { + prestoExceptionReference.compareAndSet(null, new PrestoException(HUDI_CANNOT_OPEN_SPLIT, + "Failed to generate splits for " + schemaTableName, throwable)); + queue.finish(); + }); this.splitLoaderFuture = splitLoaderExecutorService.schedule( this.splitLoader, 0, TimeUnit.MILLISECONDS); } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableLayoutHandle.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableLayoutHandle.java index 0af5268e5e8e4..6068bcf5bc2b6 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableLayoutHandle.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiTableLayoutHandle.java @@ -14,46 +14,92 @@ package com.facebook.presto.hudi; +import com.facebook.airlift.log.Logger; import com.facebook.presto.common.predicate.TupleDomain; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorTableLayoutHandle; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.org.apache.avro.Schema; +import org.apache.hudi.util.Lazy; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import static java.util.Objects.requireNonNull; public class HudiTableLayoutHandle implements ConnectorTableLayoutHandle { - private final HudiTableHandle table; + private static final Logger log = Logger.get(HudiTableHandle.class); + private final HudiTableHandle tableHandle; private final List dataColumns; private final List partitionColumns; private final Map tableParameters; - private final TupleDomain tupleDomain; + private final TupleDomain regularPredicates; + private final TupleDomain partitionPredicates; + private final Optional> hudiTableSchemaOpt; @JsonCreator public HudiTableLayoutHandle( - @JsonProperty("table") HudiTableHandle table, + @JsonProperty("table") HudiTableHandle tableHandle, @JsonProperty("dataColumns") List dataColumns, @JsonProperty("partitionColumns") List partitionColumns, @JsonProperty("tableParameters") Map tableParameters, - @JsonProperty("tupleDomain") TupleDomain tupleDomain) + @JsonProperty("regularPredicates") TupleDomain regularPredicates, + @JsonProperty("partitionPredicates") TupleDomain partitionPredicates, + @JsonProperty("tableSchemaStr") String tableSchemaStr) { - this.table = requireNonNull(table, "table is null"); + this(tableHandle, dataColumns, partitionColumns, tableParameters, regularPredicates, partitionPredicates, buildTableSchema(tableSchemaStr)); + } + + public HudiTableLayoutHandle( + HudiTableHandle tableHandle, + List dataColumns, + List partitionColumns, + Map tableParameters, + TupleDomain partitionPredicates, + TupleDomain regularPredicates, + Optional> hudiTableSchemaOpt) + { + this.tableHandle = requireNonNull(tableHandle, "table is null"); this.dataColumns = requireNonNull(dataColumns, "dataColumns is null"); this.partitionColumns = requireNonNull(partitionColumns, "partitionColumns is null"); this.tableParameters = requireNonNull(tableParameters, "tableParameters is null"); - this.tupleDomain = requireNonNull(tupleDomain, "tupleDomain is null"); + this.regularPredicates = requireNonNull(regularPredicates, "regularPredicates is null"); + this.partitionPredicates = requireNonNull(partitionPredicates, "partitionPredicates is null"); + this.hudiTableSchemaOpt = requireNonNull(hudiTableSchemaOpt, "hudiTableSchemaOpt is null"); + } + + /** + * Builds a lazily-parsed Avro schema from the given schema string. + *

+ * Returns {@code Optional.empty()} if the input string is null/empty + * or if parsing the schema fails. + */ + private static Optional> buildTableSchema(String tableSchemaStr) + { + if (StringUtils.isNullOrEmpty(tableSchemaStr)) { + return Optional.empty(); + } + + try { + Lazy lazySchema = Lazy.lazily(() -> new Schema.Parser().parse(tableSchemaStr)); + return Optional.of(lazySchema); + } + catch (Exception e) { + log.warn(e, "Failed to parse table schema: %s", tableSchemaStr); + return Optional.empty(); + } } @JsonProperty - public HudiTableHandle getTable() + public HudiTableHandle getTableHandle() { - return table; + return tableHandle; } @JsonProperty @@ -75,9 +121,30 @@ public Map getTableParameters() } @JsonProperty - public TupleDomain getTupleDomain() + public TupleDomain getRegularPredicates() + { + return regularPredicates; + } + + @JsonProperty + public TupleDomain getPartitionPredicates() + { + return partitionPredicates; + } + + @JsonProperty + public String getTableSchemaStr() + { + return hudiTableSchemaOpt + .map(Lazy::get) + .map(Schema::toString) + .orElse(""); + } + + @JsonIgnore + public Schema getTableSchema() { - return tupleDomain; + return hudiTableSchemaOpt.map(Lazy::get).orElse(null); } @Override @@ -90,19 +157,20 @@ public boolean equals(Object o) return false; } HudiTableLayoutHandle that = (HudiTableLayoutHandle) o; - return Objects.equals(table, that.table) && - Objects.equals(tupleDomain, that.tupleDomain); + return Objects.equals(tableHandle, that.tableHandle) && + Objects.equals(regularPredicates, that.regularPredicates) && + Objects.equals(partitionPredicates, that.partitionPredicates); } @Override public int hashCode() { - return Objects.hash(table, tupleDomain); + return Objects.hash(tableHandle, regularPredicates, partitionPredicates); } @Override public String toString() { - return table.toString(); + return tableHandle.toString(); } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/HiveStylePartitionValueExtractor.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/HiveStylePartitionValueExtractor.java new file mode 100644 index 0000000000000..cffa3cce37239 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/HiveStylePartitionValueExtractor.java @@ -0,0 +1,24 @@ +package com.facebook.presto.hudi.partition; + +import java.util.Collections; +import java.util.List; + +/** + * Extractor for Hive Style Partitioned tables, when the partition folders are key value pairs. + * + *

This implementation extracts the partition value of yyyy-mm-dd from the path of type datestr=yyyy-mm-dd. + */ +public class HiveStylePartitionValueExtractor implements PartitionValueExtractor { + private static final long serialVersionUID = 1L; + + @Override + public List extractPartitionValuesInPath(String partitionPath) { + // partition path is expected to be in this format partition_key=partition_value. + String[] splits = partitionPath.split("="); + if (splits.length != 2) { + throw new IllegalArgumentException( + "Partition path " + partitionPath + " is not in the form partition_key=partition_value."); + } + return Collections.singletonList(splits[1]); + } +} \ No newline at end of file diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/MultiPartKeysValueExtractor.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/MultiPartKeysValueExtractor.java new file mode 100644 index 0000000000000..9058fe4fd9f8f --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/MultiPartKeysValueExtractor.java @@ -0,0 +1,32 @@ +package com.facebook.presto.hudi.partition; + +import org.apache.hudi.common.util.ValidationUtils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Partition Key extractor treating each value delimited by slash as separate key. + */ +public class MultiPartKeysValueExtractor implements PartitionValueExtractor { + + @Override + public List extractPartitionValuesInPath(String partitionPath) { + // If the partitionPath is empty string( which means none-partition table), the partition values + // should be empty list. + if (partitionPath.isEmpty()) { + return Collections.emptyList(); + } + String[] splits = partitionPath.split("/"); + return Arrays.stream(splits).map(s -> { + if (s.contains("=")) { + String[] moreSplit = s.split("="); + ValidationUtils.checkArgument(moreSplit.length == 2, "Partition Field (" + s + ") not in expected format"); + return moreSplit[1]; + } + return s; + }).collect(Collectors.toList()); + } +} \ No newline at end of file diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/NonPartitionedExtractor.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/NonPartitionedExtractor.java new file mode 100644 index 0000000000000..e4ddbd656eea3 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/NonPartitionedExtractor.java @@ -0,0 +1,15 @@ +package com.facebook.presto.hudi.partition; + +import java.util.ArrayList; +import java.util.List; + +/** + * Extractor for Non-partitioned hive tables. + */ +public class NonPartitionedExtractor implements PartitionValueExtractor { + + @Override + public List extractPartitionValuesInPath(String partitionPath) { + return new ArrayList<>(); + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/PartitionValueExtractor.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/PartitionValueExtractor.java new file mode 100644 index 0000000000000..fc0e0338a5da1 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/PartitionValueExtractor.java @@ -0,0 +1,16 @@ +package com.facebook.presto.hudi.partition; + +import java.io.Serializable; +import java.util.List; + +/** + * HDFS Path contain hive partition values for the keys it is partitioned on. This mapping is not straight forward and + * requires a pluggable implementation to extract the partition value from HDFS path. + *

+ * e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd] + */ +public interface PartitionValueExtractor extends Serializable +{ + + List extractPartitionValuesInPath(String partitionPath); +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/SinglePartPartitionValueExtractor.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/SinglePartPartitionValueExtractor.java new file mode 100644 index 0000000000000..ced4b6e387575 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/partition/SinglePartPartitionValueExtractor.java @@ -0,0 +1,19 @@ +package com.facebook.presto.hudi.partition; + +import java.util.Collections; +import java.util.List; + +/** + * Extractor for a partition path from a single column. + *

+ * This implementation extracts the partition value from the partition path as a single part + * even if the relative partition path contains slashes, e.g., the `TimestampBasedKeyGenerator` + * transforms the timestamp column into the partition path in the format of "yyyyMM/dd/HH". + * The slash (`/`) is replaced with dash (`-`), e.g., `202210/01/20` -> `202210-01-20`. + */ +public class SinglePartPartitionValueExtractor implements PartitionValueExtractor { + @Override + public List extractPartitionValuesInPath(String partitionPath) { + return Collections.singletonList(partitionPath.replace('/', '-')); + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/HudiDirectoryLister.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/HudiDirectoryLister.java new file mode 100644 index 0000000000000..3ffa424eee7d6 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/HudiDirectoryLister.java @@ -0,0 +1,13 @@ +package com.facebook.presto.hudi.query; + +import com.facebook.presto.hudi.HudiPartition; +import org.apache.hudi.common.model.FileSlice; + +import java.io.Closeable; +import java.util.stream.Stream; + +public interface HudiDirectoryLister + extends Closeable +{ + Stream listStatus(HudiPartition hudiPartition, boolean useIndex); +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/HudiSnapshotDirectoryLister.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/HudiSnapshotDirectoryLister.java new file mode 100644 index 0000000000000..362811857f92e --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/HudiSnapshotDirectoryLister.java @@ -0,0 +1,88 @@ +package com.facebook.presto.hudi.query; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.hudi.HudiPartition; +import com.facebook.presto.hudi.HudiTableHandle; +import com.facebook.presto.hudi.HudiTableLayoutHandle; +import com.facebook.presto.hudi.query.index.HudiIndexSupport; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.SchemaTableName; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.util.Lazy; + +import java.util.Optional; +import java.util.stream.Stream; + +import static com.facebook.presto.hudi.HudiSessionProperties.isHudiMetadataTableEnabled; +import static com.facebook.presto.hudi.query.index.IndexSupportFactory.createIndexSupport; +import static org.apache.hudi.common.table.view.FileSystemViewManager.createInMemoryFileSystemViewWithTimeline; + +public class HudiSnapshotDirectoryLister + implements HudiDirectoryLister +{ + private static final Logger log = Logger.get(HudiSnapshotDirectoryLister.class); + private final HudiTableHandle tableHandle; + private final Lazy lazyFileSystemView; + private final Optional indexSupportOpt; + + public HudiSnapshotDirectoryLister( + ConnectorSession session, + HudiTableLayoutHandle layoutHandle, + boolean enableMetadataTable, + Lazy lazyTableMetadata) + { + this.tableHandle = layoutHandle.getTableHandle(); + SchemaTableName schemaTableName = tableHandle.getSchemaTableName(); + this.lazyFileSystemView = Lazy.lazily(() -> { + HoodieTimer timer = HoodieTimer.start(); + HoodieTableMetaClient metaClient = layoutHandle.getTableHandle().getMetaClient(); + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().enable(isHudiMetadataTableEnabled(session)).build(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getStorageConf()); + HoodieTableFileSystemView fsView = createInMemoryFileSystemViewWithTimeline(engineContext, metaClient, metadataConfig, timeline); + log.info("Created file system view of table %s in %s ms", schemaTableName, timer.endTimer()); + return fsView; + }); + + Lazy lazyMetaClient = Lazy.lazily(tableHandle::getMetaClient); + this.indexSupportOpt = enableMetadataTable ? + createIndexSupport(layoutHandle, lazyMetaClient, lazyTableMetadata, layoutHandle.getRegularPredicates(), session) : Optional.empty(); + } + + @Override + public Stream listStatus(HudiPartition hudiPartition, boolean useIndex) + { + HoodieTimer timer = HoodieTimer.start(); + Stream slices = lazyFileSystemView.get().getLatestFileSlicesBeforeOrOn( + hudiPartition.getRelativePartitionPath(), + tableHandle.getLatestCommitTime(), + false); + + if (!useIndex) { + return slices; + } + + Stream fileSlices = slices + .filter(slice -> indexSupportOpt + .map(indexSupport -> !indexSupport.shouldSkipFileSlice(slice)) + .orElse(true)); + log.info("Listed partition [%s] on table %s.%s in %s ms", + hudiPartition, tableHandle.getSchemaName(), tableHandle.getTableName(), timer.endTimer()); + return fileSlices; + } + + @Override + public void close() + { + if (!lazyFileSystemView.get().isClosed()) { + lazyFileSystemView.get().close(); + } + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiDirectoryLister.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiDirectoryLister.java new file mode 100644 index 0000000000000..cd961896af00d --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiDirectoryLister.java @@ -0,0 +1,2 @@ +package com.facebook.presto.hudi.query.index;public class HudiDirectoryLister { +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiPartitionStatsIndexSupport.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiPartitionStatsIndexSupport.java new file mode 100644 index 0000000000000..15e4dad0d92c0 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiPartitionStatsIndexSupport.java @@ -0,0 +1,138 @@ +package com.facebook.presto.hudi.query.index; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.common.predicate.Domain; +import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.SchemaTableName; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.hash.ColumnIndexID; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.util.Lazy; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.facebook.presto.hudi.util.TupleDomainUtils.areSomeFieldsReferenced; +import static com.facebook.presto.hudi.util.TupleDomainUtils.filter; +import static com.facebook.presto.hudi.util.TupleDomainUtils.hasSimpleNullCheck; + +public class HudiPartitionStatsIndexSupport + extends HudiColumnStatsIndexSupport +{ + private static final Logger log = Logger.get(HudiColumnStatsIndexSupport.class); + private final Lazy lazyMetadataTable; + + public HudiPartitionStatsIndexSupport(ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + super(log, session, schemaTableName, lazyMetaClient, lazyTableMetadata, regularColumnPredicates); + this.lazyMetadataTable = lazyTableMetadata; + } + + public Optional> prunePartitions( + Collection allPartitions) + { + HoodieTimer timer = HoodieTimer.start(); + + // Filter out predicates containing simple null checks (`IS NULL` or `IS NOT NULL`) + TupleDomain filteredRegularPredicates = filter(regularColumnPredicates, (ignored, domain) -> !hasSimpleNullCheck(domain)); + + // Sanity check, if no regular domains, return immediately + if (filteredRegularPredicates.getDomains().isEmpty()) { + timer.endTimer(); + return Optional.empty(); + } + + List regularColumns = new ArrayList<>(filteredRegularPredicates.getDomains().get().keySet()); + + // Get columns to filter on + List encodedTargetColumnNames = regularColumns.stream() + .map(col -> new ColumnIndexID(col).asBase64EncodedString()).toList(); + + Map columnTypes = regularColumnPredicates.getDomains().get().entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getType())); + + // Map of domains with partition stats keyed by partition name and column name + Map> domainsWithStats = lazyMetadataTable.get().getRecordsByKeyPrefixes( + encodedTargetColumnNames, + HoodieTableMetadataUtil.PARTITION_NAME_PARTITION_STATS, true) + .collectAsList() + .stream() + .filter(f -> f.getData().getColumnStatMetadata().isPresent()) + .map(f -> f.getData().getColumnStatMetadata().get()) + .collect(Collectors.groupingBy( + HoodieMetadataColumnStats::getFileName, + Collectors.toMap( + HoodieMetadataColumnStats::getColumnName, + // Pre-compute the Domain object for each HoodieMetadataColumnStats + stats -> getDomainFromColumnStats(stats.getColumnName(), columnTypes.get(stats.getColumnName()), stats)))); + + // For each partition, determine if it should be kept based on stats availability and predicate evaluation + Stream prunedPartitions = allPartitions.stream() + .filter(partition -> { + // Check if stats exist for this partition + Map partitionDomainsWithStats = domainsWithStats.get(partition); + if (partitionDomainsWithStats == null) { + // Partition has no stats in the index, keep it + return true; + } + else { + // Partition has stats, evaluate the predicate against them + // Keep the partition only if the predicate evaluates to true + // Important: If some columns in encodedTargetColumnNames is not available in partition stats, partition will not be pruned iff all available predicate + // evaluates to true. Since we cannot determine if the predicate will evaluate to true or not on the missing stat, adopt conservative measure to true, + // i.e. to not prune + return evaluateStatisticPredicate(filteredRegularPredicates, partitionDomainsWithStats, regularColumns); + } + }); + + log.info("Took %s ms to prune partitions using Partition Stats Index for table %s", timer.endTimer(), schemaTableName); + return Optional.of(prunedPartitions); + } + + @Override + public boolean isIndexSupportAvailable() + { + return lazyMetaClient.get().getTableConfig().getMetadataPartitions() + .contains(HoodieTableMetadataUtil.PARTITION_NAME_PARTITION_STATS); + } + + @Override + public boolean canApply(TupleDomain tupleDomain) + { + // Important: has the same implementation as col stats superclass, only difference is that log messages are different + if (!isIndexSupportAvailable()) { + log.debug("Partition Stats Index partition is not enabled in metadata table."); + return false; + } + + Map indexDefinitions = getAllIndexDefinitions(); + HoodieIndexDefinition partitionsStatsIndex = indexDefinitions.get(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS); + if (partitionsStatsIndex == null || partitionsStatsIndex.getSourceFields() == null || partitionsStatsIndex.getSourceFields().isEmpty()) { + log.warn("Partition stats index definition is missing or has no source fields defined"); + return false; + } + + // Optimization applied: Only consider applicable if predicates reference indexed columns + List sourceFields = partitionsStatsIndex.getSourceFields(); + boolean applicable = areSomeFieldsReferenced(tupleDomain, sourceFields); + + if (applicable) { + log.debug("Partition Stats Index is available and applicable (predicates reference indexed columns)."); + } + else { + log.debug("Partition Stats Index is available, but predicates do not reference any indexed columns."); + } + return applicable; + } +} \ No newline at end of file diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiRecordLevelIndexSupport.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiRecordLevelIndexSupport.java new file mode 100644 index 0000000000000..0d852f8359c39 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/HudiRecordLevelIndexSupport.java @@ -0,0 +1,317 @@ +package com.facebook.presto.hudi.query.index; + +import com.facebook.airlift.log.Logger; +import com.facebook.airlift.units.Duration; +import com.facebook.presto.common.predicate.Domain; +import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.SchemaTableName; +import io.airlift.slice.Slice; +import org.apache.hudi.common.data.HoodieListData; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieRecordGlobalLocation; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.HoodieDataUtils; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.util.Lazy; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_BAD_DATA; +import static com.facebook.presto.hudi.HudiSessionProperties.getRecordIndexWaitTimeout; +import static com.facebook.presto.hudi.util.TupleDomainUtils.areAllFieldsReferenced; +import static com.facebook.presto.hudi.util.TupleDomainUtils.areDomainsInOrEqualOnly; +import static com.facebook.presto.hudi.util.TupleDomainUtils.getDiscreteSet; +import static com.facebook.presto.hudi.util.TupleDomainUtils.isDiscreteSet; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class HudiRecordLevelIndexSupport + extends HudiBaseIndexSupport +{ + private static final Logger log = Logger.get(HudiRecordLevelIndexSupport.class); + + public static final String DEFAULT_COLUMN_VALUE_SEPARATOR = ":"; + public static final String DEFAULT_RECORD_KEY_PARTS_SEPARATOR = ","; + private final CompletableFuture>> relevantFileIdsFuture; + private final Duration recordIndexWaitTimeout; + private final long futureStartTimeMs; + + public HudiRecordLevelIndexSupport(ConnectorSession session, SchemaTableName schemaTableName, Lazy lazyMetaClient, Lazy lazyTableMetadata, TupleDomain regularColumnPredicates) + { + super(log, schemaTableName, lazyMetaClient); + this.recordIndexWaitTimeout = getRecordIndexWaitTimeout(session); + if (regularColumnPredicates.isAll()) { + log.debug("Predicates cover all data, skipping record level index lookup."); + this.relevantFileIdsFuture = CompletableFuture.completedFuture(Optional.empty()); + } + else { + this.relevantFileIdsFuture = CompletableFuture.supplyAsync(() -> { + HoodieTimer timer = HoodieTimer.start(); + Option recordKeyFieldsOpt = lazyMetaClient.get().getTableConfig().getRecordKeyFields(); + if (recordKeyFieldsOpt.isEmpty() || recordKeyFieldsOpt.get().length == 0) { + // Should not happen since canApply checks for this, include for safety + throw new PrestoException(HUDI_BAD_DATA, "Record key fields must be defined to use Record Level Index."); + } + List recordKeyFields = Arrays.asList(recordKeyFieldsOpt.get()); + + // Only extract the predicates relevant to the record key fields + TupleDomain filteredDomains = extractPredicatesForColumns(regularColumnPredicates, recordKeyFields); + + // Construct the actual record keys based on the filtered predicates using Hudi's encoding scheme + List recordKeys = constructRecordKeys(filteredDomains, recordKeyFields); + + if (recordKeys.isEmpty()) { + // If key construction fails (e.g., incompatible predicates not caught by canApply, or placeholder issue) + log.warn("Took %s ms, but could not construct record keys from predicates. Skipping record index pruning for table %s", + timer.endTimer(), schemaTableName); + return Optional.empty(); + } + log.debug(String.format("Constructed %d record keys for index lookup.", recordKeys.size())); + + // Perform index lookup in metadataTable + // TODO: document here what this map is keyed by + Map recordIndex = HoodieDataUtils.dedupeAndCollectAsMap(lazyTableMetadata.get().readRecordIndexLocationsWithKeys(HoodieListData.lazy(recordKeys))); + if (recordIndex.isEmpty()) { + log.debug("Record level index lookup took %s ms but returned no locations for the given keys %s for table %s", + timer.endTimer(), recordKeys, schemaTableName); + // Return all original fileSlices + return Optional.empty(); + } + + // Collect fileIds for pruning + Set relevantFiles = recordIndex.values().stream() + .map(HoodieRecordGlobalLocation::getFileId) + .collect(Collectors.toSet()); + log.debug("Record level index lookup took %s ms and identified %d relevant file IDs.", timer.endTimer(), relevantFiles.size()); + + return Optional.of(relevantFiles); + }); + } + this.futureStartTimeMs = System.currentTimeMillis(); + } + + @Override + public boolean shouldSkipFileSlice(FileSlice slice) + { + try { + if (relevantFileIdsFuture.isDone()) { + Optional> relevantFileIds = relevantFileIdsFuture.get(); + return relevantFileIds.map(fileIds -> !fileIds.contains(slice.getFileId())).orElse(false); + } + + long elapsedMs = System.currentTimeMillis() - futureStartTimeMs; + if (elapsedMs > recordIndexWaitTimeout.toMillis()) { + // Took too long; skip decision + return false; + } + + long remainingMs = Math.max(0, recordIndexWaitTimeout.toMillis() - elapsedMs); + Optional> relevantFileIds = relevantFileIdsFuture.get(remainingMs, MILLISECONDS); + return relevantFileIds.map(fileIds -> !fileIds.contains(slice.getFileId())).orElse(false); + } + catch (TimeoutException | InterruptedException | ExecutionException e) { + return false; + } + } + + /** + * Checks if the Record Level Index is available and the query predicates + * reference all record key fields with compatible (IN/EQUAL) constraints. + */ + @Override + public boolean canApply(TupleDomain tupleDomain) + { + if (!isIndexSupportAvailable()) { + log.debug("Record Level Index partition is not enabled in metadata."); + return false; + } + + Option recordKeyFieldsOpt = lazyMetaClient.get().getTableConfig().getRecordKeyFields(); + if (recordKeyFieldsOpt.isEmpty() || recordKeyFieldsOpt.get().length == 0) { + log.debug("Record key fields are not defined in table config."); + return false; + } + List recordKeyFields = Arrays.asList(recordKeyFieldsOpt.get()); + + // Ensure that predicates reference all record key fields and use IN/EQUAL + boolean applicable = areAllFieldsReferenced(tupleDomain, recordKeyFields) + && areDomainsInOrEqualOnly(tupleDomain, recordKeyFields); + + if (!applicable) { + log.debug("Predicates do not reference all record key fields or use non-compatible (non IN/EQUAL) constraints."); + } + else { + log.debug("Record Level Index is available and applicable based on predicates."); + } + return applicable; + } + + private boolean isIndexSupportAvailable() + { + return lazyMetaClient.get().getTableConfig().getMetadataPartitions() + .contains(HoodieTableMetadataUtil.PARTITION_NAME_RECORD_INDEX); + } + + /** + * Extracts predicates from a TupleDomain that match a given set of columns. + * Preserves all complex predicate properties including multi-value domains, + * range-based predicates, and nullability. + * + * @param tupleDomain The source TupleDomain containing all predicates + * @param columnFields The set of columns for which to extract predicates + * @return A new TupleDomain containing only the predicates for the specified columns + */ + public static TupleDomain extractPredicatesForColumns(TupleDomain tupleDomain, List columnFields) + { + if (tupleDomain.isNone()) { + return TupleDomain.none(); + } + + if (tupleDomain.isAll()) { + return TupleDomain.all(); + } + + // Extract the domains matching the specified columns + Map allDomains = tupleDomain.getDomains().get(); + Map filteredDomains = allDomains.entrySet().stream().filter(entry -> columnFields.contains(entry.getKey())) // Ensure key is in the column set + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + // If no domains matched, but we had some columns to extract, return ALL + if (filteredDomains.isEmpty() && !columnFields.isEmpty() && !allDomains.isEmpty()) { + return TupleDomain.all(); + } + + return TupleDomain.withColumnDomains(filteredDomains); + } + + /** + * Constructs a record key from TupleDomain based on whether it's a complex key or not. + *

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

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

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

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

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

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

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

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

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

+ * If at least one secondary index definition meets these conditions, the method returns {@code true}; otherwise, + * it returns {@code false}. + * + * @param tupleDomain the domain representing the constraints on the columns + * HoodieIndexDefinition object + * @return {@code true} if at least one secondary index can be used based on the given tuple domain; otherwise, + * {@code false} + */ + @Override + public boolean canApply(TupleDomain tupleDomain) + { + if (!isIndexSupportAvailable()) { + log.debug("Secondary Index partition is not enabled in metadata."); + return false; + } + + Map secondaryIndexDefinitions = getApplicableIndexDefinitions(tupleDomain, true); + if (secondaryIndexDefinitions.isEmpty()) { + log.debug("No applicable secondary index definitions found."); + return false; + } + + boolean atLeastOneIndexUsable = secondaryIndexDefinitions.values().stream() + .anyMatch(indexDef -> { + List sourceFields = indexDef.getSourceFields(); + // Predicates referencing columns with secondary index needs to be IN or EQUAL only + boolean usable = TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFields) + && TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields); + if (log.isDebugEnabled() && usable) { + log.debug(String.format("Secondary index '%s' on fields '%s' is usable for the query.", indexDef.getIndexName(), sourceFields)); + } + return usable; + }); + if (!atLeastOneIndexUsable) { + log.debug("Although secondary indexes exist, none match the required fields and predicate types (IN/EQUAL) for the query."); + } + return atLeastOneIndexUsable; + } + + private boolean isIndexSupportAvailable() + { + // Filter out definitions that are secondary indices + Map secondaryIndexDefinitions = getAllIndexDefinitions() + .entrySet().stream() + .filter(e -> e.getKey().contains(HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX)) + .collect(Collectors.toMap(e -> e.getValue().getIndexName(), + Map.Entry::getValue)); + return !secondaryIndexDefinitions.isEmpty(); + } + + private Map getApplicableIndexDefinitions(TupleDomain tupleDomain, boolean checkPredicateCompatibility) + { + Map allDefinitions = getAllIndexDefinitions(); + if (allDefinitions.isEmpty()) { + return Map.of(); + } + // Filter out definitions that are secondary indices + return allDefinitions.entrySet().stream() + .filter(entry -> entry.getKey().contains(HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX)) + .filter(entry -> { + if (!checkPredicateCompatibility) { + return true; + } + // Perform additional compatibility checks + List sourceFields = entry.getValue().getSourceFields(); + return TupleDomainUtils.areAllFieldsReferenced(tupleDomain, sourceFields) + && TupleDomainUtils.areDomainsInOrEqualOnly(tupleDomain, sourceFields); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private Optional> findFirstApplicableSecondaryIndex(TupleDomain queryPredicates) + { + // Predicate checks would have already been done, skip predicate checks here + Map secondaryIndexDefinitions = getApplicableIndexDefinitions(queryPredicates, false); + if (queryPredicates.getDomains().isEmpty()) { + return Optional.empty(); + } + List queryReferencedColumns = List.copyOf(queryPredicates.getDomains().get().keySet()); + return secondaryIndexDefinitions.entrySet().stream() + .filter(entry -> { + List sourceFields = entry.getValue().getSourceFields(); + // Only filter for sourceFields that match the predicates + return !sourceFields.isEmpty() && queryReferencedColumns.contains(sourceFields.get(0)); + }) + .findFirst(); + } +} \ No newline at end of file diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/IndexSupportFactory.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/IndexSupportFactory.java index 6daee2023df86..7cb33864f4932 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/IndexSupportFactory.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/query/index/IndexSupportFactory.java @@ -2,8 +2,9 @@ import com.facebook.airlift.log.Logger; import com.facebook.presto.common.predicate.TupleDomain; -import com.facebook.presto.hive.HiveColumnHandle; +import com.facebook.presto.hudi.HudiColumnHandle; import com.facebook.presto.hudi.HudiTableHandle; +import com.facebook.presto.hudi.HudiTableLayoutHandle; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.SchemaTableName; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -16,6 +17,11 @@ import static com.facebook.presto.hudi.HudiSessionProperties.isColumnStatsIndexEnabled; import static com.facebook.presto.hudi.HudiSessionProperties.isNoOpIndexEnabled; +import static com.facebook.presto.hudi.HudiSessionProperties.isPartitionStatsIndexEnabled; +import static com.facebook.presto.hudi.HudiSessionProperties.isRecordLevelIndexEnabled; +import static com.facebook.presto.hudi.HudiSessionProperties.isResolveColumnNameCasingEnabled; +import static com.facebook.presto.hudi.HudiSessionProperties.isSecondaryIndexEnabled; +import static com.facebook.presto.hudi.util.HudiUtil.getFieldFromSchema; import static java.util.Objects.requireNonNull; /** @@ -34,24 +40,31 @@ private IndexSupportFactory() {} * Creates the most suitable HudiIndexSupport strategy, considering configuration. * Uses Supplier-based lazy instantiation combined with config checks. * - * @param hudiTableHandle The hudi table handle + * @param layoutHandle The hudi table layout handle * @param lazyMetaClient The Hudi table metadata client that is lazily instantiated. * @param tupleDomain The query predicates. * @param session Session containing session properties, which is required to control index behaviours for testing/debugging * @return An Optional containing the chosen HudiIndexSupport strategy, or empty if none are applicable or enabled. */ public static Optional createIndexSupport( - HudiTableHandle hudiTableHandle, + HudiTableLayoutHandle layoutHandle, Lazy lazyMetaClient, Lazy lazyTableMetadata, - TupleDomain tupleDomain, + TupleDomain tupleDomain, ConnectorSession session) { - TupleDomain transformedTupleDomain = tupleDomain.transform(HiveColumnHandle::getName);; + TupleDomain transformedTupleDomain = transformTupleDomain(session, layoutHandle, tupleDomain); + HudiTableHandle hudiTableHandle = layoutHandle.getTableHandle(); SchemaTableName schemaTableName = hudiTableHandle.getSchemaTableName(); // Define strategies as Suppliers paired with their config (isEnabled) flag // IMPORTANT: Order of strategy here determines which index implementation is preferred first List strategyProviders = List.of( + new StrategyProvider( + () -> isRecordLevelIndexEnabled(session), + () -> new HudiRecordLevelIndexSupport(session, schemaTableName, lazyMetaClient, lazyTableMetadata, transformedTupleDomain)), + new StrategyProvider( + () -> isSecondaryIndexEnabled(session), + () -> new HudiSecondaryIndexSupport(session, schemaTableName, lazyMetaClient, lazyTableMetadata, transformedTupleDomain)), new StrategyProvider( () -> isColumnStatsIndexEnabled(session), () -> new HudiColumnStatsIndexSupport(session, schemaTableName, lazyMetaClient, lazyTableMetadata, transformedTupleDomain)), @@ -83,6 +96,35 @@ public static Optional createIndexSupport( log.debug("No suitable and enabled index support strategy found to be applicable."); return Optional.empty(); } + + public static Optional createPartitionStatsIndexSupport( + HudiTableHandle hudiTableHandle, + Lazy lazyMetaClient, + Lazy lazyTableMetadata, + TupleDomain tupleDomain, + ConnectorSession session) + { + TupleDomain transformedTupleDomain = tupleDomain.transform(HudiColumnHandle::getName);;; + + StrategyProvider partitionStatsStrategy = new StrategyProvider( + () -> isPartitionStatsIndexEnabled(session), () -> new HudiPartitionStatsIndexSupport(session, hudiTableHandle.getSchemaTableName(), lazyMetaClient, lazyTableMetadata, transformedTupleDomain)); + + if (partitionStatsStrategy.isEnabled() && partitionStatsStrategy.getStrategy().canApply(transformedTupleDomain)) { + return Optional.of((HudiPartitionStatsIndexSupport) partitionStatsStrategy.getStrategy()); + } + return Optional.empty(); + } + + private static TupleDomain transformTupleDomain(ConnectorSession session, HudiTableLayoutHandle layoutHandle, TupleDomain tupleDomain) + { + if (isResolveColumnNameCasingEnabled(session)) { + // if column case reconciliation is enabled, transform the tuple domain keys to match the column names from the Hudi table. + return tupleDomain.transform(hiveColumnHandle -> + getFieldFromSchema(hiveColumnHandle.getName(), layoutHandle.getTableSchema()).name()); + } + return tupleDomain.transform(HudiColumnHandle::getName); + } + /** * Helper class to pair the configuration check with the strategy supplier to allow for lazy initialization. */ diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java index 9fa4b00cd63cf..987151c8a8615 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiBackgroundSplitLoader.java @@ -14,27 +14,69 @@ package com.facebook.presto.hudi.split; +import com.facebook.airlift.concurrent.BoundedExecutor; import com.facebook.airlift.log.Logger; +import com.facebook.presto.hive.metastore.Column; import com.facebook.presto.hive.metastore.Partition; +import com.facebook.presto.hive.metastore.StorageFormat; +import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.hive.util.AsyncQueue; +import com.facebook.presto.hudi.HudiColumnHandle; +import com.facebook.presto.hudi.HudiPartition; +import com.facebook.presto.hudi.HudiTableHandle; import com.facebook.presto.hudi.HudiTableLayoutHandle; +import com.facebook.presto.hudi.partition.HiveStylePartitionValueExtractor; +import com.facebook.presto.hudi.partition.MultiPartKeysValueExtractor; +import com.facebook.presto.hudi.partition.NonPartitionedExtractor; +import com.facebook.presto.hudi.partition.PartitionValueExtractor; +import com.facebook.presto.hudi.partition.SinglePartPartitionValueExtractor; +import com.facebook.presto.hudi.query.HudiDirectoryLister; +import com.facebook.presto.hudi.query.index.HudiPartitionStatsIndexSupport; +import com.facebook.presto.hudi.query.index.IndexSupportFactory; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSplit; import com.facebook.presto.spi.PrestoException; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; -import org.apache.hudi.common.util.HoodieTimer; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Streams; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.Lazy; import java.util.ArrayList; +import java.util.Deque; +import java.util.HexFormat; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; -import static com.facebook.presto.hudi.HudiErrorCode.HUDI_CANNOT_GENERATE_SPLIT; +import static com.facebook.airlift.concurrent.MoreFutures.addExceptionCallback; +import static com.facebook.presto.hive.metastore.MetastoreUtil.extractPartitionValues; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_INVALID_METADATA; +import static com.facebook.presto.hudi.HudiMetadata.fromDataColumns; import static com.facebook.presto.hudi.HudiSessionProperties.getSplitGeneratorParallelism; +import static com.facebook.presto.hudi.HudiSessionProperties.isMetadataPartitionListingEnabled; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; /** @@ -44,59 +86,294 @@ public class HudiBackgroundSplitLoader implements Runnable { private static final Logger log = Logger.get(HudiBackgroundSplitLoader.class); + public static final String DELIMITER_STR = "/"; + public static final String EQUALS_STR = "="; private final ConnectorSession session; private final HudiTableLayoutHandle layout; - private final Lazy lazyFsView; + private final HudiDirectoryLister hudiDirectoryLister; private final AsyncQueue asyncQueue; private final Lazy> lazyPartitionMap; private final int splitGeneratorNumThreads; private final ExecutorService splitGeneratorExecutorService; + private final boolean enableMetadataTable; + private final Lazy lazyTableMetadata; + private final Consumer errorListener; + private final Optional partitionIndexSupportOpt; + private final boolean isMetadataPartitionListingEnabled; public HudiBackgroundSplitLoader( ConnectorSession session, ExecutorService splitGeneratorExecutorService, HudiTableLayoutHandle layout, - Lazy lazyFsView, + HudiDirectoryLister hudiDirectoryLister, AsyncQueue asyncQueue, - Lazy> lazyPartitionMap) + Lazy> lazyPartitionMap, + boolean enableMetadataTable, + Lazy lazyTableMetadata, + Consumer errorListener) { this.session = requireNonNull(session, "session is null"); this.layout = requireNonNull(layout, "layout is null"); - this.lazyFsView = requireNonNull(lazyFsView, "fsView is null"); + this.hudiDirectoryLister = requireNonNull(hudiDirectoryLister, "hudiDirectoryLister is null"); this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); this.lazyPartitionMap = requireNonNull(lazyPartitionMap, "partitions is null"); this.splitGeneratorNumThreads = getSplitGeneratorParallelism(session); this.splitGeneratorExecutorService = requireNonNull(splitGeneratorExecutorService, "splitGeneratorExecutorService is null"); + this.enableMetadataTable = enableMetadataTable; + this.errorListener = errorListener; + this.partitionIndexSupportOpt = enableMetadataTable ? + IndexSupportFactory.createPartitionStatsIndexSupport(layout.getTableHandle(), Lazy.lazily(() -> layout.getTableHandle().getMetaClient()), lazyTableMetadata, layout.getRegularPredicates(), session) : Optional.empty(); + this.isMetadataPartitionListingEnabled = isMetadataPartitionListingEnabled(session); + this.lazyTableMetadata = lazyTableMetadata; } @Override public void run() { - HoodieTimer timer = HoodieTimer.start(); - List splitGeneratorList = new ArrayList<>(); - List splitGeneratorFutures = new ArrayList<>(); - ConcurrentLinkedQueue concurrentPartitionQueue = new ConcurrentLinkedQueue<>(lazyPartitionMap.get().keySet()); + try { + if (enableMetadataTable) { + generateSplits(true); + return; + } + + // Fallback to partition pruning generator + generateSplits(false); + } + catch (Exception e) { + errorListener.accept(e); + } + } + + private void generateSplits(boolean useIndex) { + // Attempt to apply partition pruning using partition stats index + Deque partitionQueue = getPartitions(useIndex); + if (partitionQueue.isEmpty()) { + asyncQueue.finish(); + return; + } + + List splitGenerators = new ArrayList<>(); + List> futures = new ArrayList<>(); - // Start a number of partition split generators to generate the splits in parallel - for (int i = 0; i < splitGeneratorNumThreads; i++) { + int splitGeneratorParallelism = Math.max(1, Math.min(splitGeneratorNumThreads, partitionQueue.size())); + Executor splitGeneratorExecutor = new BoundedExecutor(splitGeneratorExecutorService, splitGeneratorParallelism); + + for (int i = 0; i < splitGeneratorParallelism; i++) { HudiPartitionSplitGenerator generator = new HudiPartitionSplitGenerator( - session, layout, lazyFsView, lazyPartitionMap.get(), asyncQueue, concurrentPartitionQueue); - splitGeneratorList.add(generator); - splitGeneratorFutures.add(splitGeneratorExecutorService.submit(generator)); + session, layout, hudiDirectoryLister, lazyPartitionMap.get(), asyncQueue, partitionQueue, useIndex); + splitGenerators.add(generator); + ListenableFuture future = Futures.submit(generator, splitGeneratorExecutor); + addExceptionCallback(future, errorListener); + futures.add(future); + } + + // Signal all generators to stop once partition queue is drained + splitGenerators.forEach(HudiPartitionSplitGenerator::stopRunning); + + log.info("Wait for partition pruning split generation to finish on table %s.%s", layout.getTableHandle().getSchemaName(), layout.getTableHandle().getTableName()); + try { + Futures.whenAllComplete(futures) + .run(asyncQueue::finish, directExecutor()) + .get(); + log.info("Partition pruning split generation finished on table %s.%s", layout.getTableHandle().getSchemaName(), layout.getTableHandle().getTableName()); + } + catch (InterruptedException | ExecutionException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new PrestoException(HUDI_CANNOT_OPEN_SPLIT, "Error generating Hudi split", e); + } + } + + private Deque getPartitions(boolean useIndex) + { + Map metadataPartitions; + HudiTableHandle tableHandle = layout.getTableHandle(); + if (enableMetadataTable && isMetadataPartitionListingEnabled) { + try { + PartitionValueExtractor partitionValueExtractor = getPartitionValueExtractor(tableHandle.getMetaClient().getTableConfig()); + List hudiPartitionColumns = layout.getPartitionColumns(); + + List partitionColumns = hudiPartitionColumns.stream() + .map(column -> new Column( + column.getName(), + column.getHiveType(), + column.getComment(), + Optional.empty())) + .toList(); + log.info("Listing partitions for %s.%s via metadata table using partition value extractor %s", + tableHandle.getSchemaName(), tableHandle.getTableName(), partitionValueExtractor.getClass().getSimpleName()); + metadataPartitions = lazyTableMetadata.get() + .getAllPartitionPaths().stream() + .map(partitionPath -> buildPartition(partitionPath, partitionColumns, tableHandle, partitionValueExtractor)) + .collect(Collectors.toMap( + this::getHivePartitionName, + Function.identity())); + + if (metadataPartitions.isEmpty()) { + log.warn("No partitions found via metadata table for %s.%s, switching to metastore-based listing.", + tableHandle.getSchemaName(), tableHandle.getTableName()); + metadataPartitions = lazyPartitionMap.get(); + } + } + catch (Exception e) { + log.error(e, "Failed to get partitions from metadata table %s.%s, falling back to metastore based partition listing", + tableHandle.getSchemaName(), tableHandle.getTableName()); + metadataPartitions = lazyPartitionMap.get(); + } + } + else { + metadataPartitions = lazyPartitionMap.get(); + } + + Set allPartitions = metadataPartitions.keySet(); + Stream effectivePartitions = Optional.ofNullable(useIndex && partitionIndexSupportOpt.isPresent() + ? partitionIndexSupportOpt.get().prunePartitions(allPartitions).orElse(null) + : null).orElse(allPartitions.stream()); + + return effectivePartitions + .map(partitionName -> getHudiPartition(layout, partitionName)) + .collect(Collectors.toCollection(ConcurrentLinkedDeque::new)); + } + + private HudiPartition getHudiPartition(HudiTableLayoutHandle tableLayout, String partitionName) + { + String databaseName = tableLayout.getTableHandle().getSchemaName(); + String tableName = tableLayout.getTableHandle().getTableName(); + List partitionColumns = tableLayout.getPartitionColumns(); + Path tablePath = new Path(layout.getTableHandle().getPath()); + if (partitionColumns.isEmpty()) { + // non-partitioned tableLayout + Table metastoreTable = Optional.ofNullable(layout.getTableHandle().getTable()) + .orElseThrow(() -> new PrestoException(HUDI_INVALID_METADATA, format("Table %s.%s expected but not found", databaseName, tableName))); + Path partitionPath = new Path(metastoreTable.getStorage().getLocation()); + String relativePartitionPath = FSUtils.getRelativePartitionPath(new StoragePath(tablePath.toUri()), new StoragePath(partitionPath.toUri())); + return new HudiPartition(partitionName, ImmutableList.of(), ImmutableMap.of(), metastoreTable.getStorage(), tableLayout.getDataColumns(), relativePartitionPath); + } + else { + // partitioned tableLayout + List partitionValues = extractPartitionValues(partitionName); + checkArgument(partitionColumns.size() == partitionValues.size(), + format("Invalid partition name %s for partition columns %s", partitionName, partitionColumns)); + Partition partition = Optional.ofNullable(lazyPartitionMap.get().get(partitionName)) + .orElseThrow(() -> new PrestoException(HUDI_INVALID_METADATA, format("Partition %s expected but not found", partitionName))); + Map keyValues = zipPartitionKeyValues(partitionColumns, partitionValues); + Path partitionPath = new Path(partition.getStorage().getLocation()); + String relativePartitionPath = FSUtils.getRelativePartitionPath(new StoragePath(tablePath.toUri()), new StoragePath(partitionPath.toUri())); + return new HudiPartition(partitionName, partitionValues, keyValues, partition.getStorage(), fromDataColumns(partition.getColumns()), relativePartitionPath); } + } + + private Map zipPartitionKeyValues(List partitionColumns, List partitionValues) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + Streams.forEachPair(partitionColumns.stream(), partitionValues.stream(), + (column, value) -> builder.put(column.getName(), value)); + return builder.build(); + } + + private Partition buildPartition(String partitionPath, List partitionColumns, HudiTableHandle tableHandle, PartitionValueExtractor partitionValueExtractor) + { + if (partitionPath == null || partitionPath.isEmpty()) { + return Partition.builder() + .setDatabaseName(tableHandle.getSchemaName()) + .setTableName(tableHandle.getTableName()) + .withStorage(storageBuilder -> + storageBuilder.setLocation(layout.getTableHandle().getPath()) + .setStorageFormat(StorageFormat.VIEW_STORAGE_FORMAT)) + .setColumns(ImmutableList.of()) + .setValues(ImmutableList.of()) + .build(); + } + + List values = partitionValueExtractor.extractPartitionValuesInPath(partitionPath).stream().map(this::unescapePathName).toList(); + if (partitionColumns.size() != values.size()) { + throw new HoodieException("Cannot extract partition values from partition path: " + partitionPath); + } + + return Partition.builder() + .setDatabaseName(tableHandle.getSchemaName()) + .setTableName(tableHandle.getTableName()) + .withStorage(storageBuilder -> + storageBuilder.setLocation(getFullPath(layout.getTableHandle().getPath(), partitionPath)) + // Storage format is unused by the Hudi connector + .setStorageFormat(StorageFormat.VIEW_STORAGE_FORMAT)) + .setColumns(partitionColumns) + .setValues(values) + .build(); + } + + private String getFullPath(String basePath, String relativePartitionPath) + { + return basePath.endsWith(DELIMITER_STR) + ? basePath + relativePartitionPath + : basePath + DELIMITER_STR + relativePartitionPath; + } + + String getHivePartitionName(Partition partition) + { + List columns = partition.getColumns(); + List values = partition.getValues(); - // Wait for all split generators to finish - for (Future future : splitGeneratorFutures) { + return IntStream.range(0, columns.size()) + .mapToObj(i -> columns.get(i).getName() + EQUALS_STR + values.get(i)) + .collect(Collectors.joining(DELIMITER_STR)); + } + + private PartitionValueExtractor getPartitionValueExtractor(HoodieTableConfig tableConfig) + { + Option partitionFieldsOpt = tableConfig.getPartitionFields(); + + if (!partitionFieldsOpt.isPresent()) { + return new NonPartitionedExtractor(); + } + + String[] partitionFields = partitionFieldsOpt.get(); + if (partitionFields.length == 1) { + return Boolean.parseBoolean(tableConfig.getHiveStylePartitioningEnable()) ? + new HiveStylePartitionValueExtractor() : new SinglePartPartitionValueExtractor(); + } + + return new MultiPartKeysValueExtractor(); + } + + private String unescapePathName(String path) + { + // fast path, no escaped characters and therefore no copying necessary + int escapedAtIndex = path.indexOf('%'); + if (escapedAtIndex < 0 || escapedAtIndex + 2 >= path.length()) { + return path; + } + + // slow path, unescape into a new string copy + StringBuilder sb = new StringBuilder(); + int fromIndex = 0; + while (escapedAtIndex >= 0 && escapedAtIndex + 2 < path.length()) { + // preceding sequence without escaped characters + if (escapedAtIndex > fromIndex) { + sb.append(path, fromIndex, escapedAtIndex); + } + // try to parse the to digits after the percent sign as hex try { - future.get(); + int code = HexFormat.fromHexDigits(path, escapedAtIndex + 1, escapedAtIndex + 3); + sb.append((char) code); + // advance past the percent sign and both hex digits + fromIndex = escapedAtIndex + 3; } - catch (InterruptedException | ExecutionException e) { - throw new PrestoException(HUDI_CANNOT_GENERATE_SPLIT, "Error generating Hudi split", e); + catch (NumberFormatException e) { + // invalid escape sequence, only advance past the percent sign + sb.append('%'); + fromIndex = escapedAtIndex + 1; } + // find next escaped character + escapedAtIndex = path.indexOf('%', fromIndex); + } + // trailing sequence without escaped characters + if (fromIndex < path.length()) { + sb.append(path, fromIndex, path.length()); } - asyncQueue.finish(); - log.debug("Finished getting all splits in %d ms", timer.endTimer()); + return sb.toString(); } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java index 2a21f27b15a62..4aeb27a732eaa 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/split/HudiPartitionSplitGenerator.java @@ -17,45 +17,33 @@ import com.facebook.airlift.log.Logger; import com.facebook.airlift.units.DataSize; import com.facebook.presto.hive.metastore.Partition; -import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.hive.util.AsyncQueue; -import com.facebook.presto.hudi.HudiColumnHandle; import com.facebook.presto.hudi.HudiFile; import com.facebook.presto.hudi.HudiPartition; import com.facebook.presto.hudi.HudiSplit; import com.facebook.presto.hudi.HudiTableHandle; import com.facebook.presto.hudi.HudiTableLayoutHandle; import com.facebook.presto.hudi.HudiTableType; +import com.facebook.presto.hudi.query.HudiDirectoryLister; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSplit; -import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.schedule.NodeSelectionStrategy; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Streams; import org.apache.hadoop.fs.Path; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; -import org.apache.hudi.storage.StoragePath; -import org.apache.hudi.util.Lazy; +import java.util.Deque; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Queue; import java.util.stream.Stream; -import static com.facebook.presto.hive.metastore.MetastoreUtil.extractPartitionValues; -import static com.facebook.presto.hudi.HudiErrorCode.HUDI_INVALID_METADATA; -import static com.facebook.presto.hudi.HudiMetadata.fromDataColumns; import static com.facebook.presto.hudi.HudiSessionProperties.getMinimumAssignedSplitWeight; import static com.facebook.presto.hudi.HudiSessionProperties.getStandardSplitWeightSize; import static com.facebook.presto.hudi.HudiSessionProperties.isSizeBasedSplitWeightsEnabled; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; /** @@ -70,90 +58,59 @@ public class HudiPartitionSplitGenerator private final HudiTableLayoutHandle layout; private final HudiTableHandle table; - private final Path tablePath; - private final Lazy lazyFsView; + private final HudiDirectoryLister hudiDirectoryLister; private final AsyncQueue asyncQueue; - private final Queue concurrentPartitionQueue; + private final Queue concurrentPartitionQueue; private final HudiSplitWeightProvider splitWeightProvider; private final Map partitionMap; + private final boolean useIndex; + + private boolean isRunning; public HudiPartitionSplitGenerator( ConnectorSession session, HudiTableLayoutHandle layout, - Lazy lazyFsView, + HudiDirectoryLister hudiDirectoryLister, Map partitionMap, AsyncQueue asyncQueue, - Queue concurrentPartitionQueue) + Deque concurrentPartitionQueue, + boolean useIndex) { this.layout = requireNonNull(layout, "layout is null"); - this.table = layout.getTable(); - this.tablePath = new Path(table.getPath()); - this.lazyFsView = requireNonNull(lazyFsView, "fsView is null"); + this.table = layout.getTableHandle(); + this.hudiDirectoryLister = requireNonNull(hudiDirectoryLister, "fsView is null"); this.partitionMap = requireNonNull(partitionMap, "partitionMap is null"); this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); this.concurrentPartitionQueue = requireNonNull(concurrentPartitionQueue, "concurrentPartitionQueue is null"); this.splitWeightProvider = createSplitWeightProvider(requireNonNull(session, "session is null")); + this.useIndex = useIndex; + this.isRunning = true; } @Override public void run() { HoodieTimer timer = HoodieTimer.start(); - while (!concurrentPartitionQueue.isEmpty()) { - String partitionName = concurrentPartitionQueue.poll(); - if (partitionName != null) { - generateSplitsFromPartition(partitionName); + while (isRunning || !concurrentPartitionQueue.isEmpty()) { + HudiPartition hudiPartition = concurrentPartitionQueue.poll(); + + if (hudiPartition != null && hudiPartition.getName() != null) { + generateSplitsFromPartition(hudiPartition); } } log.debug("Partition split generator finished in %d ms", timer.endTimer()); } - private void generateSplitsFromPartition(String partitionName) + private void generateSplitsFromPartition(HudiPartition hudiPartition) { - HudiPartition hudiPartition = getHudiPartition(layout, partitionName); - Path partitionPath = new Path(hudiPartition.getStorage().getLocation()); - String relativePartitionPath = FSUtils.getRelativePartitionPath(new StoragePath(tablePath.toUri()), new StoragePath(partitionPath.toUri())); - Stream fileSlices = HudiTableType.MOR.equals(table.getTableType()) ? - lazyFsView.get().getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, table.getLatestCommitTime()) : - lazyFsView.get().getLatestFileSlicesBeforeOrOn(relativePartitionPath, table.getLatestCommitTime(), false); - fileSlices.map(fileSlice -> createHudiSplit(table, fileSlice, table.getLatestCommitTime(), hudiPartition, splitWeightProvider)) + Stream partitionFileSlices = hudiDirectoryLister.listStatus(hudiPartition, useIndex); + + partitionFileSlices.map(fileSlice -> createHudiSplit(table, fileSlice, table.getLatestCommitTime(), hudiPartition, splitWeightProvider)) .filter(Optional::isPresent) .map(Optional::get) .forEach(asyncQueue::offer); } - private HudiPartition getHudiPartition(HudiTableLayoutHandle tableLayout, String partitionName) - { - String databaseName = tableLayout.getTable().getSchemaName(); - String tableName = tableLayout.getTable().getTableName(); - List partitionColumns = tableLayout.getPartitionColumns(); - - if (partitionColumns.isEmpty()) { - // non-partitioned tableLayout - Table metastoreTable = Optional.ofNullable(table.getTable()) - .orElseThrow(() -> new PrestoException(HUDI_INVALID_METADATA, format("Table %s.%s expected but not found", databaseName, tableName))); - return new HudiPartition(partitionName, ImmutableList.of(), ImmutableMap.of(), metastoreTable.getStorage(), tableLayout.getDataColumns()); - } - else { - // partitioned tableLayout - List partitionValues = extractPartitionValues(partitionName); - checkArgument(partitionColumns.size() == partitionValues.size(), - format("Invalid partition name %s for partition columns %s", partitionName, partitionColumns)); - Partition partition = Optional.ofNullable(partitionMap.get(partitionName)) - .orElseThrow(() -> new PrestoException(HUDI_INVALID_METADATA, format("Partition %s expected but not found", partitionName))); - Map keyValues = zipPartitionKeyValues(partitionColumns, partitionValues); - return new HudiPartition(partitionName, partitionValues, keyValues, partition.getStorage(), fromDataColumns(partition.getColumns())); - } - } - - private Map zipPartitionKeyValues(List partitionColumns, List partitionValues) - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - Streams.forEachPair(partitionColumns.stream(), partitionValues.stream(), - (column, value) -> builder.put(column.getName(), value)); - return builder.build(); - } - private Optional createHudiSplit( HudiTableHandle table, FileSlice slice, @@ -168,7 +125,7 @@ private Optional createHudiSplit( List logFiles = slice.getLogFiles() .map(logFile -> new HudiFile(logFile.getPath().toString(), 0, logFile.getFileSize())) .collect(toImmutableList()); - long logFilesSize = logFiles.size() > 0 ? logFiles.stream().map(HudiFile::getLength).reduce(0L, Long::sum) : 0L; + long logFilesSize = logFiles.isEmpty() ? 0L : logFiles.stream().map(HudiFile::getLength).reduce(0L, Long::sum); long sizeInBytes = baseFile != null ? baseFile.getLength() + logFilesSize : logFilesSize; return Optional.of(new HudiSplit( @@ -191,4 +148,9 @@ private static HudiSplitWeightProvider createSplitWeightProvider(ConnectorSessio } return HudiSplitWeightProvider.uniformStandardWeightProvider(); } + + public void stopRunning() + { + this.isRunning = false; + } } diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/util/HudiUtil.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/util/HudiUtil.java new file mode 100644 index 0000000000000..c41135e14ed98 --- /dev/null +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/util/HudiUtil.java @@ -0,0 +1,121 @@ +package com.facebook.presto.hudi.util; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.hive.filesystem.ExtendedFileSystem; +import com.facebook.presto.spi.PrestoException; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.hadoop.fs.HadoopFSUtils; +import org.apache.hudi.org.apache.avro.Schema; +import org.apache.hudi.storage.StorageConfiguration; + +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_BAD_DATA; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_FILESYSTEM_ERROR; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_META_CLIENT_ERROR; +import static com.facebook.presto.hudi.HudiErrorCode.HUDI_SCHEMA_ERROR; + +public class HudiUtil +{ + private static final Logger log = Logger.get(HudiUtil.class); + private static final Cache> SCHEMA_FIELD_CACHE = + CacheBuilder.newBuilder() + .maximumSize(1000) // evict after 1000 entries + .expireAfterWrite(10, TimeUnit.MINUTES) // expire 10 mins after write + .build(); + + private HudiUtil() {} + + public static HoodieTableMetaClient buildTableMetaClient( + ExtendedFileSystem fileSystem, + String tableName, + String basePath) + { + try { + StorageConfiguration conf = HadoopFSUtils.getStorageConfWithCopy(fileSystem.getConf()); + return HoodieTableMetaClient.builder() + .setConf(conf) + .setBasePath(basePath) + .build(); + } + catch (TableNotFoundException e) { + throw new PrestoException(HUDI_BAD_DATA, + "Location of table %s does not contain Hudi table metadata: %s".formatted(tableName, basePath)); + } + catch (Throwable e) { + throw new PrestoException(HUDI_META_CLIENT_ERROR, + "Unable to load Hudi meta client for table %s (%s)".formatted(tableName, basePath)); + } + } + + /** + * Retrieves a field from the given Avro schema by column name. + *

+ * The lookup proceeds in two steps: + *

    + *
  • First, attempts an exact match on the column name.
  • + *
  • If not found, falls back to a case-insensitive match using a cached lookup table
  • + *
+ *

+ * + * @param columnName Column name to search for. + * @param schema Avro {@link org.apache.avro.Schema} in which to search. + * @return The matching {@link org.apache.avro.Schema.Field}, if found. + * @throws PrestoException if no field matches the given column name. + */ + public static Schema.Field getFieldFromSchema(String columnName, Schema schema) + { + Schema.Field field = schema.getField(columnName); + if (field != null) { + return field; + } + + try { + field = SCHEMA_FIELD_CACHE + .get(schema, () -> buildFieldLookup(schema)).get(columnName.toLowerCase(Locale.ROOT)); + if (field != null) { + return field; + } + } + catch (ExecutionException e) { + throw new PrestoException(HUDI_SCHEMA_ERROR, + "Failed to build field lookup for schema", e); + } + + throw new PrestoException(HUDI_SCHEMA_ERROR, + "Failed to get column " + columnName + " from table schema"); + } + + private static Map buildFieldLookup(Schema schema) + { + return schema.getFields().stream() + .collect(Collectors.toMap( + f -> f.name().toLowerCase(Locale.ROOT), + f -> f)); + } + + + public static Schema getLatestTableSchema(HoodieTableMetaClient metaClient, String tableName) + { + try { + HoodieTimer timer = HoodieTimer.start(); + Schema schema = new TableSchemaResolver(metaClient).getTableAvroSchema(); + log.info("Fetched table schema for table %s in %s ms", tableName, timer.endTimer()); + return schema; + } + catch (Exception e) { + // failed to read schema + throw new PrestoException(HUDI_FILESYSTEM_ERROR, e); + } + } +} diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/util/TupleDomainUtils.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/util/TupleDomainUtils.java index 1361aa7637e40..563454c73a647 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/util/TupleDomainUtils.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/util/TupleDomainUtils.java @@ -1,10 +1,29 @@ package com.facebook.presto.hudi.util; +import com.facebook.presto.common.predicate.AllOrNoneValueSet; +import com.facebook.presto.common.predicate.DiscreteValues; +import com.facebook.presto.common.predicate.Domain; +import com.facebook.presto.common.predicate.EquatableValueSet; +import com.facebook.presto.common.predicate.EquatableValueSet.ValueEntry; +import com.facebook.presto.common.predicate.Range; +import com.facebook.presto.common.predicate.Ranges; +import com.facebook.presto.common.predicate.SortedRangeSet; import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.common.predicate.ValueSet; +import java.util.ArrayList; import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.BiPredicate; + +import static com.facebook.presto.common.predicate.TupleDomain.toLinkedMap; +import static com.facebook.presto.common.predicate.TupleDomain.withColumnDomains; +import static java.util.Collections.unmodifiableList; +import static java.util.Objects.requireNonNull; public class TupleDomainUtils { @@ -22,6 +41,17 @@ public static List getReferencedColumns(TupleDomain tupleDomain) return tupleDomain.getDomains().get().keySet().stream().toList(); } + /** + * Check if all of the provided source fields are referenced in the tupleDomain predicates. + */ + public static boolean areAllFieldsReferenced(TupleDomain tupleDomain, List sourceFields) + { + Set referenceColSet = new HashSet<>(TupleDomainUtils.getReferencedColumns(tupleDomain)); + Set sourceFieldSet = new HashSet<>(sourceFields); + + return referenceColSet.containsAll(sourceFieldSet); + } + /** * Check if at least one of the provided source field is referenced in the tupleDomain predicates. */ @@ -35,4 +65,185 @@ public static boolean areSomeFieldsReferenced(TupleDomain tupleDomain, L } return false; } + + /** + * Check all columns referencing sourceFields are either IN or EQUAL predicates. + */ + public static boolean areDomainsInOrEqualOnly(TupleDomain tupleDomain, List sourceFields) + { + // If no recordKeys or no recordKeyDomains, return empty list + if (sourceFields == null || sourceFields.isEmpty() || tupleDomain.isAll() || tupleDomain.isNone()) { + return false; + } + + Optional> domainsOpt = tupleDomain.getDomains(); + // Not really necessary, as tupleDomain.isNone() already checks for this + if (domainsOpt.isEmpty()) { + return false; + } + + boolean areReferencedInOrEqual = true; + for (String sourceField : sourceFields) { + Domain domain = domainsOpt.get().get(sourceField); + // For cases where sourceField does not exist in tupleDomain + if (domain == null) { + return false; + } + areReferencedInOrEqual &= (domain.isSingleValue() || isDiscreteSet(domain.getValues())); + } + return areReferencedInOrEqual; + } + + /** + * Checks if a specific Domain represents ONLY an 'IS NULL' constraint. + * This means null is allowed, and no other non-null values are allowed. + * Important: Not handling `= NULL` predicates as colA `= NULL` does not evaluate to TRUE or FALSE, it evaluates to UNKNOWN, which is treated as false. + * + * @param domain The Domain to check. + * @return true if the domain represents 'IS NULL', false otherwise. + */ + private static boolean isOnlyNullConstraint(Domain domain) + { + // Null must be allowed, and the ValueSet must allow *no* non-null values. + return domain.isNullAllowed() && domain.getValues().isNone(); + } + + /** + * Checks if a specific Domain represents ONLY an 'IS NOT NULL' constraint. + * This means null is not allowed, and all non-null values are allowed (no other range/value restrictions). + * Important: Not handling `!= NULL` or `<> NULL` predicates as this does not evaluate to TRUE or FALSE, it evaluates to UNKNOWN, which is treated as false. + * + * @param domain The Domain to check. + * @return true if the domain represents 'IS NOT NULL', false otherwise. + */ + private static boolean isOnlyNotNullConstraint(Domain domain) + { + // Null must *NOT* be allowed, and the ValueSet must allow *ALL* possible non-null values. + return !domain.isNullAllowed() && domain.getValues().isAll(); + } + + /** + * Overloaded function to test if a Domain contains null checks or not. + * + * @param domain The Domain to check. + * @return true if the domain represents 'IS NOT NULL' or 'IS NULL', false otherwise. + */ + public static boolean hasSimpleNullCheck(Domain domain) + { + return isOnlyNullConstraint(domain) || isOnlyNotNullConstraint(domain); + } + + /** + * Checks if a TupleDomain contains at least one column Domain that represents + * exclusively an 'IS NULL' or 'IS NOT NULL' constraint. + * + * @param tupleDomain The TupleDomain to inspect. + * @return true if a simple null check constraint exists, false otherwise. + */ + public static boolean hasSimpleNullCheck(TupleDomain tupleDomain) + { + // A 'None' TupleDomain implies contradiction, not a simple null check + if (tupleDomain.isNone()) { + return false; + } + Optional> domains = tupleDomain.getDomains(); + // An 'All' TupleDomain has no constraints + if (domains.isEmpty()) { + return false; + } + + // Iterate through the domains for each column in the TupleDomain + for (Domain domain : domains.get().values()) { + if (hasSimpleNullCheck(domain)) { + // Found a domain that is purely an IS NULL or IS NOT NULL check + return true; + } + } + // No domain matched the simple null check patterns + return false; + } + + public static TupleDomain filter(TupleDomain tupleDomain, BiPredicate predicate) + { + requireNonNull(predicate, "predicate is null"); + return transformDomains(tupleDomain, (key, domain) -> { + if (!predicate.test(key, domain)) { + return Domain.all(domain.getType()); + } + return domain; + }); + } + + public static TupleDomain transformDomains(TupleDomain tupleDomain, BiFunction transformation) + { + requireNonNull(transformation, "transformation is null"); + if (tupleDomain.isNone() || tupleDomain.isAll()) { + return tupleDomain; + } + + return withColumnDomains(tupleDomain.getDomains().get().entrySet().stream() + .collect(toLinkedMap( + Map.Entry::getKey, + entry -> { + Domain newDomain = transformation.apply(entry.getKey(), entry.getValue()); + return requireNonNull(newDomain, "newDomain is null"); + }))); + } + + public static boolean isDiscreteSet(ValueSet valueSet) + { + if (valueSet instanceof AllOrNoneValueSet) + { + return false; + } else if (valueSet instanceof EquatableValueSet equatableValueSet) + { + DiscreteValues discreteValues = equatableValueSet.getDiscreteValues(); + return discreteValues.isWhiteList() && !equatableValueSet.getDiscreteValues().getValues().isEmpty(); + } else if (valueSet instanceof SortedRangeSet sortedRangeSet) + { + Ranges ranges = sortedRangeSet.getRanges(); + List orderedRanges = ranges.getOrderedRanges(); + for (int i = 0; i < ranges.getRangeCount(); i++) { + if (!orderedRanges.get(i).isSingleValue()) { + return false; + } + } + return !sortedRangeSet.isNone(); + } + return false; + } + + public static List getDiscreteSet(ValueSet valueSet) + { + if (valueSet instanceof AllOrNoneValueSet) + { + throw new UnsupportedOperationException(); + } else if (valueSet instanceof EquatableValueSet equatableValueSet) + { + if (!isDiscreteSet(valueSet)) { + throw new IllegalStateException("EquatableValueSet is not a discrete set"); + } + return equatableValueSet.getEntries().stream() + .map(ValueEntry::getValue) + .toList(); + } else if (valueSet instanceof SortedRangeSet sortedRangeSet) + { + List values = new ArrayList<>(sortedRangeSet.getRangeCount()); + Ranges ranges = sortedRangeSet.getRanges(); + List orderedRanges = ranges.getOrderedRanges(); + for (int rangeIndex = 0; rangeIndex < sortedRangeSet.getRangeCount(); rangeIndex++) { + Range range = orderedRanges.get(rangeIndex); + try + { + values.add(range.getSingleValue()); + } catch (IllegalStateException e) + { + throw new IllegalStateException("SortedRangeSet is not a discrete set", e); + } + } + return unmodifiableList(values); + } + + throw new IllegalStateException("Unsupported value set type: " + valueSet.getClass().getName()); + } } \ No newline at end of file diff --git a/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiSmokeTest.java b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiSmokeTest.java new file mode 100644 index 0000000000000..0f98e181127cd --- /dev/null +++ b/presto-hudi/src/test/java/com/facebook/presto/hudi/TestHudiSmokeTest.java @@ -0,0 +1,1390 @@ +package com.facebook.presto.hudi; + +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.AbstractTestQueryFramework; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Resources; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.util.Option; +import org.intellij.lang.annotations.Language; +import org.joda.time.DateTimeZone; +import org.testng.annotations.Test; + +import java.io.File; +import java.time.LocalDateTime; +import java.time.ZonedDateTime; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; + +public class TestHudiSmokeTest + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return HudiQueryRunner.builder() + .setDataLoader(new ResourceHudiTablesInitializer()) + .addConnectorProperties(getAdditionalHudiProperties()) + .build(); + } + + protected ImmutableMap getAdditionalHudiProperties() + { + return ImmutableMap.of(); + } + + @Test + public void testReadNonPartitionedTable() + { + assertQuery( + "SELECT id, name FROM " + HUDI_NON_PART_COW, + "SELECT * FROM VALUES (1, 'a1'), (2, 'a2')"); + } + + @Test + public void testReadPartitionedTables() + { + assertQuery("SELECT symbol, max(ts) FROM " + STOCK_TICKS_COW + " GROUP BY symbol HAVING symbol = 'GOOG'", + "SELECT * FROM VALUES ('GOOG', '2018-08-31 10:59:00')"); + + assertQuery("SELECT symbol, max(ts) FROM " + STOCK_TICKS_MOR + " GROUP BY symbol HAVING symbol = 'GOOG'", + "SELECT * FROM VALUES ('GOOG', '2018-08-31 10:59:00')"); + System.out.println(getQueryRunner().execute(getSession(), "EXPLAIN ANALYZE SELECT * FROM " + HUDI_STOCK_TICKS_COW).toString()); + + System.out.println("test start"); + getQueryRunner().execute(getSession(), "SET SESSION hudi.metadata_enabled=true"); + String res = getQueryRunner().execute(getSession(), "SELECT * FROM " + HUDI_STOCK_TICKS_COW).toString(); + System.out.println(res); + assertQuery("SELECT dt, count(1) FROM " + STOCK_TICKS_MOR + " GROUP BY dt", + "SELECT * FROM VALUES ('2018-08-31', '99')"); + } + + @Test + public void testReadNonExtractablePartitionPathTable() + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .build(); + String res = getQueryRunner().execute(session, "SELECT * FROM " + HUDI_NON_EXTRACTABLE_PARTITION_PATH).toString(); + System.out.println(res); + assertQuery(session, "SELECT name FROM " + HUDI_NON_EXTRACTABLE_PARTITION_PATH + " where dt='2018-10-05'", + "SELECT * FROM VALUES ('Alice'), ('Bob')"); + assertQuery(session, "SELECT name FROM " + HUDI_NON_EXTRACTABLE_PARTITION_PATH + " where dt='2018-10-05' and hh='10'", + "SELECT * FROM VALUES ('Alice'), ('Bob')"); + } + + @Test + public void testReadPartitionedCOWTableVer8() + { + String res = getQueryRunner().execute(getSession(), "SELECT * FROM " + HUDI_STOCK_TICKS_COW).toString(); + System.out.println(res); + assertQuery("SELECT date FROM " + HUDI_STOCK_TICKS_COW + " GROUP BY date", + "SELECT * FROM VALUES ('2018-08-31')"); + assertQuery("SELECT date, count(1) FROM " + HUDI_STOCK_TICKS_COW + " GROUP BY date", + "SELECT * FROM VALUES ('2018-08-31', '99')"); + } + + // Verifies query results on a partitioned COW table (Ver8). + // Since the table defines only a single partition column, its partition + // values can be correctly parsed by Hudi's PartitionValueExtractor. + // This test asserts that grouping and aggregations on the partition column + // return the expected results. + @Test + public void testReadPartitionedMORTableVer8() + { + getQueryRunner().execute(getSession(), "SET SESSION hudi.metadata_enabled=true"); + String res = getQueryRunner().execute(getSession(), "SELECT * FROM " + HUDI_STOCK_TICKS_COW).toString(); + System.out.println(res); + assertQuery("SELECT date FROM " + HUDI_STOCK_TICKS_MOR + " GROUP BY date", + "SELECT * FROM VALUES ('2018-08-31')"); + assertQuery("SELECT date, count(1) FROM " + HUDI_STOCK_TICKS_COW + " GROUP BY date", + "SELECT * FROM VALUES ('2018-08-31', '99')"); + } + + @Test + public void testBaseFileOnlyReadWithProjection() + { + Session session = SessionBuilder.from(getSession()).build(); + MaterializedResult countResult = getQueryRunner().execute( + session, "SELECT count(*) FROM " + HUDI_TRIPS_COW_V8); + assertThat(countResult.getOnlyValue()).isEqualTo(40000L); + assertThat(countResult.getStatementStats().get().getPhysicalInputBytes()).isLessThan(500000L); + MaterializedResult groupByResult = getQueryRunner().execute( + session, "SELECT driver, count(*) FROM " + HUDI_TRIPS_COW_V8 + " group by 1"); + assertThat(groupByResult.getMaterializedRows().size()).isEqualTo(1); + assertThat(groupByResult.getMaterializedRows().getFirst().getFieldCount()).isEqualTo(2); + assertThat(groupByResult.getMaterializedRows().getFirst().getField(0)).isEqualTo("driver-563"); + assertThat(groupByResult.getMaterializedRows().getFirst().getField(1)).isEqualTo(40000L); + assertThat(groupByResult.getStatementStats().get().getPhysicalInputBytes()).isLessThan(500000L); + } + + @Test + public void testReadPartitionedMORTables() + { + getQueryRunner().execute(getSession(), "SET SESSION hudi.metadata_enabled=true"); + String res = getQueryRunner().execute(getSession(), "SELECT * FROM " + HUDI_STOCK_TICKS_MOR).toString(); + System.out.println(res); + } + + @Test + public void testMultiPartitionedTable() + { + assertQuery("SELECT _hoodie_partition_path, id, name, ts, dt, hh FROM " + HUDI_COW_PT_TBL + " WHERE id = 1", + "SELECT * FROM VALUES ('dt=2021-12-09/hh=10', 1, 'a1', 1000, '2021-12-09', '10')"); + assertQuery("SELECT _hoodie_partition_path, id, name, ts, dt, hh FROM " + HUDI_COW_PT_TBL + " WHERE id = 2", + "SELECT * FROM VALUES ('dt=2021-12-09/hh=11', 2, 'a2', 1000, '2021-12-09', '11')"); + } + + @Test + public void testShowCreateTable() + { + assertThat((String) computeActual("SHOW CREATE TABLE " + STOCK_TICKS_COW).getOnlyValue()) + .matches("CREATE TABLE \\w+\\.\\w+\\.stock_ticks_cow \\Q(\n" + + " _hoodie_commit_time varchar,\n" + + " _hoodie_commit_seqno varchar,\n" + + " _hoodie_record_key varchar,\n" + + " _hoodie_partition_path varchar,\n" + + " _hoodie_file_name varchar,\n" + + " volume bigint,\n" + + " ts varchar,\n" + + " symbol varchar,\n" + + " year integer,\n" + + " month varchar,\n" + + " high double,\n" + + " low double,\n" + + " key varchar,\n" + + " date varchar,\n" + + " close double,\n" + + " open double,\n" + + " day varchar,\n" + + " dt varchar\n" + + ")\n" + + "WITH (\n" + + " location = \\E'.*/stock_ticks_cow',\n\\Q" + + " partitioned_by = ARRAY['dt']\n" + + ")"); + // multi-partitioned table + assertThat((String) computeActual("SHOW CREATE TABLE " + HUDI_COW_PT_TBL).getOnlyValue()) + .matches("CREATE TABLE \\w+\\.\\w+\\.hudi_cow_pt_tbl \\Q(\n" + + " _hoodie_commit_time varchar,\n" + + " _hoodie_commit_seqno varchar,\n" + + " _hoodie_record_key varchar,\n" + + " _hoodie_partition_path varchar,\n" + + " _hoodie_file_name varchar,\n" + + " id bigint,\n" + + " name varchar,\n" + + " ts bigint,\n" + + " dt varchar,\n" + + " hh varchar\n" + + ")\n" + + "WITH (\n" + + " location = \\E'.*/hudi_cow_pt_tbl',\n\\Q" + + " partitioned_by = ARRAY['dt','hh']\n" + + ")"); + } + + @Test + public void testMetaColumns() + { + assertQuery("SELECT _hoodie_commit_time FROM hudi_cow_pt_tbl", "VALUES ('20220906063435640'), ('20220906063456550')"); + assertQuery("SELECT _hoodie_commit_seqno FROM hudi_cow_pt_tbl", "VALUES ('20220906063435640_0_0'), ('20220906063456550_0_0')"); + assertQuery("SELECT _hoodie_record_key FROM hudi_cow_pt_tbl", "VALUES ('id:1'), ('id:2')"); + assertQuery("SELECT _hoodie_partition_path FROM hudi_cow_pt_tbl", "VALUES ('dt=2021-12-09/hh=10'), ('dt=2021-12-09/hh=11')"); + assertQuery( + "SELECT _hoodie_file_name FROM hudi_cow_pt_tbl", + "VALUES ('719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet'), ('4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet')"); + } + + @Test + public void testPathColumn() + throws Exception + { + String path1 = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + assertThat(toInputFile(path1).exists()).isTrue(); + String path2 = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_STOCK_TICKS_MOR + " WHERE volume = 6794"); + assertThat(toInputFile(path2).exists()).isTrue(); + } + + @Test + public void testFileSizeColumn() + throws Exception + { + String path = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + long fileSize = (long) computeScalar("SELECT \"$file_size\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + assertThat(fileSize).isEqualTo(toInputFile(path).length()); + } + + @Test + public void testFileModifiedColumn() + throws Exception + { + String path = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + ZonedDateTime fileModifiedTime = (ZonedDateTime) computeScalar("SELECT \"$file_modified_time\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); + assertThat(fileModifiedTime.toInstant().toEpochMilli()) + .isEqualTo(toInputFile(path).lastModified().toEpochMilli()); + } + + @Test + public void testPartitionColumn() + { + assertQuery("SELECT \"$partition\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1", "VALUES 'dt=2021-12-09/hh=10'"); + assertQuery("SELECT \"$partition\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 2", "VALUES 'dt=2021-12-09/hh=11'"); + + assertQueryFails("SELECT \"$partition\" FROM " + HUDI_NON_PART_COW, ".* Column '\\$partition' cannot be resolved"); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testTableStatistics(boolean tableStatisticsEnabled) + throws InterruptedException + { + @Language("SQL") String query = "EXPLAIN (FORMAT JSON) SELECT t1.uuid, t1.driver, t1.fare, t1.ts " + + "FROM " + HUDI_TRIPS_COW_V8 + " t1 " + + "INNER JOIN " + HUDI_TRIPS_COW_V8 + " t2 ON t1.uuid = t2.uuid " + + "WHERE t2.ts > 0"; + Session session = SessionBuilder.from(getSession()) + .withJoinDistributionType(OptimizerConfig.JoinDistributionType.AUTOMATIC.name()) + .withTableStatisticsEnabled(tableStatisticsEnabled) + .withMdtEnabled(true) + .build(); + QueryRunner queryRunner = getQueryRunner(); + // First time the asynchronous table statistics calculation is triggered + queryRunner.execute(session, query); + Thread.sleep(2000); + // Second time the table statistics is available for CBO and the join distribution type should be REPLICATED + String jsonPlanString = (String) queryRunner.execute(session, query).getOnlyValue(); + + // Navigate to the ScanFilterProject node + String tableName = "tests.hudi_trips_cow_v8"; + JSONObject scanNode = findNodeInPlan(jsonPlanString, "ScanFilterProject", Option.of(tableName)); + assertThat(scanNode).isNotNull(); + + // Verify the estimates are based on the table statistics if enabled + JSONArray estimatesArray = scanNode.getJSONArray("estimates"); + assertThat(estimatesArray).isNotNull(); + assertThat(estimatesArray.length()).isGreaterThan(0); + JSONObject estimates = estimatesArray.getJSONObject(0); + assertThat(estimates).isNotNull(); + if (tableStatisticsEnabled) { + assertThat(estimates.getDouble("outputRowCount")).isEqualTo(40000.0); + assertThat(estimates.getDouble("outputSizeInBytes")).isGreaterThan(20000.0); + } + else { + assertThat(estimates.getString("outputRowCount")).isEqualTo("NaN"); + assertThat(estimates.getString("outputSizeInBytes")).isEqualTo("NaN"); + } + + // Verify the join distribution type is REPLICATED if table statistics is enabled; PARTITIONED otherwise + JSONObject joinNode = findNodeInPlan(jsonPlanString, "InnerJoin", Option.empty()); + String distributionDetails = findDetailContaining(joinNode, "Distribution"); + assertThat(distributionDetails).isNotNull(); + String distribution = distributionDetails.split(":")[1].trim(); + assertThat(distribution).isEqualTo(tableStatisticsEnabled ? "REPLICATED" : "PARTITIONED"); + } + + @Test + public void testPartitionFilterRequired() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQueryFails( + session, + "SELECT * FROM " + HUDI_COW_PT_TBL, + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredPredicateOnNonPartitionColumn() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQueryFails( + session, + "SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE id = 1", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredNestedQueryWithInnerPartitionPredicate() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQuery(session, "SELECT name FROM (SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE dt = '2021-12-09') WHERE id = 1", "VALUES 'a1'"); + } + + @Test + public void testPartitionFilterRequiredNestedQueryWithOuterPartitionPredicate() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQuery(session, "SELECT name FROM (SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE id = 1) WHERE dt = '2021-12-09'", "VALUES 'a1'"); + } + + @Test + public void testPartitionFilterRequiredNestedWithIsNotNullFilter() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE dt IS NOT null", "VALUES 'a1', 'a2'"); + } + + @Test + public void testPartitionFilterRequiredFilterRemovedByPlanner() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + assertQueryFails( + session, + "SELECT id FROM " + HUDI_COW_PT_TBL + " WHERE dt IS NOT null OR true", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredOnJoin() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + + // ON with partition column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt)", + errorMessage); + // ON with partition column and WHERE with same left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with same right table's regular column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with different left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.hh = '10'", + "VALUES ('a1', 'a1'), ('a1', 'a2')"); + // ON with partition column and WHERE with different regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.hh = '10'", + errorMessage); + // ON with partition column and WHERE with regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.id = 1", + errorMessage); + + // ON with regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.id = t2.id)", + errorMessage); + // ON with regular column and WHERE with left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.id = t2.id) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2')"); + // ON with partition column and WHERE with right table's regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.id = 1", + errorMessage); + } + + @Test + public void testPartitionFilterRequiredOnJoinBothTablePartitioned() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + + // ON with partition column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt)", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + // ON with partition column and WHERE with same left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with same right table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + // ON with partition column and WHERE with different left table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t1.hh = '10'", errorMessage); + // ON with partition column and WHERE with different right table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.hh = '10'", errorMessage); + // ON with partition column and WHERE with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.id = 1", errorMessage); + + // ON with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id)", errorMessage); + // ON with regular column and WHERE with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id) WHERE t1.id = 1", errorMessage); + // ON with regular column and WHERE with left table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id) WHERE t1.dt = '2021-12-09'", errorMessage); + } + + @Test + public void testPartitionFilterRequiredWithLike() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + assertQueryFails( + session, + "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1'", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredFilterIncluded() + { + Session session = SessionBuilder.from(getSession()) + .withPartitionFilterRequired(true) + .build(); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh = '10'", "VALUES 'a1'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh < '12'", "VALUES 2"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE Hh < '11'", "VALUES 1"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE HH < '10'", "VALUES 0"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 = 1 and hh IS NOT NULL", "VALUES 'a2'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh IS NULL", "VALUES 0"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh IS NOT NULL", "VALUES 2"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh LIKE '10'", "VALUES 'a1'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh LIKE '1%'", "VALUES 'a1', 'a2'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE id = 1 AND dt = '2021-12-09'", "VALUES 'a1'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh = '11' AND dt = '2021-12-09'", "VALUES 'a2'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '12' AND dt = '2021-12-19'", "VALUES 0"); + + // Predicate which could not be translated into tuple domain + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 = 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) - 11 = 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) * 2 = 20", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 > 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1' OR hh LIKE '%1'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1' AND hh LIKE '%0'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE id = 1 OR dt = '2021-12-09'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '11' OR dt = '2021-12-09'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '12' OR dt = '2021-12-19'", errorMessage); + assertQueryFails(session, "SELECT count(*) AS COUNT FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) > 2 GROUP BY name ", errorMessage); + } + + @Test + public void testHudiLongTimestampType() + throws Exception + { + testTimestampMicros(HiveTimestampPrecision.MILLISECONDS, LocalDateTime.parse("2020-10-12T16:26:02.907")); + testTimestampMicros(HiveTimestampPrecision.MICROSECONDS, LocalDateTime.parse("2020-10-12T16:26:02.906668")); + testTimestampMicros(HiveTimestampPrecision.NANOSECONDS, LocalDateTime.parse("2020-10-12T16:26:02.906668")); + } + + @Test + public void testHudiCustomKeygenEpochMillisPartitionedTables() + { + // NOTE: As of now, the partition_path value that is synced to metastore will be returned instead of the raw value that is used by the keygen + Session session = SessionBuilder.from(getSession()).build(); + @Language("SQL") String actualQuery = "SELECT _hoodie_partition_path, partition_field_country, partition_field_date FROM " + HUDI_CUSTOM_KEYGEN_PT_V8_MOR; + @Language("SQL") String expectedQuery = "VALUES ('partition_field_country=MY/partition_field_date=2025-05-13', 'MY', '2025-05-13')," + + "('partition_field_country=CN/partition_field_date=2025-06-05', 'CN', '2025-06-05')," + + "('partition_field_country=US/partition_field_date=2025-06-06', 'US', '2025-06-06')," + + "('partition_field_country=SG/partition_field_date=2025-06-06', 'SG', '2025-06-06')," + + "('partition_field_country=SG/partition_field_date=2025-06-06', 'SG', '2025-06-06')," + + "('partition_field_country=SG/partition_field_date=2025-06-07', 'SG', '2025-06-07')," + + "('partition_field_country=SG/partition_field_date=2025-06-07', 'SG', '2025-06-07')"; + assertQuery(session, actualQuery, expectedQuery); + + // Ensure that partition pruning is working (using partition_path value) of level 3 partition_path value + @Language("SQL") String actualPartitionPruningQuery = actualQuery + " WHERE partition_field_date='2025-06-06'"; + MaterializedResult partitionPruningResult = getQueryRunner().execute(session, actualPartitionPruningQuery); + // Only one split in the partition, hence, only one split processed + assertThat(partitionPruningResult.getStatementStats().get().getTotalSplits()).isEqualTo(2); + // 2 splits/filegroups, but 3 rows + assertQuery(actualPartitionPruningQuery, "VALUES ('partition_field_country=US/partition_field_date=2025-06-06', 'US', '2025-06-06'), " + + "('partition_field_country=SG/partition_field_date=2025-06-06', 'SG', '2025-06-06'), " + + "('partition_field_country=SG/partition_field_date=2025-06-06', 'SG', '2025-06-06')"); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testHudiTimestampKeygenEpochMillisPartitionedTables(boolean isRtTable) + { + String tableName = isRtTable ? HUDI_TIMESTAMP_KEYGEN_PT_EPOCH_TO_YYYY_MM_DD_HH_V8_MOR.getRtTableName() + : HUDI_TIMESTAMP_KEYGEN_PT_EPOCH_TO_YYYY_MM_DD_HH_V8_MOR.getRoTableName(); + // NOTE: As of now, the partition_path value that is synced to metastore will be returned instead of the raw value that is used by the keygen + Session session = SessionBuilder.from(getSession()).build(); + @Language("SQL") String actualQuery = "SELECT _hoodie_partition_path, partition_field FROM " + tableName; + @Language("SQL") String expectedQuery = "VALUES ('2025-06-07 08', '2025-06-07 08'), ('2025-06-06 10', '2025-06-06 10'), ('2025-06-06 09', '2025-06-06 09'), " + + "('2025-06-05 05', '2025-06-05 05'), ('2025-05-13 02', '2025-05-13 02')"; + assertQuery(session, actualQuery, expectedQuery); + + // Ensure that partition pruning is working (using partition_path value) + @Language("SQL") String actualPartPruningQuery = actualQuery + " WHERE partition_field='2025-06-07 08'"; + MaterializedResult partPruneRes = getQueryRunner().execute(session, actualPartPruningQuery); + // Only one split in the partition, hence, only one split processed + assertThat(partPruneRes.getStatementStats().get().getTotalSplits()).isEqualTo(1); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testHudiTimestampKeygenScalarPartitionedTables(boolean isRtTable) + { + String tableName = isRtTable ? HUDI_TIMESTAMP_KEYGEN_PT_SCALAR_TO_YYYY_MM_DD_HH_V8_MOR.getRtTableName() + : HUDI_TIMESTAMP_KEYGEN_PT_SCALAR_TO_YYYY_MM_DD_HH_V8_MOR.getRoTableName(); + // NOTE: As of now, the partition_path value that is synced to metastore will be returned instead of the raw value that is used by the keygen + Session session = SessionBuilder.from(getSession()).build(); + @Language("SQL") String actualQuery = "SELECT _hoodie_partition_path, partition_field FROM " + + tableName; + @Language("SQL") String expectedQuery = "VALUES ('2024-10-04 12', '2024-10-04 12'), ('2024-10-05 12', '2024-10-05 12'), ('2024-10-06 12', '2024-10-06 12'), " + + "('2024-10-07 12', '2024-10-07 12'), ('2024-10-08 12', '2024-10-08 12')"; + assertQuery(session, actualQuery, expectedQuery); + + // Ensure that partition pruning is working (using partition_path value) + @Language("SQL") String actualPartPruningQuery = actualQuery + " WHERE partition_field='2024-10-04 12'"; + MaterializedResult partPruneRes = getQueryRunner().execute(session, actualPartPruningQuery); + // Only one split in the partition, hence, only one split processed + assertThat(partPruneRes.getStatementStats().get().getTotalSplits()).isEqualTo(1); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testPartitionPruningReadMultiFgPartitionedMOR(ResourceHudiTablesInitializer.TestingTable table) + { + // Test for partition pruning without MDT (i.e. w/o partition pruning using partition stats index) + Session session = SessionBuilder.from(getSession()).build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + " WHERE country='SG'"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With partition pruning, only 2 splits in the partition should be returned + assertThat(prunedSplits).isEqualTo(2); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testColStatsFileSkipping(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(true) + .withColumnStatsTimeout("1s") + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(false) + .build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + " WHERE country='SG' AND name='a1'"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With colstats file skipping, only 1 split should be returned + assertThat(prunedSplits).isEqualTo(1); + } + + @Test + public void testColStatsFileSkippingMORTable() + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(true) + .withColumnStatsTimeout("1s") + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(false) + .build(); + MaterializedResult roTableRes = getQueryRunner().execute(session, "SELECT id, name FROM " + HUDI_NON_PART_MOR + " where name = 'Alice'"); + MaterializedResult rtTableRes = getQueryRunner().execute(session, "SELECT id, name FROM " + HUDI_NON_PART_MOR + "_rt where name = 'Cathy'"); + + // verify ro table returns results from base file + int roTableSplits = roTableRes.getStatementStats().get().getTotalSplits(); + int roTableRows = roTableRes.getRowCount(); + assertThat(roTableSplits).isEqualTo(1); + assertThat(roTableRows).isEqualTo(1); + assertThat(roTableRes.getMaterializedRows().getFirst().getField(1)).isEqualTo("Alice"); + + // verify rt table returns results from log file + int rtTableSplits = rtTableRes.getStatementStats().get().getTotalSplits(); + int rtTableRows = rtTableRes.getRowCount(); + assertThat(rtTableRows).isEqualTo(1); + assertThat(rtTableSplits).isEqualTo(1); + assertThat(rtTableRes.getMaterializedRows().getFirst().getField(1)).isEqualTo("Cathy"); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_COW_TABLE_WITH_FIELD_NAMES_IN_CAPS", "HUDI_MOR_TABLE_WITH_FIELD_NAMES_IN_CAPS"}) + public void testFileSkippingWithColumnNameUsingUppercaseLetters(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(true) + .withColumnStatsTimeout("5s") + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(false) + .withResolveColumnNameCasingEnabled(true) + .build(); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + " WHERE name='Alice'"); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int totalRows = totalRes.getRowCount(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + int prunedRows = prunedRes.getRowCount(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With colstats file skipping, only 1 split should be returned + assertThat(prunedSplits).isEqualTo(1); + assertThat(totalRows).isEqualTo(2); + assertThat(prunedRows).isEqualTo(1); + } + + @Test + public void testRLIWithColumnNameUsingUppercaseLetters() + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(true) + .withRecordIndexTimeout("1s") + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(false) + .withResolveColumnNameCasingEnabled(true) + .build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + HUDI_COW_TABLE_WITH_FIELD_NAMES_IN_CAPS); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + HUDI_COW_TABLE_WITH_FIELD_NAMES_IN_CAPS + " WHERE id='1'"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int totalRows = totalRes.getRowCount(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + int prunedRows = prunedRes.getRowCount(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With record index file skipping, only 1 split should be returned + assertThat(prunedSplits).isEqualTo(1); + assertThat(totalRows).isEqualTo(2); + assertThat(prunedRows).isEqualTo(1); + } + + @Test + public void testMultiKeyRLIWithColumnNameUsingUppercaseLetters() + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(true) + .withRecordIndexTimeout("1s") + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(false) + .withResolveColumnNameCasingEnabled(true) + .build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + HUDI_COW_TABLE_WITH_MULTI_KEYS_AND_FIELD_NAMES_IN_CAPS); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + HUDI_COW_TABLE_WITH_MULTI_KEYS_AND_FIELD_NAMES_IN_CAPS + " WHERE id='1' and age=30"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int totalRows = totalRes.getRowCount(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + int prunedRows = prunedRes.getRowCount(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With record index file skipping, only 1 split should be returned + assertThat(prunedSplits).isEqualTo(1); + assertThat(totalRows).isEqualTo(2); + assertThat(prunedRows).isEqualTo(1); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testRecordLevelFileSkipping(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(true) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(false) + .withColumnStatsTimeout("1s") + .build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + + " WHERE country='SG' AND id IN (1, 3) AND name = 'a1'"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + assertThat(prunedSplits).isLessThan(totalSplits); + // With RLI file skipping, only 1 split should be returned + assertThat(prunedSplits).isEqualTo(1); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testSecondaryIndexFileSkipping(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(true) + .withPartitionStatsIndexEnabled(false) + .withSecondaryIndexTimeout("10s") + .build(); + MaterializedResult totalRes = getQueryRunner().execute(session, "SELECT * FROM " + table); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + + " WHERE country='SG' AND price = 101.00"); + int totalSplits = totalRes.getStatementStats().get().getTotalSplits(); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + assertThat(prunedSplits).isLessThan(totalSplits); + // SI is only available for table versions >= 8 + // With SI file skipping, only 1 split should be returned + int expectedSplits = table.getHoodieTableVersion() + .greaterThanOrEquals(HoodieTableVersion.EIGHT) ? 1 : 2; + assertThat(prunedSplits).isEqualTo(expectedSplits); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testPartitionStatsIndexPartitionPruning(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(true) + .build(); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + table + // Add a constraint that is in colstats + + " WHERE ts < 1001 " + + // Add a constraint that is in colstats + "AND price < 200.00 " + + // Add a constraint on a column that is not in colstats + "AND _hoodie_file_name = 'abc' " + + // Add a simple null check constraint + "AND id is not null"); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + + // Partition stats index is only available for table versions >= 8 + // With PSI, only 2 splits in the SG partitions will be scanned + int expectedSplits = table.getHoodieTableVersion() + .greaterThanOrEquals(HoodieTableVersion.EIGHT) ? 2 : 4; + assertThat(prunedSplits).isEqualTo(expectedSplits); + } + + @Test + public void testPartitionStatsIndexPartitionPruningWithColumnNameUsingUppercaseLetters() + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .withColStatsIndexEnabled(false) + .withRecordLevelIndexEnabled(false) + .withSecondaryIndexEnabled(false) + .withPartitionStatsIndexEnabled(true) + .build(); + MaterializedResult prunedRes = getQueryRunner().execute(session, "SELECT * FROM " + HUDI_COW_PT_TABLE_WITH_FIELD_NAMES_IN_CAPS + + " WHERE country = 'US'"); + int prunedSplits = prunedRes.getStatementStats().get().getTotalSplits(); + + assertThat(prunedSplits).isEqualTo(2); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testDynamicFilterEnabledPredicatePushdown(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder + .from(getSession()) + .withDynamicFilterTimeout("10s") + .build(); + final String tableIdentifier = "hudi:tests." + table.getRoTableName(); + + @Language("SQL") String query = "SELECT t1.id, t1.name, t1.price, t1.ts FROM " + + table + " t1 " + + "INNER JOIN " + table + " t2 ON t1.id = t2.id " + + "WHERE t2.price <= 102"; + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isTrue(); + + // matcher#group() must be invoked after matcher#find() + String rowsInputString = matcher.group(1); + long actualInputRows = Long.parseLong(rowsInputString); + long expectedInputRowsAfterFiltering = 2; + assertThat(actualInputRows) + .describedAs("Number of rows input to the ScanFilter for the probe side table (%s) should reflect effective dynamic filtering", tableIdentifier) + .isEqualTo(expectedInputRowsAfterFiltering); + + // Exercise query and check output + assertQuery(query, "VALUES (1, 'a1', 100.0, 1000), (3, 'a3', 101.0, 1001)"); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testDynamicFilterDisabledPredicatePushdown(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withDynamicFilterEnabled(false) + .build(); + final String tableIdentifier = "hudi:tests." + table.getRoTableName(); + + @Language("SQL") String query = "SELECT t1.id, t1.name, t1.price, t1.ts FROM " + + table + " t1 " + + "INNER JOIN " + table + " t2 ON t1.id = t2.id " + + "WHERE t2.price <= 102"; + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + // Should not find any dynamic filtering keywords + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isFalse(); + + // Skip check on whether optimization is not applied or not, just check that output is queryable + assertQuery(query, "VALUES (1, 'a1', 100.0, 1000), (3, 'a3', 101.0, 1001)"); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testDynamicFilterEnabled_withPartitionPruningUsingDynamicFilter(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withDynamicFilterTimeout("10s") + .build(); + final String tableIdentifier = "hudi:tests." + table.getRoTableName(); + // Query is joined-on partitionField + @Language("SQL") String query = "SELECT t1.id, t1.name, t1.price, t1.ts, t1.country FROM " + + table + " t1 " + + "INNER JOIN " + table + " t2 ON t1.country = t2.country " + + "WHERE t2.price <= 102"; + + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isTrue(); + + // matcher#group() must be invoked after matcher#find() + String rowsInputString = matcher.group(1); + long actualInputRows = Long.parseLong(rowsInputString); + long expectedInputRowsAfterFiltering = 2; + assertThat(actualInputRows) + .describedAs("Number of rows input to the ScanFilter for the probe side table (%s) should reflect effective dynamic filtering", tableIdentifier) + .isEqualTo(expectedInputRowsAfterFiltering); + + // Exercise query and check output + assertQuery(query, "VALUES (1, 'a1', 100.0, 1000, 'SG'), (3, 'a3', 101.0, 1001, 'SG'), (1, 'a1', 100.0, 1000, 'SG'), (3, 'a3', 101.0, 1001, 'SG')"); + } + + @Test + public void testDynamicFilterEnabled_withPartitionPruningUsingDynamicFilterOnNestedPartitions() + { + Session session = SessionBuilder.from(getSession()) + .withDynamicFilterTimeout("10s") + .build(); + final String tableIdentifier = "hudi:tests." + HUDI_MULTI_PT_V8_MOR.getRoTableName(); + // Query is joined-on recordKey and partitionField + @Language("SQL") String query = "SELECT t1.id FROM " + + HUDI_MULTI_PT_V8_MOR + " t1 " + + "INNER JOIN " + HUDI_MULTI_PT_V8_MOR + " t2 ON t1.id = t2.id AND t1.part_int = t2.part_int " + + "WHERE t2.part_int = 2023"; + + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isTrue(); + + // matcher#group() must be invoked after matcher#find() + String rowsInputString = matcher.group(1); + long actualInputRows = Long.parseLong(rowsInputString); + // 1 row in each split, should only scan 3 splits, i.e. 3 rows + // For a more strict search, we can check the number of splits scanned on the builder side + long expectedInputRowsAfterFiltering = 3; + assertThat(actualInputRows) + .describedAs("Number of rows input to the ScanFilter for the probe side table (%s) should reflect effective dynamic filtering", tableIdentifier) + .isEqualTo(expectedInputRowsAfterFiltering); + + // Exercise query and check output + assertQuery(query, "VALUES (1), (2), (4)"); + } + + @ParameterizedTest + @EnumSource( + value = ResourceHudiTablesInitializer.TestingTable.class, + names = {"HUDI_MULTI_FG_PT_V6_MOR", "HUDI_MULTI_FG_PT_V8_MOR"}) + public void testDynamicFilterDisabled_withPartitionPruningUsingDynamicFilter(ResourceHudiTablesInitializer.TestingTable table) + { + Session session = SessionBuilder.from(getSession()) + .withDynamicFilterEnabled(false) + .build(); + final String tableIdentifier = "hudi:tests." + table.getRoTableName(); + + // Query is joined-on recordKey and partitionField + @Language("SQL") String query = "SELECT t1.id, t1.name, t1.price, t1.ts, t1.country FROM " + + table + " t1 " + + "INNER JOIN " + table + " t2 ON t1.country = t2.country " + + "WHERE t2.price <= 102"; + + MaterializedResult explainRes = getQueryRunner().execute(session, "EXPLAIN ANALYZE " + query); + Pattern scanFilterInputRowsPattern = getScanFilterInputRowsPattern(tableIdentifier); + Matcher matcher = scanFilterInputRowsPattern.matcher(explainRes.toString()); + assertThat(matcher.find()) + .withFailMessage("Could not find 'ScanFilter' for table '%s' with 'dynamicFilters' and 'Input: X rows' stats in EXPLAIN output.\nOutput was:\n%s", + tableIdentifier, explainRes.toString()) + .isFalse(); + + // Skip check on whether optimization is not applied or not, just check that output is queryable + // Cartesian product of result is produced since we are joining by partition column + assertQuery(query, "VALUES (1, 'a1', 100.0, 1000, 'SG'), (3, 'a3', 101.0, 1001, 'SG'), (1, 'a1', 100.0, 1000, 'SG'), (3, 'a3', 101.0, 1001, 'SG')"); + } + + @Test + public void testPartitionPruningOnNestedPartitions() + { + // Should only scan paths that match the part_str=*/part_int=2023/part_date=*/part_bigint=*/part_decimal=*/part_timestamp=*/part_bool=* + Session session = getSession(); + // No partition pruning + @Language("SQL") String actualQuery = "SELECT part_str, part_int, part_date, part_bigint, part_bool FROM " + HUDI_MULTI_PT_V8_MOR; + MaterializedResult actualRes = getQueryRunner().execute(session, actualQuery); + int actualTotalSplits = actualRes.getStatementStats().get().getTotalSplits(); + assertThat(actualTotalSplits).isEqualTo(5); + + // With partition pruning + @Language("SQL") String actualPartPruneQuery = actualQuery + " WHERE part_int = 2023"; + MaterializedResult actualPartPruneRes = getQueryRunner().execute(session, actualPartPruneQuery); + int actualPartPruneSplits = actualPartPruneRes.getStatementStats().get().getTotalSplits(); + assertThat(actualPartPruneSplits).isLessThan(actualTotalSplits); + assertThat(actualPartPruneSplits).isEqualTo(3); + } + + @ParameterizedTest + @MethodSource("comprehensiveTestParameters") + public void testComprehensiveTypes(ResourceHudiTablesInitializer.TestingTable table, boolean isRtTable) + { + Session session = SessionBuilder.from(getSession()) + .withMdtEnabled(true) + .build(); + // Not using #assertQuery() as it uses H2QueryRunner, which restricts the types which can be defined, particularly MAP types + // Use #getQueryRunner(), which uses TrinoQueryRunner instead + + // Define the columns that are being tested: + ImmutableList columnsToTest = ImmutableList.of( + "uuid", "precombine_field", "col_boolean", "col_tinyint", "col_smallint", "col_int", "col_bigint", "col_float", "col_double", "col_decimal", "col_string", + "col_varchar", "col_char", "col_binary", "col_date", "col_timestamp", "col_array_int", "col_array_string", "col_map_string_int", "col_struct", "col_array_struct", + "col_map_string_struct", "col_array_struct_with_map", "col_map_struct_with_array", "col_struct_nested_struct", "col_array_array_int", "col_map_string_array_double", + "col_map_string_map_string_date", "col_struct_array_struct", "col_struct_map", "part_col"); + + // Define expected output + ImmutableList> expectedRowValues = ImmutableList.of( + // uuid STRING + ImmutableList.of("'uuid1'", "'uuid2'", "'uuid3'"), + // precombine_field LONG -> BIGINT + ImmutableList.of( + // Updates were performed (RT table holds the updated value) + isRtTable ? "BIGINT '1001'" : "BIGINT '1000'", + isRtTable ? "BIGINT '1006'" : "BIGINT '1005'", + isRtTable ? "BIGINT '1101'" : "BIGINT '1100'"), + // col_boolean BOOLEAN + ImmutableList.of("true", "false", "CAST(NULL AS BOOLEAN)"), + // col_tinyint TINYINT + ImmutableList.of("TINYINT '1'", "TINYINT '2'", "CAST(NULL AS TINYINT)"), + // col_smallint SMALLINT + ImmutableList.of("SMALLINT '100'", "SMALLINT '200'", "CAST(NULL AS SMALLINT)"), + // col_int + ImmutableList.of("INTEGER '1000'", "INTEGER '2000'", "CAST(NULL AS INTEGER)"), + // col_bigint BIGINT + ImmutableList.of("BIGINT '100000'", "BIGINT '200000'", "CAST(NULL AS BIGINT)"), + // col_float + ImmutableList.of("REAL '1.1'", "REAL '2.2'", "CAST(NULL AS REAL)"), + // col_double + ImmutableList.of( + // Updates were performed on partition A values (RT table holds the updated value) + isRtTable ? "DOUBLE '110.123'" : "DOUBLE '10.123'", + isRtTable ? "DOUBLE '120.456'" : "DOUBLE '20.456'", + "CAST(NULL AS DOUBLE)"), + // col_decimal + ImmutableList.of("DECIMAL '123.45'", "DECIMAL '234.56'", "CAST(NULL AS DECIMAL(10,2))"), + // col_string + ImmutableList.of( + "'string val 1'", + "'string val 2'", + // Updates were performed on partition B values (RT table holds the updated value) + isRtTable ? "'updated string'" : "NULL"), + // col_varchar + ImmutableList.of("CAST('varchar val 1' AS VARCHAR(50))", "CAST('varchar val 2' AS VARCHAR(50))", "CAST(NULL AS VARCHAR(50))"), + // col_char + ImmutableList.of("CAST('charval1' AS CHAR(10))", "CAST('charval2' AS CHAR(10))", "CAST(NULL AS CHAR(10))"), + // col_binary BINARY -> VARBINARY: UTF-8 bytes of "binary1", "binary2", null + ImmutableList.of("X'62696e61727931'", "X'62696e61727932'", "CAST(NULL AS VARBINARY)"), + // col_date + ImmutableList.of("DATE '2025-01-15'", "DATE '2025-02-20'", "CAST(NULL AS DATE)"), + // col_timestamp TIMESTAMP + ImmutableList.of("TIMESTAMP '2025-01-15 11:30:00.000'", "TIMESTAMP '2025-02-20 12:45:00.000'", "CAST(NULL AS TIMESTAMP)"), + // col_array_int ARRAY + ImmutableList.of("ARRAY[1, 2, 3]", "ARRAY[4, 5]", "CAST(NULL AS ARRAY)"), + // col_array_string ARRAY + ImmutableList.of("ARRAY['a', 'b', 'c']", "ARRAY['d', 'e', 'f']", "CAST(NULL AS ARRAY)"), + // col_map_string_int MAP + ImmutableList.of("MAP(ARRAY['key1', 'key2'], ARRAY[10, 20])", "MAP(ARRAY['key3'], ARRAY[30])", "CAST(NULL AS MAP(VARCHAR, INTEGER))"), + // col_struct + ImmutableList.of( + "CAST(ROW('struct_str1', 55, false) AS ROW(f1 VARCHAR, f2 INTEGER, f3 BOOLEAN))", + "CAST(ROW('struct_str2', 66, true) AS ROW(f1 VARCHAR, f2 INTEGER, f3 BOOLEAN))", + "CAST(NULL AS ROW(f1 VARCHAR, f2 INTEGER, f3 BOOLEAN))"), + // col_array_struct + ImmutableList.of( + "ARRAY[CAST(ROW(1.1E0, ARRAY['n1','n2']) AS ROW(nested_f1 DOUBLE, nested_f2 ARRAY)), CAST(ROW(2.2E0, ARRAY['n3']) AS ROW(nested_f1 DOUBLE, nested_f2 ARRAY))]", + "CAST(NULL AS ARRAY)>)", + "ARRAY[CAST(ROW(3.3E0, ARRAY['n4']) AS ROW(nested_f1 DOUBLE, nested_f2 ARRAY))]"), + // col_map_string_struct + ImmutableList.of( + "MAP(ARRAY['mapkey1'], ARRAY[CAST(ROW(DATE '2024-11-01', DECIMAL '9.80') AS ROW(nested_f3 DATE, nested_f4 DECIMAL(5,2)))])", + "MAP(ARRAY['mapkey2'], ARRAY[CAST(ROW(DATE '2024-12-10', DECIMAL '7.60') AS ROW(nested_f3 DATE, nested_f4 DECIMAL(5,2)))])", + "CAST(NULL AS MAP)"), + // col_array_struct_with_map + ImmutableList.of( + "ARRAY[CAST(ROW('arr_struct1', MAP(ARRAY['map_in_struct_k1'], ARRAY[1])) AS ROW(f_arr_struct_str VARCHAR, f_arr_struct_map MAP)), CAST(ROW('arr_struct2', MAP(ARRAY['map_in_struct_k2', 'map_in_struct_k3'], ARRAY[2, 3])) AS ROW(f_arr_struct_str VARCHAR, f_arr_struct_map MAP))]", + // inner map is null + "ARRAY[CAST(ROW('arr_struct3', MAP(ARRAY['map_in_struct_k4'], ARRAY[4])) AS ROW(f_arr_struct_str VARCHAR, f_arr_struct_map MAP)), CAST(ROW('arr_struct4', CAST(NULL AS MAP)) AS ROW(f_arr_struct_str VARCHAR, f_arr_struct_map MAP))]", + "CAST(NULL AS ARRAY)>)"), + // col_map_struct_with_array + ImmutableList.of( + "MAP(ARRAY['map_struct1', 'map_struct2'], ARRAY[CAST(ROW(ARRAY[true, false], TIMESTAMP '2025-01-01 01:01:01.000') AS ROW(f_map_struct_arr ARRAY, f_map_struct_ts TIMESTAMP(3))), CAST(ROW(ARRAY[false], TIMESTAMP '2025-02-02 02:02:02.000') AS ROW(f_map_struct_arr ARRAY, f_map_struct_ts TIMESTAMP(3)))])", + // inner map is null + "MAP(ARRAY['map_struct3', 'map_struct4'], ARRAY[CAST(ROW(CAST(NULL AS ARRAY), TIMESTAMP '2025-03-03 03:03:03.000') AS ROW(f_map_struct_arr ARRAY, f_map_struct_ts TIMESTAMP(3))), CAST(ROW(ARRAY[true], CAST(NULL AS TIMESTAMP(3))) AS ROW(f_map_struct_arr ARRAY, f_map_struct_ts TIMESTAMP(3)))])", + "CAST(NULL AS MAP, f_map_struct_ts TIMESTAMP(3))>)"), + // col_struct_nested_struct + ImmutableList.of( + "CAST(ROW(101, CAST(ROW('inner_str_1', true) AS ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN))) AS ROW(outer_f1 INTEGER, nested_struct ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN)))", + // inner struct is null + "CAST(ROW(102, CAST(NULL AS ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN))) AS ROW(outer_f1 INTEGER, nested_struct ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN)))", + "CAST(NULL AS ROW(outer_f1 INTEGER, nested_struct ROW(inner_f1 VARCHAR, inner_f2 BOOLEAN)))"), + // col_array_array_int + ImmutableList.of("ARRAY[ARRAY[1, 2], ARRAY[3, 4, 5]]", "ARRAY[ARRAY[6], ARRAY[7, 8]]", "CAST(NULL AS ARRAY>)"), + // col_map_string_array_double + ImmutableList.of( + "MAP(ARRAY['arr_key1', 'arr_key2'], ARRAY[ARRAY[1.1E0, 2.2E0], ARRAY[3.3E0]])", + // inner array is null + "MAP(ARRAY['arr_key3'], ARRAY[CAST(NULL AS ARRAY)])", + "CAST(NULL AS MAP>)"), + // col_map_string_map_string_date + ImmutableList.of( + "MAP(ARRAY['map_key1'], ARRAY[MAP(ARRAY['mapkey10', 'mapkey20'], ARRAY[DATE '2024-01-01', DATE '2024-02-02'])])", + // inner map value/map is null, assuming int key 30 coerced to '30' + "MAP(ARRAY['map_key2', 'map_key3'], ARRAY[MAP(ARRAY[CAST('30' AS VARCHAR)], ARRAY[CAST(NULL AS DATE)]), CAST(NULL AS MAP)])", + "CAST(NULL AS MAP>)"), + // col_struct_array_struct + ImmutableList.of( + "CAST(ROW('outer_str_1', ARRAY[CAST(ROW(TIMESTAMP '2023-11-11 11:11:11.000', 'inner_str_1') AS ROW(inner_f3 TIMESTAMP(3), inner_f4 VARCHAR))]) AS ROW(outer_f2 VARCHAR, struct_array ARRAY))", + "CAST(ROW('outer_str_2', ARRAY[CAST(ROW(TIMESTAMP '2023-12-12 12:12:12.000', 'inner_str_2') AS ROW(inner_f3 TIMESTAMP(3), inner_f4 VARCHAR))]) AS ROW(outer_f2 VARCHAR, struct_array ARRAY))", + "CAST(NULL AS ROW(outer_f2 VARCHAR, struct_array ARRAY))"), + // col_struct_map (BIGINT literals don't need L) + ImmutableList.of( + "CAST(ROW(true, MAP(ARRAY['struct_map_k1', 'struct_map_k2'], ARRAY[1000, 2000])) AS ROW(outer_f3 BOOLEAN, struct_map MAP))", + // inner map is null + "CAST(ROW(false, CAST(NULL AS MAP)) AS ROW(outer_f3 BOOLEAN, struct_map MAP))", + "CAST(NULL AS ROW(outer_f3 BOOLEAN, struct_map MAP))"), + // part_col + ImmutableList.of("'A'", "'A'", "'B'")); + + // "Zip" results up for convenient lookup + Map> mapping = listsToMap(columnsToTest, expectedRowValues); + + // Determine which table to use base on test parameters + final String sourceTable = isRtTable ? table.getRtTableName() : table.getTableName(); + + // Test each column individually so that errors thrown are more specific/useful + for (String column : mapping.keySet()) { + // Use UNION ALL so that de-dupes will not happen + @Language("SQL") String expectedQuery = mapping.get(column).stream().map(l -> "SELECT " + l).collect(Collectors.joining(" UNION ALL ")); + @Language("SQL") String actualQuery = "SELECT " + column + " FROM " + sourceTable; + + MaterializedResult actualResults = getQueryRunner().execute(session, actualQuery); + MaterializedResult expectedResults = getQueryRunner().execute(session, expectedQuery); + assertThat(actualResults.getMaterializedRows()) + .describedAs("failedQuery: " + actualQuery) + .hasSameSizeAs(expectedResults.getMaterializedRows()) + .containsAll(expectedResults.getMaterializedRows()); + } + + // Perform same test on all columns together + int numRows = expectedRowValues.getFirst().size(); + @Language("SQL") String expectedQuery = IntStream.range(0, numRows) + .mapToObj(rowIndex -> { + // For each row, collect the corresponding values for all columns in the defined order + String rowValuesString = columnsToTest.stream() + .map(columnName -> { + List columnData = mapping.get(columnName); + return columnData.get(rowIndex); + }) + .collect(Collectors.joining(", ")); // Joins column values: "val1, val2, val3" + return "SELECT " + rowValuesString; // Forms: "SELECT val1, val2, val3" + }) + .collect(Collectors.joining(" UNION ALL ")); + @Language("SQL") String actualQuery = "SELECT " + String.join(", ", columnsToTest) + " FROM " + sourceTable; + MaterializedResult actualResults = getQueryRunner().execute(session, actualQuery); + MaterializedResult expectedResults = getQueryRunner().execute(session, expectedQuery); + assertThat(actualResults.getMaterializedRows()) + .hasSameSizeAs(expectedResults.getMaterializedRows()) + .containsAll(expectedResults.getMaterializedRows()); + + // Perform test on selecting nested field + String columnToTest = "col_map_string_struct"; + // 1. Extract all values from the map into an array. Since each map has one entry, this array will have one ROW (or be NULL if the map is NULL). + // 2. Access the first (and only) ROW object from this array. (Using 1-based indexing for arrays, which Trino and Presto uses) + // 3. Access the 'nested_f4' field from that ROW object. + @Language("SQL") String nestedFieldQuery = "SELECT (map_values(" + columnToTest + "))[1].nested_f4 AS extracted_nested_f4 FROM " + sourceTable; + @Language("SQL") String expectedNestedFieldQuery = "WITH " + sourceTable + " AS ( " + + mapping.get(columnToTest).stream() + .map(l -> "SELECT " + l + " AS " + columnToTest) + .collect(Collectors.joining(" UNION ALL ")) + + ") " + + nestedFieldQuery; + @Language("SQL") String actualNestedFieldQuery = nestedFieldQuery; + MaterializedResult expectedNestedResult = getQueryRunner().execute(session, expectedNestedFieldQuery); + MaterializedResult actualNestedResult = getQueryRunner().execute(session, actualNestedFieldQuery); + assertThat(actualNestedResult.getMaterializedRows()) + .hasSameSizeAs(expectedNestedResult.getMaterializedRows()) + .containsAll(expectedNestedResult.getMaterializedRows()); + } + + @Test + public void testHudiPartitionFieldsWithMultipleTypes() + { + Session session = getSession(); + @Language("SQL") String actualQuery = "SELECT part_str, part_int, part_date, part_bigint, part_bool FROM " + HUDI_MULTI_PT_V8_MOR; + @Language("SQL") String expectedQuery = "VALUES " + + "('apparel', 2024, DATE '2024-01-05', 20000000001, false), " + + "('books', 2023, DATE '2023-01-15', 10000000001, true), " + + "('books', 2024, DATE '2024-02-20', 10000000003, true), " + + "('electronics', 2023, DATE '2023-03-10', 10000000002, false), " + + "('electronics', 2023, DATE '2023-03-10', 10000000002, true) "; + assertQuery(session, actualQuery, expectedQuery); + } + + private void testTimestampMicros(HiveTimestampPrecision timestampPrecision, LocalDateTime expected) + throws Exception + { + File parquetFile = new File(Resources.getResource("long_timestamp.parquet").toURI()); + Type columnType = createTimestampType(timestampPrecision.getPrecision()); + HudiSplit hudiSplit = new HudiSplit( + new HudiBaseFile(parquetFile.getPath(), parquetFile.getName(), parquetFile.length(), parquetFile.lastModified(), 0, parquetFile.length()), + ImmutableList.of(), + "000", + TupleDomain.all(), + ImmutableList.of(), + SplitWeight.standard()); + + HudiConfig config = new HudiConfig().setUseParquetColumnNames(false); + HudiSessionProperties sessionProperties = new HudiSessionProperties(config, new ParquetReaderConfig()); + ConnectorSession session = TestingConnectorSession.builder() + .setPropertyMetadata(sessionProperties.getSessionProperties()) + .build(); + + try (ConnectorPageSource pageSource = createPageSource( + session, + List.of(createBaseColumn("created", 0, HIVE_TIMESTAMP, columnType, REGULAR, Optional.empty())), + hudiSplit, + new LocalInputFile(parquetFile), + new FileFormatDataSourceStats(), + new ParquetReaderOptions(), + DateTimeZone.UTC, DynamicFilter.EMPTY, true)) { + MaterializedResult result = materializeSourceDataStream(session, pageSource, List.of(columnType)).toTestTypes(); + assertThat(result.getMaterializedRows()) + .containsOnly(new MaterializedRow(List.of(expected))); + } + } + + private static Pattern getScanFilterInputRowsPattern(String tableIdentifier) + { + // Regex to find the ScanFilter for the specific table that received a dynamic filter and extract the 'Input: X rows' value associated with it. + return Pattern.compile( + // Match the ScanFilter line for the specific table, ensuring dynamicFilters is present + "ScanFilter\\[table = " + Pattern.quote(tableIdentifier) + ".*dynamicFilters = \\{.*?\\}.*?\\]" + + ".*?" + // Match subsequent lines non-greedily until the target line is found + "\\n\\s+Input:\\s+(\\d+)\\s+rows", // Match the 'Input: X rows' line, ensuring it's indented relative to ScanFilter + Pattern.DOTALL); + } + + private TrinoInputFile toInputFile(String path) + { + return ((HudiConnector) getDistributedQueryRunner().getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")) + .newInputFile(Location.of(path)); + } + + public static Map listsToMap(List keys, List values) + { + if (keys == null || values == null) { + throw new IllegalArgumentException("Key and Value lists cannot be null."); + } + + // Determine the number of entries based on the shorter list + int limit = Math.min(keys.size(), values.size()); + + return IntStream.range(0, limit) + .boxed() + .collect(Collectors.toMap( + keys::get, + values::get, + // Merge function for duplicate keys, last one wins + (_, newValue) -> newValue)); + } + + private static Stream comprehensiveTestParameters() + { + ResourceHudiTablesInitializer.TestingTable[] tablesToTest = { + HUDI_COMPREHENSIVE_TYPES_V6_MOR, + HUDI_COMPREHENSIVE_TYPES_V8_MOR + }; + Boolean[] booleanValues = {true, false}; + + return Stream.of(tablesToTest) + .flatMap(table -> + Stream.of(booleanValues) + .map(boolValue -> Arguments.of(table, boolValue))); + } + + /** + * Entry point for finding a node in the complete JSON plan string. + * It iterates through each plan fragment ("0", "1", etc.) and starts the recursive search. + * + * @param jsonPlanString the complete JSON string of the execution plan + * @param nodeType the "name" of the node type to find (e.g., "InnerJoin", "ScanFilterProject") + * @param tableName the name of the table to match for nodes like "ScanFilterProject". can be empty + * @return The found JSONObject, or null if not found in any fragment. + */ + public static JSONObject findNodeInPlan(String jsonPlanString, String nodeType, Option tableName) + { + JSONObject fullPlan = new JSONObject(jsonPlanString); + + // Iterate over the fragment keys ("0", "1", etc.) + Iterator fragmentKeys = fullPlan.keys(); + while (fragmentKeys.hasNext()) { + String key = fragmentKeys.next(); + JSONObject fragmentNode = fullPlan.getJSONObject(key); + + // Start the recursive search from the root node of the fragment + JSONObject result = findNodeRecursive(fragmentNode, nodeType, tableName); + if (result != null) { + return result; + } + } + return null; + } + + /** + * Recursively searches for a node within a plan tree starting from the given node. + * + * @param currentNode the current {@link JSONObject} node in the plan tree to inspect + * @param nodeType the "name" of the node type to find + * @param tableName the table name to match for specific node types. can be empty + * @return the found {@link JSONObject}, or null if not found. + */ + private static JSONObject findNodeRecursive(JSONObject currentNode, String nodeType, Option tableName) + { + if (currentNode.has("name") && currentNode.getString("name").equals(nodeType)) { + // If a tableName is specified, we must match it (for Scan/Filter/Project nodes) + if (tableName.isPresent()) { + if (currentNode.has("descriptor") && currentNode.getJSONObject("descriptor").has("table")) { + String table = currentNode.getJSONObject("descriptor").getString("table"); + if (table.contains(tableName.get())) { + return currentNode; + } + } + } + else { + // If no tableName is required, found a match by nodeType alone + return currentNode; + } + } + + // If not a match, recurse into the children + if (currentNode.has("children")) { + JSONArray children = currentNode.getJSONArray("children"); + for (int i = 0; i < children.length(); i++) { + JSONObject childNode = children.getJSONObject(i); + JSONObject result = findNodeRecursive(childNode, nodeType, tableName); + if (result != null) { + return result; + } + } + } + + return null; + } + + /** + * Searches the "details" array of a given plan node for a string containing specific text. + * + * @param node the {@link JSONObject} plan node to search within + * @param content the substring to search for in the details array + * @return the full text of the first matching detail, or null if no match is found. + */ + public static String findDetailContaining(JSONObject node, String content) + { + if (node != null && node.has("details")) { + JSONArray details = node.getJSONArray("details"); + for (int i = 0; i < details.length(); i++) { + String detailText = details.getString(i); + if (detailText.contains(content)) { + return detailText; + } + } + } + return null; + } +} diff --git a/presto-hudi/src/test/java/com/facebook/presto/hudi/testing/HudiTablesInitializer.java b/presto-hudi/src/test/java/com/facebook/presto/hudi/testing/HudiTablesInitializer.java new file mode 100644 index 0000000000000..11f0b170d1ca4 --- /dev/null +++ b/presto-hudi/src/test/java/com/facebook/presto/hudi/testing/HudiTablesInitializer.java @@ -0,0 +1,10 @@ +package com.facebook.presto.hudi.testing; + +import com.facebook.presto.testing.QueryRunner; +import org.apache.hadoop.fs.Path; + +public interface HudiTablesInitializer +{ + void initializeTables(QueryRunner queryRunner, Path externalLocation, String schemaName) + throws Exception; +} \ No newline at end of file diff --git a/presto-hudi/src/test/java/com/facebook/presto/hudi/testing/ResourceHudiTablesInitializer.java b/presto-hudi/src/test/java/com/facebook/presto/hudi/testing/ResourceHudiTablesInitializer.java new file mode 100644 index 0000000000000..6095c60802a34 --- /dev/null +++ b/presto-hudi/src/test/java/com/facebook/presto/hudi/testing/ResourceHudiTablesInitializer.java @@ -0,0 +1,768 @@ +package com.facebook.presto.hudi.testing; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.hive.HiveType; +import com.facebook.presto.hive.filesystem.ExtendedFileSystem; +import com.facebook.presto.hive.metastore.Column; +import com.facebook.presto.hive.metastore.Partition; +import com.facebook.presto.hive.metastore.PartitionStatistics; +import com.facebook.presto.hive.metastore.PartitionWithStatistics; +import com.facebook.presto.hive.metastore.StorageFormat; +import com.facebook.presto.hive.metastore.thrift.HiveMetastore; +import com.facebook.presto.hudi.HudiConnector; +import com.facebook.presto.testing.QueryRunner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.security.DigestInputStream; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.facebook.presto.hive.HiveType.HIVE_BINARY; +import static com.facebook.presto.hive.HiveType.HIVE_BOOLEAN; +import static com.facebook.presto.hive.HiveType.HIVE_BYTE; +import static com.facebook.presto.hive.HiveType.HIVE_DATE; +import static com.facebook.presto.hive.HiveType.HIVE_DOUBLE; +import static com.facebook.presto.hive.HiveType.HIVE_FLOAT; +import static com.facebook.presto.hive.HiveType.HIVE_INT; +import static com.facebook.presto.hive.HiveType.HIVE_LONG; +import static com.facebook.presto.hive.HiveType.HIVE_SHORT; +import static com.facebook.presto.hive.HiveType.HIVE_STRING; +import static com.facebook.presto.hive.HiveType.HIVE_TIMESTAMP; +import static com.facebook.presto.hive.metastore.MetastoreUtil.extractPartitionValues; +import static com.facebook.presto.hive.metastore.PrestoTableType.EXTERNAL_TABLE; + +public class ResourceHudiTablesInitializer + implements HudiTablesInitializer +{ + private static final Logger log = Logger.get(ResourceHudiTablesInitializer.class); + private static final String HASH_ALGORITHM = "SHA-256"; + private static final String TEST_RESOURCE_NAME = "hudi-testing-data"; + + // There might be other entry points that are using this initializer, make the location unique so it is more identifiable via logs + private final String baseLocationPrefix = UUID.randomUUID().toString(); + private final Path tempDir; + + /** + * Manually declaring a temp directory here and performing a manual cleanup as this constructor is invoked in HudiQueryRunner in a @BeforeAll static function. + * This means that jupiter's managed @TempDir annotation cannot be used as the path will be passed as null. + */ + public ResourceHudiTablesInitializer() + { + // There are multiple entry points and they may perform unzipping together, ensure that they are all unzipping to different paths + try { + this.tempDir = Files.createTempDirectory(TEST_RESOURCE_NAME + "_" + baseLocationPrefix); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void initializeTables(QueryRunner queryRunner, org.apache.hadoop.fs.Path externalLocation, String schemaName) + throws Exception + { + // Inflate all deflated test resource archives to a temporary directory + HudiTableUnzipper.unzipAllItemsInResource(TEST_RESOURCE_NAME, tempDir); + ExtendedFileSystem fileSystem = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + String locationSuffix = schemaName + "_" + baseLocationPrefix; + Path baseLocation = externalLocation.appendSuffix(locationSuffix); + log.info("Initialized test resource directory as: %s", baseLocation.toString()); + copyDir(tempDir, fileSystem, baseLocation); + // Perform a cleanup + HudiTableUnzipper.deleteInflatedFiles(tempDir); + + for (TestingTable table : TestingTable.values()) { + String tableName = table.getTableName(); + Path tablePath = baseLocation.appendPath(tableName); + + // Always create ro table by default + createTable( + queryRunner, + schemaName, + tablePath, + tableName, + table.getDataColumns(), + table.getPartitionColumns(), + table.getPartitions(), + false); + + if (table.isCreateRtTable()) { + createTable( + queryRunner, + schemaName, + tablePath, + table.getRtTableName(), + table.getDataColumns(), + table.getPartitionColumns(), + table.getPartitions(), + true); + } + + // Set table version + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setStorage(new HudiTrinoStorage(fileSystem, new TrinoStorageConfiguration())) + .setBasePath(tablePath.toString()) + .build(); + table.setTableVersion(metaClient.getTableConfig().getTableVersion()); + } + } + + private void createTable( + QueryRunner queryRunner, + String schemaName, + Path tablePath, + String tableName, + List dataColumns, + List partitionColumns, + Map partitions, + boolean isRtTable) + { + StorageFormat roStorageFormat = StorageFormat.create( + PARQUET_HIVE_SERDE_CLASS, + HUDI_PARQUET_INPUT_FORMAT, + MAPRED_PARQUET_OUTPUT_FORMAT_CLASS); + + StorageFormat rtStorageFormat = StorageFormat.create( + PARQUET_HIVE_SERDE_CLASS, + HUDI_PARQUET_REALTIME_INPUT_FORMAT, + MAPRED_PARQUET_OUTPUT_FORMAT_CLASS); + + Table table = Table.builder() + .setDatabaseName(schemaName) + .setTableName(tableName) + .setTableType(EXTERNAL_TABLE.name()) + .setOwner(Optional.of("public")) + .setDataColumns(dataColumns) + .setPartitionColumns(partitionColumns) + .setParameters(ImmutableMap.of("serialization.format", "1", "EXTERNAL", "TRUE")) + .withStorage(storageBuilder -> storageBuilder + .setStorageFormat(isRtTable ? rtStorageFormat : roStorageFormat) + .setLocation(tablePath.toString())) + .build(); + + HiveMetastore metastore = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + metastore.createTable(table, PrincipalPrivileges.NO_PRIVILEGES); + + List partitionsToAdd = new ArrayList<>(); + partitions.forEach((partitionName, partitionPath) -> { + Partition partition = Partition.builder() + .setDatabaseName(schemaName) + .setTableName(tableName) + .setValues(extractPartitionValues(partitionName)) + .withStorage(storageBuilder -> storageBuilder + .setStorageFormat(isRtTable ? rtStorageFormat : roStorageFormat) + .setLocation(tablePath.appendPath(partitionPath).toString())) + .setColumns(dataColumns) + .build(); + partitionsToAdd.add(new PartitionWithStatistics(partition, partitionName, PartitionStatistics.empty())); + }); + metastore.addPartitions(schemaName, tableName, partitionsToAdd); + } + + private static Column column(String name, HiveType type) + { + return new Column(name, type, Optional.empty(), Map.of()); + } + + public static void copyDir(Path sourceDirectory, ExtendedFileSystem fileSystem, Path destinationDirectory) + throws IOException + { + try (Stream paths = Files.walk(sourceDirectory)) { + for (Iterator iterator = paths.iterator(); iterator.hasNext(); ) { + Path path = iterator.next(); + if (path.toFile().isDirectory()) { + continue; + } + + // hudi blows up if crc files are present + if (path.toString().endsWith(".crc")) { + continue; + } + + HashAndSizeResult srcHashAndSize; + try { + srcHashAndSize = calculateHashAndSize(path); + } + catch (NoSuchAlgorithmException e) { + throw new IOException("Failed to calculate source hash: Algorithm not found", e); + } + + Path location = destinationDirectory.appendPath(sourceDirectory.relativize(path).toString()); + fileSystem.createDirectory(location.parentDirectory()); + try (OutputStream out = fileSystem.newOutputFile(location).create()) { + Files.copy(path, out); + // Flush all data before close() to ensure durability + out.flush(); + } + + HashAndSizeResult dstHashAndSize; + try { + dstHashAndSize = calculateHashAndSize(location, fileSystem); + } + catch (NoSuchAlgorithmException e) { + throw new IOException("Failed to calculate destination hash: Algorithm not found", e); + } + catch (IOException e) { + throw new IOException("Failed to read back " + location + " for hash verification", e); + } + + if (!Arrays.equals(srcHashAndSize.hash, dstHashAndSize.hash)) { + // Hashes do not match, file is corrupt or copy failed + String errorMessage = String.format( + "Hash mismatch for file: %s (source size: %d bytes) copied to %s (destination size: %d bytes). Content hashes differ", + path, + srcHashAndSize.size, + location, + dstHashAndSize.size); + throw new IOException(errorMessage); + } + } + } + } + + /** + * Helper method to calculate hash and size from an input stream + */ + private static HashAndSizeResult calculateHashAndSize(InputStream inputStream) + throws IOException, NoSuchAlgorithmException + { + MessageDigest md = MessageDigest.getInstance(HASH_ALGORITHM); + try (DigestInputStream dis = new DigestInputStream(inputStream, md)) { + byte[] buffer = new byte[8192]; + int bytesRead; + long streamSize = 0; + while ((bytesRead = dis.read(buffer)) != -1) { + streamSize += bytesRead; + } + return new HashAndSizeResult(md.digest(), streamSize); + } + } + + /** + * Helper method to calculate hash for a local Path + */ + private static HashAndSizeResult calculateHashAndSize(Path path) + throws IOException, NoSuchAlgorithmException + { + try (InputStream is = Files.newInputStream(path)) { + return calculateHashAndSize(is); + } + } + + /** + * Helper method to calculate hash for a file on TrinoFileSystem + */ + private static HashAndSizeResult calculateHashAndSize(Path location, ExtendedFileSystem fileSystem) + throws IOException, NoSuchAlgorithmException + { + try (InputStream is = fileSystem.newInputFile(location).newStream()) { + return calculateHashAndSize(is); + } + } + + public enum TestingTable + { + HUDI_NON_PART_COW(nonPartitionRegularColumns()), + HUDI_NON_PART_MOR(simpleRegularColumns(), ImmutableList.of(), ImmutableMap.of(), true), + HUDI_TRIPS_COW_V8(tripsRegularColumns()), + HUDI_COW_TABLE_WITH_FIELD_NAMES_IN_CAPS(hudiTableWithFieldNamesInCapsRegularColumns()), + HUDI_COW_PT_TABLE_WITH_FIELD_NAMES_IN_CAPS(hudiTableWithFieldNamesInCapsRegularColumns(), hudiTableWithFieldNamesInCapsPartitionColumns(), hudiTableWithFieldNamesInCapsPartitions(), false), // delete + HUDI_COW_TABLE_WITH_MULTI_KEYS_AND_FIELD_NAMES_IN_CAPS(hudiTableWithFieldNamesInCapsRegularColumns()), + HUDI_MOR_TABLE_WITH_FIELD_NAMES_IN_CAPS(hudiTableWithFieldNamesInCapsRegularColumns(), ImmutableList.of(), ImmutableMap.of(), true), + HUDI_COW_PT_TBL(multiPartitionRegularColumns(), multiPartitionColumns(), multiPartitions(), false), + STOCK_TICKS_COW(stockTicksRegularColumns(), stockTicksPartitionColumns(), stockTicksPartitions(), false), + STOCK_TICKS_MOR(stockTicksRegularColumns(), stockTicksPartitionColumns(), stockTicksPartitions(), false), + HUDI_STOCK_TICKS_COW(hudiStockTicksRegularColumns(), hudiStockTicksPartitionColumns(), hudiStockTicksPartitions(), false), + HUDI_STOCK_TICKS_MOR(hudiStockTicksRegularColumns(), hudiStockTicksPartitionColumns(), hudiStockTicksPartitions(), false), + HUDI_MULTI_FG_PT_V6_MOR(hudiMultiFgRegularColumns(), hudiMultiFgPartitionsColumn(), hudiMultiFgPartitions(), false), + HUDI_MULTI_FG_PT_V8_MOR(hudiMultiFgRegularColumns(), hudiMultiFgPartitionsColumn(), hudiMultiFgPartitions(), false), + HUDI_COMPREHENSIVE_TYPES_V6_MOR(hudiComprehensiveTypesColumns(), hudiComprehensiveTypesPartitionColumns(), hudiComprehensiveTypesPartitions(), true), + HUDI_COMPREHENSIVE_TYPES_V8_MOR(hudiComprehensiveTypesColumns(), hudiComprehensiveTypesPartitionColumns(), hudiComprehensiveTypesPartitions(), true), + HUDI_MULTI_PT_V8_MOR(hudiMultiPtMorColumns(), hudiMultiPtMorPartitionColumns(), hudiMultiPtMorPartitions(), false), + HUDI_TIMESTAMP_KEYGEN_PT_EPOCH_TO_YYYY_MM_DD_HH_V8_MOR(hudiTimestampKeygenColumns(), hudiTimestampKeygenPartitionColumns(), hudiTimestampKeygenPartitions("EPOCHMILLISECONDS"), true), + HUDI_TIMESTAMP_KEYGEN_PT_SCALAR_TO_YYYY_MM_DD_HH_V8_MOR(hudiTimestampKeygenColumns(), hudiTimestampKeygenPartitionColumns(), hudiTimestampKeygenPartitions("SCALAR"), true), + HUDI_CUSTOM_KEYGEN_PT_V8_MOR(hudiCustomKeyGenColumns(), hudiCustomKeyGenPartitionColumns(), hudiCustomKeyGenPartitions(), false), + HUDI_NON_EXTRACTABLE_PARTITION_PATH(multiPartitionRegularColumns(), multiPartitionColumns(), multiPartitionsWithNonExtractablePartitionPaths(), false), + /**/; + + private static final List HUDI_META_COLUMNS = ImmutableList.of( + new Column("_hoodie_commit_time", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_commit_seqno", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_record_key", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_partition_path", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_file_name", HIVE_STRING, Optional.empty(), Map.of())); + + private final List regularColumns; + private final List partitionColumns; + private final Map partitions; + private final boolean isCreateRtTable; + private HoodieTableVersion tableVersion; + + TestingTable( + List regularColumns, + List partitionColumns, + Map partitions, + boolean isCreateRtTable) + { + this.regularColumns = regularColumns; + this.partitionColumns = partitionColumns; + this.partitions = partitions; + this.isCreateRtTable = isCreateRtTable; + } + + TestingTable(List regularColumns) + { + this(regularColumns, ImmutableList.of(), ImmutableMap.of(), false); + } + + public String getTableName() + { + return name().toLowerCase(Locale.ROOT); + } + + public String getRtTableName() + { + return name().toLowerCase(Locale.ROOT) + "_rt"; + } + + public String getRoTableName() + { + // ro tables do not have suffix + return getTableName(); + } + + public void setTableVersion(HoodieTableVersion tableVersion) + { + this.tableVersion = tableVersion; + } + + public HoodieTableVersion getHoodieTableVersion() + { + return this.tableVersion; + } + + public List getDataColumns() + { + return Stream.of(HUDI_META_COLUMNS, regularColumns) + .flatMap(Collection::stream) + .collect(Collectors.toUnmodifiableList()); + } + + public List getPartitionColumns() + { + return partitionColumns; + } + + public Map getPartitions() + { + return partitions; + } + + public boolean isCreateRtTable() + { + return isCreateRtTable; + } + + private static List nonPartitionRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_LONG), + column("name", HIVE_STRING), + column("ts", HIVE_LONG), + column("dt", HIVE_STRING), + column("hh", HIVE_STRING)); + } + + // Table schema has capitalized column names, but the catalog returns them in lowercase. + // Using lowercase here to match the catalog for testing. + private static List hudiTableWithFieldNamesInCapsRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_STRING), + column("name", HIVE_STRING), + column("age", HIVE_INT)); + } + + // The actual Hudi table has "Country" as the partition field name, but the catalog provides it in lowercase. + // Using lowercase here to stay consistent with the catalog for testing. + private static Map hudiTableWithFieldNamesInCapsPartitions() + { + return ImmutableMap.of( + "country=IND", "IND", + "country=US", "US"); + } + + // The actual Hudi table has "Country" as the partition field name, but the catalog provides it in lowercase. + // Using lowercase here to stay consistent with the catalog for testing. + private static List hudiTableWithFieldNamesInCapsPartitionColumns() + { + return ImmutableList.of(column("country", HIVE_STRING)); + } + + private static List simpleRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_STRING), + column("name", HIVE_STRING), + column("age", HIVE_INT)); + } + + private static List tripsRegularColumns() + { + return ImmutableList.of( + column("begin_lat", HIVE_DOUBLE), + column("begin_lon", HIVE_DOUBLE), + column("driver", HIVE_STRING), + column("end_lat", HIVE_DOUBLE), + column("end_lon", HIVE_DOUBLE), + column("fare", HIVE_DOUBLE), + column("partitionpath", HIVE_STRING), + column("rider", HIVE_STRING), + column("ts", HIVE_LONG), + column("uuid", HIVE_STRING)); + } + + private static List stockTicksRegularColumns() + { + return ImmutableList.of( + column("volume", HIVE_LONG), + column("ts", HIVE_STRING), + column("symbol", HIVE_STRING), + column("year", HIVE_INT), + column("month", HIVE_STRING), + column("high", HIVE_DOUBLE), + column("low", HIVE_DOUBLE), + column("key", HIVE_STRING), + column("date", HIVE_STRING), + column("close", HIVE_DOUBLE), + column("open", HIVE_DOUBLE), + column("day", HIVE_STRING)); + } + + private static List stockTicksPartitionColumns() + { + return ImmutableList.of(column("dt", HIVE_STRING)); + } + + private static Map stockTicksPartitions() + { + return ImmutableMap.of("dt=2018-08-31", "2018/08/31"); + } + + private static List hudiStockTicksRegularColumns() + { + return ImmutableList.of( + column("volume", HIVE_LONG), + column("ts", HIVE_STRING), + column("symbol", HIVE_STRING), + column("year", HIVE_INT), + column("month", HIVE_STRING), + column("high", HIVE_DOUBLE), + column("low", HIVE_DOUBLE), + column("key", HIVE_STRING), + column("close", HIVE_DOUBLE), + column("open", HIVE_DOUBLE), + column("day", HIVE_STRING)); + } + + private static List hudiStockTicksPartitionColumns() + { + return ImmutableList.of(column("date", HIVE_STRING)); + } + + private static Map hudiStockTicksPartitions() + { + return ImmutableMap.of("date=2018-08-31", "2018/08/31"); + } + + private static List multiPartitionRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_LONG), + column("name", HIVE_STRING), + column("ts", HIVE_LONG)); + } + + private static List multiPartitionColumns() + { + return ImmutableList.of( + column("dt", HIVE_STRING), + column("hh", HIVE_STRING)); + } + + private static Map multiPartitions() + { + return ImmutableMap.of( + "dt=2021-12-09/hh=10", "dt=2021-12-09/hh=10", + "dt=2021-12-09/hh=11", "dt=2021-12-09/hh=11"); + } + + /** + * Returns a sample map of partition specs containing multiple partition keys separated by slashes. + * + * Example: + * "dt=2018-10-05/hh=10" -> "2018/10/05/10" + * + * Note: + * - The partition spec has 2 partition keys (dt, hh). + * - However, the corresponding value string has 4 segments when split by slashes + * (year, month, day, hour). + * - Standard Hudi partition extractors will not be able to correctly parse this mapping, + * since they expect the number of slash-separated values to match the number of partition keys + * if there's more than one partition field. + */ + private static Map multiPartitionsWithNonExtractablePartitionPaths() + { + return ImmutableMap.of( + "dt=2018-10-05/hh=10", "2018/10/05/10", + "dt=2018-10-06/hh=5", "2018/10/06/5"); + } + + private static List hudiMultiFgRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_INT), + column("name", HIVE_STRING), + column("price", HIVE_DOUBLE), + column("ts", HIVE_LONG)); + } + + private static List hudiMultiFgPartitionsColumn() + { + return ImmutableList.of( + column("country", HIVE_STRING)); + } + + private static Map hudiMultiFgPartitions() + { + return ImmutableMap.of( + "country=SG", "country=SG", + "country=US", "country=US"); + } + + private static List hudiComprehensiveTypesColumns() + { + return ImmutableList.of( + // ----- Primary Key & Precombine ----- + column("uuid", HIVE_STRING), + column("precombine_field", HIVE_LONG), + + // ----- Numeric Types ----- + column("col_boolean", HIVE_BOOLEAN), + column("col_tinyint", HIVE_BYTE), + column("col_smallint", HIVE_SHORT), + column("col_int", HIVE_INT), + column("col_bigint", HIVE_LONG), + column("col_float", HIVE_FLOAT), + column("col_double", HIVE_DOUBLE), + column("col_decimal", decimalHiveType(10, 2)), + + // ----- String Types ----- + column("col_string", HIVE_STRING), + column("col_varchar", varcharHiveType(50)), + column("col_char", charHiveType(10)), + + // ----- Binary Type ----- + column("col_binary", HIVE_BINARY), + + // ----- Datetime Types ----- + column("col_date", HIVE_DATE), + column("col_timestamp", HIVE_TIMESTAMP), + + // ----- Complex Types ----- + // ARRAY + column("col_array_int", listHiveType(INT_TYPE_INFO)), + // ARRAY + column("col_array_string", listHiveType(STRING_TYPE_INFO)), + // MAP + column("col_map_string_int", mapHiveType(STRING_TYPE_INFO, INT_TYPE_INFO)), + // STRUCT + column("col_struct", structHiveType( + ImmutableList.of("f1", "f2", "f3"), + ImmutableList.of(STRING_TYPE_INFO, INT_TYPE_INFO, BOOLEAN_TYPE_INFO))), + // ARRAY>> + column("col_array_struct", listHiveType( + getStructTypeInfo( + ImmutableList.of("nested_f1", "nested_f2"), + ImmutableList.of(DOUBLE_TYPE_INFO, ARRAY_STRING_TYPE_INFO)))), + // MAP> + column("col_map_string_struct", mapHiveType( + STRING_TYPE_INFO, + getStructTypeInfo( + ImmutableList.of("nested_f3", "nested_f4"), + ImmutableList.of(DATE_TYPE_INFO, getDecimalTypeInfo(5, 2))))), + // ARRAY>> + column("col_array_struct_with_map", listHiveType( + getStructTypeInfo( + ImmutableList.of("f_arr_struct_str", "f_arr_struct_map"), + ImmutableList.of(STRING_TYPE_INFO, MAP_STRING_INT_TYPE_INFO)))), + // MAP, f_map_struct_ts: TIMESTAMP>> + column("col_map_struct_with_array", mapHiveType( + STRING_TYPE_INFO, + getStructTypeInfo( + ImmutableList.of("f_map_struct_arr", "f_map_struct_ts"), + ImmutableList.of(ARRAY_BOOLEAN_TYPE_INFO, TIMESTAMP_TYPE_INFO)))), + // STRUCT> + column("col_struct_nested_struct", structHiveType( + ImmutableList.of("outer_f1", "nested_struct"), + ImmutableList.of( + INT_TYPE_INFO, + getStructTypeInfo( + ImmutableList.of("inner_f1", "inner_f2"), + ImmutableList.of(STRING_TYPE_INFO, BOOLEAN_TYPE_INFO))))), + // ARRAY> + column("col_array_array_int", listHiveType(ARRAY_INT_TYPE_INFO)), + // MAP> + column("col_map_string_array_double", mapHiveType(STRING_TYPE_INFO, ARRAY_DOUBLE_TYPE_INFO)), + // MAP> + column("col_map_string_map_string_date", mapHiveType(STRING_TYPE_INFO, MAP_STRING_DATE_TYPE_INFO)), + // STRUCT>> + column("col_struct_array_struct", structHiveType( + ImmutableList.of("outer_f2", "struct_array"), + ImmutableList.of( + STRING_TYPE_INFO, + getListTypeInfo(getStructTypeInfo( + ImmutableList.of("inner_f3", "inner_f4"), + ImmutableList.of(TIMESTAMP_TYPE_INFO, STRING_TYPE_INFO)))))), + // STRUCT> + column("col_struct_map", structHiveType( + ImmutableList.of("outer_f3", "struct_map"), + ImmutableList.of(BOOLEAN_TYPE_INFO, MAP_STRING_LONG_TYPE_INFO)))); + } + + private static List hudiComprehensiveTypesPartitionColumns() + { + return ImmutableList.of(column("part_col", HIVE_STRING)); + } + + private static Map hudiComprehensiveTypesPartitions() + { + return ImmutableMap.of( + "part_col=A", "part_col=A", + "part_col=B", "part_col=B"); + } + + private static List hudiMultiPtMorColumns() + { + return ImmutableList.of( + column("id", HIVE_INT), + column("name", HIVE_STRING), + column("price", HIVE_DOUBLE), + column("ts", HIVE_LONG)); + } + + private static List hudiMultiPtMorPartitionColumns() + { + return ImmutableList.of( + column("part_str", HIVE_STRING), + column("part_int", HIVE_INT), + column("part_date", HIVE_DATE), + column("part_bigint", HIVE_LONG), + column("part_decimal", decimalHiveType(10, 2)), + column("part_timestamp", HIVE_TIMESTAMP), + column("part_bool", HIVE_BOOLEAN)); + } + + private static Map hudiMultiPtMorPartitions() + { + return ImmutableMap.of( + "part_str=apparel/part_int=2024/part_date=2024-01-05/part_bigint=20000000001/part_decimal=100.00/part_timestamp=2024-01-05 18%3A00%3A00/part_bool=false", "part_str=apparel/part_int=2024/part_date=2024-01-05/part_bigint=20000000001/part_decimal=100.00/part_timestamp=2024-01-05 18%3A00%3A00/part_bool=false", + "part_str=electronics/part_int=2023/part_date=2023-03-10/part_bigint=10000000002/part_decimal=50.00/part_timestamp=2023-03-10 12%3A30%3A00/part_bool=true", "part_str=electronics/part_int=2023/part_date=2023-03-10/part_bigint=10000000002/part_decimal=50.00/part_timestamp=2023-03-10 12%3A30%3A00/part_bool=true", + "part_str=electronics/part_int=2023/part_date=2023-03-10/part_bigint=10000000002/part_decimal=50.00/part_timestamp=2023-03-10 12%3A30%3A00/part_bool=false", "part_str=electronics/part_int=2023/part_date=2023-03-10/part_bigint=10000000002/part_decimal=50.00/part_timestamp=2023-03-10 12%3A30%3A00/part_bool=false", + "part_str=books/part_int=2023/part_date=2023-01-15/part_bigint=10000000001/part_decimal=123.00/part_timestamp=2023-01-15 10%3A00%3A00/part_bool=true", "part_str=books/part_int=2023/part_date=2023-01-15/part_bigint=10000000001/part_decimal=123.00/part_timestamp=2023-01-15 10%3A00%3A00/part_bool=true", + "part_str=books/part_int=2024/part_date=2024-02-20/part_bigint=10000000003/part_decimal=75.00/part_timestamp=2024-02-20 08%3A45%3A10/part_bool=true", "part_str=books/part_int=2024/part_date=2024-02-20/part_bigint=10000000003/part_decimal=75.00/part_timestamp=2024-02-20 08%3A45%3A10/part_bool=true"); + } + + private static List hudiTimestampKeygenColumns() + { + return ImmutableList.of( + column("id", HIVE_INT), + column("name", HIVE_STRING), + column("price", HIVE_DOUBLE), + column("ts", HIVE_LONG)); + } + + private static List hudiTimestampKeygenPartitionColumns() + { + // Data stored in files are long, but partition value that is synced to metastore is String + return ImmutableList.of(column("partition_field", HIVE_STRING)); + } + + private static Map hudiTimestampKeygenPartitions(String timestampType) + { + return switch (timestampType) { + case "EPOCHMILLISECONDS" -> ImmutableMap.of( + "partition_field=2025-05-13 02", "2025-05-13 02", + "partition_field=2025-06-05 05", "2025-06-05 05", + "partition_field=2025-06-06 09", "2025-06-06 09", + "partition_field=2025-06-06 10", "2025-06-06 10", + "partition_field=2025-06-07 08", "2025-06-07 08"); + case "SCALAR" -> ImmutableMap.of( + "partition_field=2024-10-08 12", "2024-10-08 12", + "partition_field=2024-10-07 12", "2024-10-07 12", + "partition_field=2024-10-06 12", "2024-10-06 12", + "partition_field=2024-10-05 12", "2024-10-05 12", + "partition_field=2024-10-04 12", "2024-10-04 12"); + default -> ImmutableMap.of(); + }; + } + + private static List hudiCustomKeyGenColumns() + { + return hudiMultiFgRegularColumns(); + } + + private static List hudiCustomKeyGenPartitionColumns() + { + return ImmutableList.of( + column("partition_field_country", HIVE_STRING), + column("partition_field_date", HIVE_STRING)); + } + + private static Map hudiCustomKeyGenPartitions() + { + return ImmutableMap.of( + "partition_field_country=US/partition_field_date=2025-06-06", "partition_field_country=US/partition_field_date=2025-06-06", + "partition_field_country=CN/partition_field_date=2025-06-05", "partition_field_country=CN/partition_field_date=2025-06-05", + "partition_field_country=MY/partition_field_date=2025-05-13", "partition_field_country=MY/partition_field_date=2025-05-13", + "partition_field_country=SG/partition_field_date=2025-06-06", "partition_field_country=SG/partition_field_date=2025-06-06", + "partition_field_country=SG/partition_field_date=2025-06-07", "partition_field_country=SG/partition_field_date=2025-06-07"); + } + } + + static class HashAndSizeResult + { + final byte[] hash; + final long size; + + HashAndSizeResult(byte[] hash, long size) + { + this.hash = hash; + this.size = size; + } + } +} \ No newline at end of file