diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 327cdc1b4b052..9357687c86386 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -24,16 +24,12 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; -import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.client.BaseHoodieWriteClient; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.engine.HoodieReaderContext; -import org.apache.hudi.common.engine.ReaderContextFactory; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.function.SerializableFunction; @@ -53,13 +49,10 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.schema.HoodieSchema; -import org.apache.hudi.common.schema.HoodieSchemaCache; -import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; -import org.apache.hudi.common.table.read.HoodieFileGroupReader; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; @@ -71,7 +64,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -79,11 +71,13 @@ import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.TableNotFoundException; -import org.apache.hudi.index.record.HoodieRecordIndex; -import org.apache.hudi.internal.schema.InternalSchema; -import org.apache.hudi.io.storage.HoodieIOFactory; -import org.apache.hudi.internal.schema.utils.SerDeHelper; -import org.apache.hudi.metadata.HoodieTableMetadataUtil.DirectoryInfo; +import org.apache.hudi.metadata.index.ExpressionIndexRecordGenerator; +import org.apache.hudi.metadata.index.model.DataPartitionAndRecords; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.index.Indexer; +import org.apache.hudi.metadata.index.IndexerFactory; +import org.apache.hudi.metadata.model.DirectoryInfo; +import org.apache.hudi.metadata.model.FileSliceAndPartition; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.HoodieStorageUtils; import org.apache.hudi.storage.StorageConfiguration; @@ -93,6 +87,7 @@ import org.apache.hudi.util.Lazy; import lombok.Getter; +import lombok.SneakyThrows; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,26 +95,21 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Queue; import java.util.Set; -import java.util.TreeMap; import java.util.function.Function; -import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_POPULATE_META_FIELDS; -import static org.apache.hudi.common.schema.HoodieSchemaUtils.getRecordKeySchema; import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_HISTORY_PATH; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; @@ -130,24 +120,17 @@ import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX_PREFIX; -import static org.apache.hudi.metadata.HoodieTableMetadataUtil.createRecordIndexDefinition; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.existingIndexVersionOrDefault; -import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getExpressionIndexPartitionsToInit; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getPartitionLatestFileSlicesIncludingInflight; -import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getProjectedSchemaForExpressionIndex; -import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getSecondaryIndexPartitionsToInit; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.readRecordKeysFromBaseFiles; -import static org.apache.hudi.metadata.MetadataPartitionType.BLOOM_FILTERS; import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS; import static org.apache.hudi.metadata.MetadataPartitionType.EXPRESSION_INDEX; import static org.apache.hudi.metadata.MetadataPartitionType.FILES; -import static org.apache.hudi.metadata.MetadataPartitionType.PARTITION_STATS; import static org.apache.hudi.metadata.MetadataPartitionType.RECORD_INDEX; import static org.apache.hudi.metadata.MetadataPartitionType.SECONDARY_INDEX; import static org.apache.hudi.metadata.MetadataPartitionType.fromPartitionPath; import static org.apache.hudi.metadata.SecondaryIndexRecordGenerationUtils.convertWriteStatsToSecondaryIndexRecords; -import static org.apache.hudi.metadata.SecondaryIndexRecordGenerationUtils.readSecondaryKeysFromFileSlices; /** * Writer implementation backed by an internal hudi table. Partition and file listing are saved within an internal MOR table @@ -181,7 +164,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTab private boolean hasPartitionsStateChanged = false; protected final transient HoodieEngineContext engineContext; @Getter - protected final List enabledPartitionTypes; + protected final transient Map enabledIndexerMap; + protected final transient ExpressionIndexRecordGenerator expressionIndexRecordGenerator; // Is the MDT bootstrapped and ready to be read from @Getter @@ -193,8 +177,9 @@ protected HoodieBackedTableMetadataWriter(StorageConfiguration storageConf, HoodieWriteConfig writeConfig, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, + ExpressionIndexRecordGenerator expressionIndexRecordGenerator, Option inflightInstantTimestamp) { - this(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, inflightInstantTimestamp, false); + this(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, expressionIndexRecordGenerator, inflightInstantTimestamp, false); } /** @@ -211,6 +196,7 @@ protected HoodieBackedTableMetadataWriter(StorageConfiguration storageConf, HoodieWriteConfig writeConfig, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, + ExpressionIndexRecordGenerator expressionIndexRecordGenerator, Option inflightInstantTimestamp, boolean streamingWritesEnabled) { this.dataWriteConfig = writeConfig; @@ -220,7 +206,8 @@ protected HoodieBackedTableMetadataWriter(StorageConfiguration storageConf, this.dataMetaClient = HoodieTableMetaClient.builder().setConf(storageConf.newInstance()) .setBasePath(dataWriteConfig.getBasePath()) .setTimeGeneratorConfig(dataWriteConfig.getTimeGeneratorConfig()).build(); - this.enabledPartitionTypes = getEnabledPartitions(dataWriteConfig.getMetadataConfig(), dataMetaClient); + this.enabledIndexerMap = IndexerFactory.getEnabledIndexerMap(engineContext, dataWriteConfig, dataMetaClient, expressionIndexRecordGenerator); + this.expressionIndexRecordGenerator = expressionIndexRecordGenerator; if (writeConfig.isMetadataTableEnabled()) { this.metadataWriteConfig = createMetadataWriteConfig(writeConfig, failedWritesCleaningPolicy, dataMetaClient.getTableConfig().getTableVersion()); try { @@ -234,10 +221,6 @@ protected HoodieBackedTableMetadataWriter(StorageConfiguration storageConf, this.streamingWritesEnabled = streamingWritesEnabled; } - List getEnabledPartitions(HoodieMetadataConfig metadataConfig, HoodieTableMetaClient metaClient) { - return MetadataPartitionType.getEnabledPartitions(metadataConfig, metaClient); - } - private void mayBeReinitMetadataReader() { if (metadata == null || metadataMetaClient == null || metadata.getMetadataFileSystemView() == null) { initMetadataReader(); @@ -286,13 +269,13 @@ public HoodieBackedTableMetadata getTableMetadata() { protected boolean initializeIfNeeded(HoodieTableMetaClient dataMetaClient, Option inflightInstantTimestamp) throws IOException { HoodieTimer timer = HoodieTimer.start(); - List metadataPartitionsToInit = new ArrayList<>(MetadataPartitionType.getValidValues().length); + Map indexerMapForPartitionsToInit = new HashMap<>(); try { boolean exists = metadataTableExists(dataMetaClient); if (!exists) { // FILES partition is always required - metadataPartitionsToInit.add(FILES); + indexerMapForPartitionsToInit.put(FILES, enabledIndexerMap.get(FILES)); } // check if any of the enabled partition types needs to be initialized @@ -300,12 +283,12 @@ protected boolean initializeIfNeeded(HoodieTableMetaClient dataMetaClient, if (!dataWriteConfig.isMetadataAsyncIndex()) { Set completedPartitions = dataMetaClient.getTableConfig().getMetadataPartitions(); LOG.info("Async metadata indexing disabled and following partitions already initialized: {}", completedPartitions); - this.enabledPartitionTypes.stream() - .filter(p -> !completedPartitions.contains(p.getPartitionPath()) && !FILES.equals(p)) - .forEach(metadataPartitionsToInit::add); + this.enabledIndexerMap.entrySet().stream() + .filter(p -> !completedPartitions.contains(p.getKey().getPartitionPath()) && FILES != p.getKey()) + .forEach(e -> indexerMapForPartitionsToInit.put(e.getKey(), e.getValue())); } - if (metadataPartitionsToInit.isEmpty()) { + if (indexerMapForPartitionsToInit.isEmpty()) { // No partitions left to initialize, since all the metadata enabled partitions are either initialized before // or current in the process of initialization. initMetadataReader(); @@ -315,7 +298,7 @@ protected boolean initializeIfNeeded(HoodieTableMetaClient dataMetaClient, // If there is no commit on the dataset yet, use the SOLO_COMMIT_TIMESTAMP as the instant time for initial commit // Otherwise, we use the timestamp of the latest completed action. String dataTableInstantTime = dataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant().map(HoodieInstant::requestedTime).orElse(SOLO_COMMIT_TIMESTAMP); - if (!initializeFromFilesystem(dataTableInstantTime, metadataPartitionsToInit, inflightInstantTimestamp)) { + if (!initializeFromFilesystem(dataTableInstantTime, indexerMapForPartitionsToInit, inflightInstantTimestamp)) { LOG.error("Failed to initialize MDT from filesystem"); return false; } @@ -388,11 +371,13 @@ boolean shouldInitializeFromFilesystem(Set pendingDataInstants, Option partitionsToInit, Option inflightInstantTimestamp) throws IOException { + private boolean initializeFromFilesystem(String dataTableInstantTime, + Map indexerMapForPartitionsToInit, + Option inflightInstantTimestamp) throws IOException { Set pendingDataInstants = getPendingDataInstants(dataMetaClient); if (!shouldInitializeFromFilesystem(pendingDataInstants, inflightInstantTimestamp)) { return false; @@ -401,8 +386,6 @@ private boolean initializeFromFilesystem(String dataTableInstantTime, List dataMetaClient.getTableConfig().isMetadataPartitionAvailable((metadataPartition))); + indexerMapForPartitionsToInit.keySet().removeIf( + metadataPartition -> dataMetaClient.getTableConfig().isMetadataPartitionAvailable(metadataPartition)); // Get a complete list of files and partitions from the file system or from already initialized FILES partition of MDT List partitionInfoList; @@ -432,102 +416,80 @@ private boolean initializeFromFilesystem(String dataTableInstantTime, List> partitionIdToAllFilesMap = partitionInfoList.stream() - .map(p -> { - String partitionName = HoodieTableMetadataUtil.getPartitionIdentifierForFilesPartition(p.getRelativePath()); - return Pair.of(partitionName, p.getFilenameToSizeMap()); - }) - .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - - // validate that each index is eligible to be initialized - Iterator iterator = partitionsToInit.iterator(); - while (iterator.hasNext()) { - MetadataPartitionType partitionType = iterator.next(); - if (partitionType == PARTITION_STATS && !dataMetaClient.getTableConfig().isTablePartitioned()) { - // Partition stats index cannot be enabled for a non-partitioned table - iterator.remove(); - this.enabledPartitionTypes.remove(partitionType); - } + Map> partitionIdToAllFilesMap = DirectoryInfo.getPartitionToFiles(partitionInfoList); + Lazy> lazyLatestMergedPartitionFileSliceList = getLazyLatestMergedPartitionFileSliceList(); + + // FILES partition should always be initialized first if enabled + if (!filesPartitionAvailable) { + initializeMetadataPartition(FILES, indexerMapForPartitionsToInit.get(FILES), + dataTableInstantTime, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + hasPartitionsStateChanged = true; } - Lazy>> lazyLatestMergedPartitionFileSliceList = getLazyLatestMergedPartitionFileSliceList(); - for (MetadataPartitionType partitionType : partitionsToInit) { - // Find the commit timestamp to use for this partition. Each initialization should use its own unique commit time. - String instantTimeForPartition = generateUniqueInstantTime(dataTableInstantTime); - String partitionTypeName = partitionType.name(); - LOG.info("Initializing MDT partition {} at instant {}", partitionTypeName, instantTimeForPartition); - String relativePartitionPath; - Pair> fileGroupCountAndRecordsPair; - Lazy> tableSchema = Lazy.lazily(() -> HoodieTableMetadataUtil.tryResolveSchemaForTable(dataMetaClient)); - try { - switch (partitionType) { - case FILES: - fileGroupCountAndRecordsPair = initializeFilesPartition(partitionIdToAllFilesMap); - initializeFilegroupsAndCommit(partitionType, FILES.getPartitionPath(), fileGroupCountAndRecordsPair, instantTimeForPartition); - break; - case BLOOM_FILTERS: - fileGroupCountAndRecordsPair = initializeBloomFiltersPartition(dataTableInstantTime, partitionIdToAllFilesMap); - initializeFilegroupsAndCommit(partitionType, BLOOM_FILTERS.getPartitionPath(), fileGroupCountAndRecordsPair, instantTimeForPartition); - break; - case COLUMN_STATS: - Pair, Pair>> colStatsColumnsAndRecord = initializeColumnStatsPartition(partitionIdToAllFilesMap, tableSchema); - fileGroupCountAndRecordsPair = colStatsColumnsAndRecord.getValue(); - initializeFilegroupsAndCommit(partitionType, COLUMN_STATS.getPartitionPath(), fileGroupCountAndRecordsPair, instantTimeForPartition, colStatsColumnsAndRecord.getKey()); - break; - case RECORD_INDEX: - boolean isPartitionedRLI = dataWriteConfig.isRecordLevelIndexEnabled(); - initializeFilegroupsAndCommitToRecordIndexPartition(instantTimeForPartition, lazyLatestMergedPartitionFileSliceList, isPartitionedRLI); - break; - case EXPRESSION_INDEX: - Set expressionIndexPartitionsToInit = getExpressionIndexPartitionsToInit(partitionType, dataWriteConfig.getMetadataConfig(), dataMetaClient); - if (expressionIndexPartitionsToInit.size() != 1) { - if (expressionIndexPartitionsToInit.size() > 1) { - LOG.warn("Skipping expression index initialization as only one expression index bootstrap at a time is supported for now. Provided: {}", expressionIndexPartitionsToInit); - } - continue; - } - relativePartitionPath = expressionIndexPartitionsToInit.iterator().next(); - fileGroupCountAndRecordsPair = initializeExpressionIndexPartition(relativePartitionPath, dataTableInstantTime, lazyLatestMergedPartitionFileSliceList, tableSchema); - initializeFilegroupsAndCommit(partitionType, relativePartitionPath, fileGroupCountAndRecordsPair, instantTimeForPartition); - break; - case PARTITION_STATS: - // For PARTITION_STATS, COLUMN_STATS should also be enabled - if (!dataWriteConfig.isMetadataColumnStatsIndexEnabled()) { - LOG.debug("Skipping partition stats initialization as column stats index is not enabled. Please enable {}", - HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key()); - continue; - } - fileGroupCountAndRecordsPair = initializePartitionStatsIndex(lazyLatestMergedPartitionFileSliceList, tableSchema); - initializeFilegroupsAndCommit(partitionType, PARTITION_STATS.getPartitionPath(), fileGroupCountAndRecordsPair, instantTimeForPartition); - break; - case SECONDARY_INDEX: - Set secondaryIndexPartitionsToInit = getSecondaryIndexPartitionsToInit(partitionType, dataWriteConfig.getMetadataConfig(), dataMetaClient); - if (secondaryIndexPartitionsToInit.size() != 1) { - if (secondaryIndexPartitionsToInit.size() > 1) { - LOG.warn("Skipping secondary index initialization as only one secondary index bootstrap at a time is supported for now. Provided: {}", secondaryIndexPartitionsToInit); - } - continue; - } - relativePartitionPath = secondaryIndexPartitionsToInit.iterator().next(); - fileGroupCountAndRecordsPair = initializeSecondaryIndexPartition(relativePartitionPath, lazyLatestMergedPartitionFileSliceList); - initializeFilegroupsAndCommit(partitionType, relativePartitionPath, fileGroupCountAndRecordsPair, instantTimeForPartition); - break; - default: - throw new HoodieMetadataException(String.format("Unsupported MDT partition type: %s", partitionType)); - } - } catch (Exception e) { - String metricKey = partitionType.getPartitionPath() + "_" + HoodieMetadataMetrics.BOOTSTRAP_ERR_STR; - metrics.ifPresent(m -> m.setMetric(metricKey, 1)); - String errMsg = String.format("Bootstrap on %s partition failed for %s", - partitionType.getPartitionPath(), metadataMetaClient.getBasePath()); - LOG.error(errMsg, e); - throw new HoodieMetadataException(errMsg, e); - } + for (Map.Entry entry : + indexerMapForPartitionsToInit.entrySet().stream() + .filter(e -> e.getKey() != FILES).collect(Collectors.toList())) { + initializeMetadataPartition(entry.getKey(), entry.getValue(), + dataTableInstantTime, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); hasPartitionsStateChanged = true; } return true; } + @SneakyThrows + private void initializeMetadataPartition( + MetadataPartitionType partitionType, + Indexer indexer, + String dataTableInstantTime, + Map> partitionToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) { + String instantTimeForPartition = generateUniqueInstantTime(dataTableInstantTime); + // initialize metadata partitions + List initializationList; + try { + initializationList = indexer.buildInitializationData( + dataTableInstantTime, instantTimeForPartition, partitionToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + } catch (Exception e) { + String metricKey = partitionType.getPartitionPath() + "_" + HoodieMetadataMetrics.BOOTSTRAP_ERR_STR; + metrics.ifPresent(m -> m.setMetric(metricKey, 1)); + String errMsg = String.format("Bootstrap on %s partition failed for %s", + partitionType.getPartitionPath(), metadataMetaClient.getBasePath()); + LOG.error(errMsg, e); + throw new HoodieMetadataException(errMsg, e); + } + + if (initializationList.isEmpty()) { + LOG.info("Skip building {} index in metadata table", partitionType.getPartitionPath()); + return; + } + + ValidationUtils.checkArgument(initializationList.size() == 1, + "Only support the initialization of one partition per index type " + + "(HUDI-9358 for the feature support)"); + + IndexPartitionInitialization initialIndexPartitionData = initializationList.get(0); + final int numFileGroup = initialIndexPartitionData.totalFileGroups(); + String relativePartitionPath = initialIndexPartitionData.indexPartitionName(); + LOG.info("Initializing {} index with {} file groups", relativePartitionPath, numFileGroup); + + HoodieTimer partitionInitTimer = HoodieTimer.start(); + clearExistingMetadataPartition(relativePartitionPath); + HoodieData records = engineContext.emptyHoodieData(); + for (DataPartitionAndRecords dataPartitionAndRecords: initialIndexPartitionData.dataPartitionAndRecords()) { + initializeFileGroups(dataMetaClient, partitionType, instantTimeForPartition, + dataPartitionAndRecords.numFileGroups(), relativePartitionPath, dataPartitionAndRecords.dataPartition()); + records = records.union(dataPartitionAndRecords.indexRecords()); + } + + bulkCommit(instantTimeForPartition, relativePartitionPath, records, initialIndexPartitionData.indexParser()); + + indexer.postInitialization(metadataMetaClient, records, numFileGroup, relativePartitionPath); + // initialize metadata reader + initMetadataReader(); + long totalInitTime = partitionInitTimer.endTimer(); + LOG.info("Initializing {} index in metadata table took {} in ms", partitionType, totalInitTime); + } + /** * Updates the list of columns to index with col stats index. * @param columnsToIndex list of columns to index. @@ -560,131 +522,9 @@ String generateUniqueInstantTime(String initializationTime) { } } - private Pair> initializePartitionStatsIndex( - Lazy>> lazyLatestMergedPartitionFileSliceList, - Lazy> tableSchemaOpt) { - HoodieData records = HoodieTableMetadataUtil.convertFilesToPartitionStatsRecords( - engineContext, lazyLatestMergedPartitionFileSliceList.get(), dataWriteConfig.getMetadataConfig(), - dataMetaClient, tableSchemaOpt, Option.of(dataWriteConfig.getRecordMerger().getRecordType())); - final int fileGroupCount = dataWriteConfig.getMetadataConfig().getPartitionStatsIndexFileGroupCount(); - return Pair.of(fileGroupCount, records); - } - - private Pair, Pair>> initializeColumnStatsPartition(Map> partitionIdToAllFilesMap, - Lazy> tableSchema) { - final int fileGroupCount = dataWriteConfig.getMetadataConfig().getColumnStatsIndexFileGroupCount(); - if (partitionIdToAllFilesMap.isEmpty()) { - return Pair.of(Collections.emptyList(), Pair.of(fileGroupCount, engineContext.emptyHoodieData())); - } - HoodieIndexVersion columnStatsIndexVersion = existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, dataMetaClient); - // Find the columns to index - final List columnsToIndex = new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(dataMetaClient.getTableConfig(), - dataWriteConfig.getMetadataConfig(), tableSchema, true, - Option.of(dataWriteConfig.getRecordMerger().getRecordType()), columnStatsIndexVersion).keySet()); - - if (columnsToIndex.isEmpty()) { - // this can only happen if meta fields are disabled and cols to index is not explicitly overridden. - return Pair.of(columnsToIndex, Pair.of(fileGroupCount, engineContext.emptyHoodieData())); - } - - LOG.info("Indexing {} columns for column stats index", columnsToIndex.size()); - - // during initialization, we need stats for base and log files. - HoodieData records = HoodieTableMetadataUtil.convertFilesToColumnStatsRecords(engineContext, Collections.emptyMap(), partitionIdToAllFilesMap, - dataMetaClient, dataWriteConfig.getMetadataConfig(), - dataWriteConfig.getColumnStatsIndexParallelism(), - dataWriteConfig.getMetadataConfig().getMaxReaderBufferSize(), - columnsToIndex); - - return Pair.of(columnsToIndex, Pair.of(fileGroupCount, records)); - } - - private Pair> initializeBloomFiltersPartition(String createInstantTime, Map> partitionIdToAllFilesMap) { - HoodieData records = HoodieTableMetadataUtil.convertFilesToBloomFilterRecords( - engineContext, Collections.emptyMap(), partitionIdToAllFilesMap, createInstantTime, dataMetaClient, - dataWriteConfig.getBloomIndexParallelism(), dataWriteConfig.getBloomFilterType()); - - final int fileGroupCount = dataWriteConfig.getMetadataConfig().getBloomFilterIndexFileGroupCount(); - return Pair.of(fileGroupCount, records); - } - - /** - * Generates expression index records - * - * @param partitionFilePathAndSizeTriplet Triplet of file path, file size and partition name to which file belongs - * @param indexDefinition Hoodie Index Definition for the expression index for which records need to be generated - * @param metaClient Hoodie Table Meta Client - * @param parallelism Parallelism to use for engine operations - * @param tableSchema Schema of the table - * @param readerSchema Schema of reader - * @param storageConf Storage Config - * @param instantTime Instant time - * @return HoodieData wrapper of expression index HoodieRecords - */ - protected abstract HoodieData getExpressionIndexRecords(List>> partitionFilePathAndSizeTriplet, - HoodieIndexDefinition indexDefinition, - HoodieTableMetaClient metaClient, - int parallelism, HoodieSchema tableSchema, HoodieSchema readerSchema, - StorageConfiguration storageConf, - String instantTime); - protected abstract EngineType getEngineType(); - private Pair> initializeExpressionIndexPartition( - String indexName, String dataTableInstantTime, Lazy>> lazyLatestMergedPartitionFileSliceList, - Lazy> tableSchemaOpt) { - HoodieIndexDefinition indexDefinition = getIndexDefinition(indexName); - ValidationUtils.checkState(indexDefinition != null, "Expression Index definition is not present for index " + indexName); - List> partitionFileSlicePairs = lazyLatestMergedPartitionFileSliceList.get(); - List>> partitionFilePathSizeTriplet = new ArrayList<>(); - partitionFileSlicePairs.forEach(entry -> { - if (entry.getValue().getBaseFile().isPresent()) { - partitionFilePathSizeTriplet.add(Pair.of(entry.getKey(), Pair.of(entry.getValue().getBaseFile().get().getPath(), entry.getValue().getBaseFile().get().getFileSize()))); - } - entry.getValue().getLogFiles() - .forEach(hoodieLogFile -> partitionFilePathSizeTriplet.add(Pair.of(entry.getKey(), Pair.of(hoodieLogFile.getPath().toString(), hoodieLogFile.getFileSize())))); - }); - - int fileGroupCount = dataWriteConfig.getMetadataConfig().getExpressionIndexFileGroupCount(); - if (partitionFileSlicePairs.isEmpty()) { - return Pair.of(fileGroupCount, engineContext.emptyHoodieData()); - } - int parallelism = Math.min(partitionFilePathSizeTriplet.size(), dataWriteConfig.getMetadataConfig().getExpressionIndexParallelism()); - HoodieSchema tableSchema = tableSchemaOpt.get().orElseThrow(() -> new HoodieMetadataException("Table schema is not available for expression index initialization")); - HoodieSchema readerSchema = getProjectedSchemaForExpressionIndex(indexDefinition, dataMetaClient, tableSchema); - return Pair.of(fileGroupCount, getExpressionIndexRecords(partitionFilePathSizeTriplet, indexDefinition, dataMetaClient, parallelism, tableSchema, readerSchema, storageConf, dataTableInstantTime)); - } - - HoodieIndexDefinition getIndexDefinition(String indexName) { - return HoodieTableMetadataUtil.getHoodieIndexDefinition(indexName, dataMetaClient); - } - - private Pair> initializeSecondaryIndexPartition( - String indexName, Lazy>> lazyLatestMergedPartitionFileSliceList) { - HoodieIndexDefinition indexDefinition = getIndexDefinition(indexName); - ValidationUtils.checkState(indexDefinition != null, "Secondary Index definition is not present for index " + indexName); - List> partitionFileSlicePairs = lazyLatestMergedPartitionFileSliceList.get(); - - int parallelism = Math.min(partitionFileSlicePairs.size(), dataWriteConfig.getMetadataConfig().getSecondaryIndexParallelism()); - HoodieData records = readSecondaryKeysFromFileSlices( - engineContext, - partitionFileSlicePairs, - parallelism, - this.getClass().getSimpleName(), - dataMetaClient, - indexDefinition, - dataWriteConfig.getProps()); - - // Initialize the file groups - using the same estimation logic as that of record index - final int fileGroupCount = HoodieTableMetadataUtil.estimateFileGroupCount(RECORD_INDEX, records::count, - RECORD_INDEX_AVERAGE_RECORD_SIZE, dataWriteConfig.getGlobalRecordLevelIndexMinFileGroupCount(), - dataWriteConfig.getGlobalRecordLevelIndexMaxFileGroupCount(), dataWriteConfig.getRecordIndexGrowthFactor(), - dataWriteConfig.getRecordIndexMaxFileGroupSizeBytes()); - - return Pair.of(fileGroupCount, records); - } - - private Lazy>> getLazyLatestMergedPartitionFileSliceList() { + private Lazy> getLazyLatestMergedPartitionFileSliceList() { return Lazy.lazily(() -> { String latestInstant = dataMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant() .map(HoodieInstant::requestedTime).orElse(SOLO_COMMIT_TIMESTAMP); @@ -692,9 +532,9 @@ private Lazy>> getLazyLatestMergedPartitionFileSlic // Collect the list of latest file slices present in each partition List partitions = metadata.getAllPartitionPaths(); fsView.loadAllPartitions(); - List> partitionFileSlicePairs = new ArrayList<>(); + List partitionFileSlicePairs = new ArrayList<>(); partitions.forEach(partition -> fsView.getLatestMergedFileSlicesBeforeOrOn(partition, latestInstant) - .forEach(fs -> partitionFileSlicePairs.add(Pair.of(partition, fs)))); + .forEach(fs -> partitionFileSlicePairs.add(FileSliceAndPartition.of(partition, fs)))); return partitionFileSlicePairs; } catch (IOException e) { throw new HoodieIOException("Cannot get the latest merged file slices", e); @@ -702,318 +542,6 @@ private Lazy>> getLazyLatestMergedPartitionFileSlic }); } - void initializeFilegroupsAndCommit(MetadataPartitionType partitionType, - String relativePartitionPath, - Pair> fileGroupCountAndRecordsPair, - String instantTimeForPartition) throws IOException { - initializeFilegroupsAndCommit(partitionType, relativePartitionPath, fileGroupCountAndRecordsPair, - instantTimeForPartition, Collections.emptyList()); - } - - void initializeFilegroupsAndCommit(MetadataPartitionType partitionType, - String relativePartitionPath, - Pair> fileGroupCountAndRecordsPair, - String instantTimeForPartition, - List columnsToIndex) throws IOException { - String partitionTypeName = partitionType.name(); - LOG.info("Initializing {} index with {} mappings", partitionTypeName, fileGroupCountAndRecordsPair.getKey()); - HoodieTimer partitionInitTimer = HoodieTimer.start(); - - // Generate the file groups - final int fileGroupCount = fileGroupCountAndRecordsPair.getKey(); - ValidationUtils.checkArgument(fileGroupCount > 0, "FileGroup count for MDT partition " + partitionTypeName + " should be > 0"); - clearExistingMetadataPartition(relativePartitionPath); - initializeFileGroups(dataMetaClient, partitionType, instantTimeForPartition, fileGroupCount, relativePartitionPath, Option.empty()); - - // Perform the commit using bulkCommit - HoodieData records = fileGroupCountAndRecordsPair.getValue(); - bulkCommit(instantTimeForPartition, relativePartitionPath, records, new DefaultMetadataTableFileGroupIndexParser(fileGroupCount)); - if (partitionType == COLUMN_STATS) { - // initialize Col Stats index definition - updateColumnsToIndexWithColStats(columnsToIndex); - } - dataMetaClient.getTableConfig().setMetadataPartitionState(dataMetaClient, relativePartitionPath, true); - // initialize the metadata reader again so the MDT partition can be read after initialization - initMetadataReader(); - long totalInitTime = partitionInitTimer.endTimer(); - LOG.info("Initializing {} index in metadata table took {} in ms", partitionTypeName, totalInitTime); - } - - private void initializeFilegroupsAndCommitToRecordIndexPartition(String commitTimeForPartition, - Lazy>> lazyLatestMergedPartitionFileSliceList, boolean isPartitionedRLI) throws IOException { - createRecordIndexDefinition(dataMetaClient, Collections.singletonMap(HoodieRecordIndex.IS_PARTITIONED_OPTION, String.valueOf(isPartitionedRLI))); - HoodieData recordIndexRecords; - int fileGroupCount; - if (isPartitionedRLI) { - Pair> fgCountAndRecords = initializeFilegroupsAndCommitToPartitionedRecordIndexPartition(commitTimeForPartition, lazyLatestMergedPartitionFileSliceList); - fileGroupCount = fgCountAndRecords.getKey(); - recordIndexRecords = fgCountAndRecords.getValue(); - } else { - Pair> fgCountAndRecordIndexRecords = initializeRecordIndexPartition(lazyLatestMergedPartitionFileSliceList.get(), - dataWriteConfig.getMetadataConfig().getRecordIndexMaxParallelism()); - fileGroupCount = fgCountAndRecordIndexRecords.getKey(); - recordIndexRecords = fgCountAndRecordIndexRecords.getRight(); - initializeFilegroupsAndCommit(RECORD_INDEX, RECORD_INDEX.getPartitionPath(), fgCountAndRecordIndexRecords, commitTimeForPartition); - } - // Validate record index after commit if validation is enabled - if (dataWriteConfig.getMetadataConfig().isRecordIndexInitializationValidationEnabled()) { - validateRecordIndex(recordIndexRecords, fileGroupCount); - } - recordIndexRecords.unpersist(); - } - - private Pair> initializeFilegroupsAndCommitToPartitionedRecordIndexPartition(String commitTimeForPartition, - Lazy>> lazyLatestMergedPartitionFileSliceList) throws IOException { - Map>> partitionFileSlicePairsMap = lazyLatestMergedPartitionFileSliceList.get().stream() - .collect(Collectors.groupingBy(Pair::getKey)); - Map>> fileGroupCountAndRecordsPairMap = new HashMap<>(partitionFileSlicePairsMap.size()); - int maxParallelismPerHudiPartition = partitionFileSlicePairsMap.isEmpty() ? 1 : Math.max(1, dataWriteConfig.getMetadataConfig().getRecordIndexMaxParallelism() / partitionFileSlicePairsMap.size()); - for (String partition : partitionFileSlicePairsMap.keySet()) { - LOG.info("Initializing partitioned record index from data partition {}", partition); - fileGroupCountAndRecordsPairMap.put(partition, initializeRecordIndexPartition(partitionFileSlicePairsMap.get(partition), maxParallelismPerHudiPartition)); - } - - int totalFileGroupCount = fileGroupCountAndRecordsPairMap.values().stream().mapToInt(Pair::getLeft).sum(); - if (LOG.isInfoEnabled()) { - LOG.info("Initializing partitioned record index with {} mappings", totalFileGroupCount); - } - - HoodieTimer partitionInitTimer = HoodieTimer.start(); - - // Generate the file groups - HoodieData records = engineContext.emptyHoodieData(); - clearExistingMetadataPartition(RECORD_INDEX.getPartitionPath()); - TreeMap partitionSizes = new TreeMap<>(); - for (String dataPartition : fileGroupCountAndRecordsPairMap.keySet()) { - Pair> fileGroupCountAndRecordsPair = fileGroupCountAndRecordsPairMap.get(dataPartition); - ValidationUtils.checkArgument(fileGroupCountAndRecordsPair.getKey() > 0, "FileGroup count for partitioned RLI data partition " + dataPartition + " should be > 0"); - partitionSizes.put(dataPartition, fileGroupCountAndRecordsPair.getKey()); - initializeFileGroups(dataMetaClient, RECORD_INDEX, commitTimeForPartition, fileGroupCountAndRecordsPair.getKey(), RECORD_INDEX.getPartitionPath(), Option.of(dataPartition)); - records = records.union(fileGroupCountAndRecordsPair.getValue()); - } - - // Perform the commit using bulkCommit - bulkCommit(commitTimeForPartition, RECORD_INDEX.getPartitionPath(), records, new BucketizedMetadataTableFileGroupIndexParser(partitionSizes)); - dataMetaClient.getTableConfig().setMetadataPartitionState(dataMetaClient, RECORD_INDEX.getPartitionPath(), true); - // initialize the metadata reader again so the MDT partition can be read after initialization - initMetadataReader(); - long totalInitTime = partitionInitTimer.endTimer(); - LOG.info("Initializing partitioned record index in metadata table took {} in ms", totalInitTime); - return Pair.of(totalFileGroupCount, records); - } - - private Pair> initializeRecordIndexPartition( - List> latestMergedPartitionFileSliceList, - int recordIndexMaxParallelism) { - LOG.info("Initializing record index from {} file slices", latestMergedPartitionFileSliceList.size()); - HoodieData records = readRecordKeysFromFileSliceSnapshot( - engineContext, - latestMergedPartitionFileSliceList, - recordIndexMaxParallelism, - this.getClass().getSimpleName(), - dataMetaClient, - dataWriteConfig); - - // Initialize the file groups - final int fileGroupCount = estimateFileGroupCount(records); - LOG.info("Initializing record index with {} file groups.", fileGroupCount); - return Pair.of(fileGroupCount, records); - } - - private int estimateFileGroupCount(HoodieData records) { - int minFileGroupCount; - int maxFileGroupCount; - if (dataWriteConfig.isRecordLevelIndexEnabled()) { - minFileGroupCount = dataWriteConfig.getRecordLevelIndexMinFileGroupCount(); - maxFileGroupCount = dataWriteConfig.getRecordLevelIndexMaxFileGroupCount(); - } else { - minFileGroupCount = dataWriteConfig.getGlobalRecordLevelIndexMinFileGroupCount(); - maxFileGroupCount = dataWriteConfig.getGlobalRecordLevelIndexMaxFileGroupCount(); - } - Supplier recordCountSupplier = () -> { - records.persist("MEMORY_AND_DISK_SER"); - long count = records.count(); - LOG.info("Initializing record index with {} mappings", count); - return count; - }; - return HoodieTableMetadataUtil.estimateFileGroupCount( - MetadataPartitionType.RECORD_INDEX, - recordCountSupplier, - RECORD_INDEX_AVERAGE_RECORD_SIZE, - minFileGroupCount, - maxFileGroupCount, - dataWriteConfig.getRecordIndexGrowthFactor(), - dataWriteConfig.getRecordIndexMaxFileGroupSizeBytes() - ); - } - - /** - * Validates the record index after bootstrap by comparing the expected record count with the actual - * record count stored in the metadata table. The validation is performed in a distributed manner - * using the engine context to count records from HFiles in parallel. - * - * @param recordIndexRecords the HoodieData containing the expected records - * @param fileGroupCount the expected number of file groups - */ - private void validateRecordIndex(HoodieData recordIndexRecords, int fileGroupCount) { - String partitionName = MetadataPartitionType.RECORD_INDEX.getPartitionPath(); - HoodieTableFileSystemView fsView = HoodieTableMetadataUtil.getFileSystemViewForMetadataTable(metadataMetaClient); - try { - // Use merged file slices to handle cases with pending compactions - List fileSlices = HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, fsView, partitionName); - - // Filter to only file slices with base files and extract their storage paths - List baseFilePaths = fileSlices.stream() - .filter(fs -> fs.getBaseFile().isPresent()) - .map(fs -> fs.getBaseFile().get().getStoragePath()) - .collect(Collectors.toList()); - - // Count records in a distributed manner using the engine context - long totalRecords = countRecordsInHFiles(baseFilePaths); - long expectedRecordCount = recordIndexRecords.count(); - - ValidationUtils.checkArgument(totalRecords == expectedRecordCount, "Record Count Validation failed with " - + totalRecords + " present in record index vs the expected " + expectedRecordCount); - LOG.info(String.format("Record index initialized on %d shards (expected = %d) with %d records (expected = %d)", - fileSlices.size(), fileGroupCount, totalRecords, expectedRecordCount)); - } finally { - fsView.close(); - } - } - - /** - * Counts the total number of records in HFiles in a distributed manner. - * - * @param baseFilePaths list of storage paths to HFiles - * @return total number of records across all HFiles - */ - private long countRecordsInHFiles(List baseFilePaths) { - if (baseFilePaths.isEmpty()) { - return 0L; - } - - int parallelism = Math.min(baseFilePaths.size(), dataWriteConfig.getMetadataConfig().getRecordIndexMaxParallelism()); - StorageConfiguration storageConfBroadcast = storageConf; - HoodieFileFormat baseFileFormat = metadataMetaClient.getTableConfig().getBaseFileFormat(); - - return engineContext.parallelize(baseFilePaths, parallelism) - .mapPartitions(pathIterator -> { - long count = 0L; - while (pathIterator.hasNext()) { - StoragePath path = pathIterator.next(); - try { - HoodieStorage storage = HoodieStorageUtils.getStorage(path, storageConfBroadcast); - HoodieConfig readerConfig = new HoodieConfig(); - HoodieAvroFileReader reader = (HoodieAvroFileReader) HoodieIOFactory.getIOFactory(storage) - .getReaderFactory(HoodieRecord.HoodieRecordType.AVRO) - .getFileReader(readerConfig, path, baseFileFormat, Option.empty()); - try { - count += reader.getTotalRecords(); - } finally { - reader.close(); - } - } catch (IOException e) { - throw new HoodieIOException("Error reading total records from file " + path, e); - } - } - return Collections.singletonList(count).iterator(); - }, true) - .collectAsList() - .stream() - .mapToLong(Long::longValue) - .sum(); - } - - /** - * Fetch record locations from FileSlice snapshot. - * - * @param engineContext context ot use. - * @param partitionFileSlicePairs list of pairs of partition and file slice. - * @param recordIndexMaxParallelism parallelism to use. - * @param activeModule active module of interest. - * @param metaClient metaclient instance to use. - * @param dataWriteConfig write config to use. - * @return - */ - private static HoodieData readRecordKeysFromFileSliceSnapshot(HoodieEngineContext engineContext, - List> partitionFileSlicePairs, - int recordIndexMaxParallelism, - String activeModule, - HoodieTableMetaClient metaClient, - HoodieWriteConfig dataWriteConfig) { - if (partitionFileSlicePairs.isEmpty()) { - return engineContext.emptyHoodieData(); - } - - Option instantTime = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants() - .lastInstant() - .map(HoodieInstant::requestedTime); - if (!instantTime.isPresent()) { - return engineContext.emptyHoodieData(); - } - - engineContext.setJobStatus(activeModule, "Record Index: reading record keys from " + partitionFileSlicePairs.size() + " file slices"); - final int parallelism = Math.min(partitionFileSlicePairs.size(), recordIndexMaxParallelism); - ReaderContextFactory readerContextFactory = engineContext.getReaderContextFactory(metaClient); - return engineContext.parallelize(partitionFileSlicePairs, parallelism).flatMap(partitionAndFileSlice -> { - final String partition = partitionAndFileSlice.getKey(); - final FileSlice fileSlice = partitionAndFileSlice.getValue(); - final String fileId = fileSlice.getFileId(); - HoodieReaderContext readerContext = readerContextFactory.getContext(); - HoodieSchema dataSchema = HoodieSchemaCache.intern(HoodieSchemaUtils.addMetadataFields(HoodieSchema.parse(dataWriteConfig.getWriteSchema()), dataWriteConfig.allowOperationMetadataField())); - HoodieSchema requestedSchema = metaClient.getTableConfig().populateMetaFields() ? getRecordKeySchema() - : HoodieSchemaUtils.projectSchema(dataSchema, Arrays.asList(metaClient.getTableConfig().getRecordKeyFields().orElse(new String[0]))); - Option internalSchemaOption = SerDeHelper.fromJson(dataWriteConfig.getInternalSchema()); - HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() - .withReaderContext(readerContext) - .withHoodieTableMetaClient(metaClient) - .withFileSlice(fileSlice) - .withLatestCommitTime(instantTime.get()) - .withDataSchema(dataSchema) - .withRequestedSchema(requestedSchema) - .withInternalSchema(internalSchemaOption) - .withShouldUseRecordPosition(false) - .withProps(metaClient.getTableConfig().getProps()) - .build(); - String baseFileInstantTime = fileSlice.getBaseInstantTime(); - return new CloseableMappingIterator<>(fileGroupReader.getClosableIterator(), record -> { - String recordKey = readerContext.getRecordContext().getRecordKey(record, requestedSchema); - return HoodieMetadataPayload.createRecordIndexUpdate(recordKey, partition, fileId, - baseFileInstantTime, 0); - }); - }); - } - - private Pair> initializeFilesPartition(Map> partitionIdToAllFilesMap) { - // FILES partition uses a single file group - final int fileGroupCount = 1; - - Set partitions = partitionIdToAllFilesMap.keySet(); - final int totalDataFilesCount = partitionIdToAllFilesMap.values().stream().mapToInt(Map::size).sum(); - LOG.info("Committing total {} partitions and {} files to metadata", partitions.size(), totalDataFilesCount); - - // Record which saves the list of all partitions - HoodieRecord record = HoodieMetadataPayload.createPartitionListRecord(partitions); - HoodieData allPartitionsRecord = engineContext.parallelize(Collections.singletonList(record), 1); - if (partitionIdToAllFilesMap.isEmpty()) { - return Pair.of(fileGroupCount, allPartitionsRecord); - } - - // Records which save the file listing of each partition - engineContext.setJobStatus(this.getClass().getSimpleName(), "Creating records for metadata FILES partition"); - HoodieData fileListRecords = engineContext.parallelize( - new ArrayList<>(partitionIdToAllFilesMap.entrySet()), partitionIdToAllFilesMap.size()) - .map(partitionInfo -> { - Map fileNameToSizeMap = partitionInfo.getValue(); - return HoodieMetadataPayload.createPartitionFilesRecord( - partitionInfo.getKey(), fileNameToSizeMap, Collections.emptyList()); - }); - ValidationUtils.checkState(fileListRecords.count() == partitions.size()); - - return Pair.of(fileGroupCount, allPartitionsRecord.union(fileListRecords)); - } - private Set getPendingDataInstants(HoodieTableMetaClient dataMetaClient) { // Initialize excluding the pending operations on the dataset return dataMetaClient.getActiveTimeline() @@ -1182,7 +710,7 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata }, fileGroupFileIds.size()); } - void clearExistingMetadataPartition(String relativePartitionPath) throws IOException { + private void clearExistingMetadataPartition(String relativePartitionPath) throws IOException { // Remove all existing file groups or leftover files in the partition final StoragePath partitionPath = new StoragePath(metadataWriteConfig.getBasePath(), relativePartitionPath); HoodieStorage storage = metadataMetaClient.getStorage(); @@ -1270,7 +798,7 @@ private Set getMetadataPartitionsToUpdate() { } // fallback to all enabled partitions if table config returned no partitions LOG.debug("There are no partitions to update according to table config. Falling back to enabled partition types in the write config."); - return getEnabledPartitionTypes().stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); + return enabledIndexerMap.keySet().stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); } public void buildMetadataPartitions(HoodieEngineContext engineContext, List indexPartitionInfos, String instantTime) throws IOException { @@ -1288,7 +816,7 @@ public void buildMetadataPartitions(HoodieEngineContext engineContext, List streamWriteToMetadataPartitions(HoodieData, Set> getStreamingMetadataPartitionsToUpdate() { Set mdtPartitionsToTag = new HashSet<>(); // Add record index - if (enabledPartitionTypes.contains(RECORD_INDEX)) { + if (enabledIndexerMap.containsKey(RECORD_INDEX)) { mdtPartitionsToTag.add(RECORD_INDEX); } // Add secondary indexes @@ -1455,7 +983,7 @@ private HoodieData prepareAndWriteToNonStreamingPartitions(HoodieCo private Set getNonStreamingMetadataPartitionsToUpdate() { Set toReturn = new HashSet<>(); Set streamingMDTPartitions = new HashSet<>(getStreamingMetadataPartitionsToUpdate().getLeft()); - for (MetadataPartitionType partitionType: enabledPartitionTypes) { + for (MetadataPartitionType partitionType: enabledIndexerMap.keySet()) { if (!streamingMDTPartitions.contains(partitionType)) { toReturn.add(partitionType.getPartitionPath()); } @@ -1672,7 +1200,7 @@ private HoodieData getSecondaryIndexUpdates(HoodieCommitMetadata c if (allWriteStats.isEmpty() || WriteOperationType.isCompactionOrClustering(commitMetadata.getOperationType())) { return engineContext.emptyHoodieData(); } - HoodieIndexDefinition indexDefinition = getIndexDefinition(indexPartition); + HoodieIndexDefinition indexDefinition = HoodieTableMetadataUtil.getHoodieIndexDefinition(indexPartition, dataMetaClient); return convertWriteStatsToSecondaryIndexRecords(allWriteStats, instantTime, indexDefinition, dataWriteConfig.getMetadataConfig(), dataMetaClient, engineContext, dataWriteConfig); } @@ -1685,8 +1213,8 @@ private HoodieData getSecondaryIndexUpdates(HoodieCommitMetadata c @Override public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { mayBeReinitMetadataReader(); - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, - cleanMetadata, instantTime, dataMetaClient, dataWriteConfig.getMetadataConfig(), enabledPartitionTypes, + processAndCommit(instantTime, () -> HoodieMetadataWriteUtils.convertMetadataToRecords(engineContext, + cleanMetadata, instantTime, dataMetaClient, dataWriteConfig.getMetadataConfig(), enabledIndexerMap, dataWriteConfig.getBloomIndexParallelism(), Option.of(dataWriteConfig.getRecordMerger().getRecordType()))); closeInternal(); } @@ -1794,7 +1322,6 @@ static Map> convertToColumnStatsRecord(Map records = HoodieTableMetadataUtil.convertFilesToColumnStatsRecords( engineContext, partitionFilesToDelete, partitionFilesToAdd, dataMetaClient, - metadataConfig, columnStatsIndexParallelism, metadataConfig.getMaxReaderBufferSize(), columnsToIndex); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriterTableVersionSix.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriterTableVersionSix.java index a7f51400300ec..1e187986fea4f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriterTableVersionSix.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriterTableVersionSix.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.metadata.index.ExpressionIndexRecordGenerator; import org.apache.hudi.storage.StorageConfiguration; import java.util.Arrays; @@ -72,17 +73,9 @@ protected HoodieBackedTableMetadataWriterTableVersionSix(StorageConfiguration HoodieWriteConfig writeConfig, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, + ExpressionIndexRecordGenerator expressionIndexRecordGenerator, Option inflightInstantTimestamp) { - super(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, inflightInstantTimestamp); - } - - @Override - List getEnabledPartitions(HoodieMetadataConfig metadataConfig, HoodieTableMetaClient metaClient) { - return MetadataPartitionType.getEnabledPartitions(metadataConfig, metaClient).stream() - .filter(partition -> !partition.equals(MetadataPartitionType.SECONDARY_INDEX)) - .filter(partition -> !partition.equals(MetadataPartitionType.EXPRESSION_INDEX)) - .filter(partition -> !partition.equals(MetadataPartitionType.PARTITION_STATS)) - .collect(Collectors.toList()); + super(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, expressionIndexRecordGenerator, inflightInstantTimestamp); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java index 571d19b2ea223..dfcbe37a4ef4d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi.metadata; +import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.client.FailOnFirstErrorWriteStatus; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; @@ -73,6 +74,7 @@ import org.apache.hudi.config.metrics.HoodieMetricsPrometheusConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.metadata.index.Indexer; import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; @@ -106,6 +108,7 @@ import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_PARTITION_STATS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToBloomFilterRecords; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToColumnStatsRecords; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToExpressionIndexRecords; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToFilesPartitionRecords; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToPartitionStatsRecords; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToRecordIndexRecords; @@ -362,6 +365,41 @@ public static HoodieWriteConfig createMetadataWriteConfig( return metadataWriteConfig; } + /** + * Convert the clean action to metadata records. + */ + public static Map> convertMetadataToRecords(HoodieEngineContext engineContext, + HoodieCleanMetadata cleanMetadata, + String instantTime, + HoodieTableMetaClient dataMetaClient, + HoodieMetadataConfig metadataConfig, + Map enabledIndexBuilderMap, + int bloomIndexParallelism, + Option recordTypeOpt) { + final Map> partitionToRecordsMap = new HashMap<>(); + final HoodieData filesPartitionRecordsRDD = engineContext.parallelize( + convertMetadataToFilesPartitionRecords(cleanMetadata, instantTime), 1); + partitionToRecordsMap.put(MetadataPartitionType.FILES.getPartitionPath(), filesPartitionRecordsRDD); + if (enabledIndexBuilderMap.containsKey(MetadataPartitionType.BLOOM_FILTERS)) { + final HoodieData metadataBloomFilterRecordsRDD = + convertMetadataToBloomFilterRecords(cleanMetadata, engineContext, instantTime, bloomIndexParallelism); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS.getPartitionPath(), metadataBloomFilterRecordsRDD); + } + + if (enabledIndexBuilderMap.containsKey(MetadataPartitionType.COLUMN_STATS)) { + final HoodieData metadataColumnStatsRDD = + convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, + dataMetaClient, metadataConfig, recordTypeOpt); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS.getPartitionPath(), metadataColumnStatsRDD); + } + if (enabledIndexBuilderMap.containsKey(MetadataPartitionType.EXPRESSION_INDEX)) { + convertMetadataToExpressionIndexRecords(engineContext, cleanMetadata, instantTime, dataMetaClient, metadataConfig, bloomIndexParallelism, partitionToRecordsMap, + recordTypeOpt); + } + + return partitionToRecordsMap; + } + /** * Convert commit action to metadata records for the enabled partition types. * @@ -370,12 +408,14 @@ public static HoodieWriteConfig createMetadataWriteConfig( * @param commitMetadata - Commit action metadata * @param instantTime - Action instant time * @param dataMetaClient - HoodieTableMetaClient for data - * @param tableMetadata + * @param tableMetadata - metadata table reader * @param metadataConfig - HoodieMetadataConfig * @param enabledPartitionTypes - Set of enabled MDT partitions to update * @param bloomFilterType - Type of generated bloom filter records * @param bloomIndexParallelism - Parallelism for bloom filter record generation - * @param enableOptimizeLogBlocksScan - flag used to enable scanInternalV2 for log blocks in data table + * @param writesFileIdEncoding - file id encoding used while generating record index records + * @param engineType - execution engine type + * @param recordTypeOpt - record type override for generated metadata records * @return Map of partition to metadata records for the commit action */ public static Map> convertMetadataToRecords(HoodieEngineContext context, HoodieWriteConfig dataWriteConfig, HoodieCommitMetadata commitMetadata, @@ -555,8 +595,10 @@ public static Set getFilesToFetchColumnStats(List parti // Get the latest merged file slices based on the commited files part of the latest snapshot and the new files of the current commit metadata List consolidatedPathInfos = new ArrayList<>(); partitionedWriteStat.forEach( - stat -> consolidatedPathInfos.add( - new StoragePathInfo(new StoragePath(dataMetaClient.getBasePath(), stat.getPath()), stat.getFileSizeInBytes(), false, (short) 0, 0, 0))); + stat -> { + StoragePathInfo pathInfo = new StoragePathInfo(new StoragePath(dataMetaClient.getBasePath(), stat.getPath()), stat.getFileSizeInBytes(), false, (short) 0, 0, 0); + consolidatedPathInfos.add(pathInfo); + }); SyncableFileSystemView fileSystemViewForCommitedFiles = FileSystemViewManager.createViewManager(new HoodieLocalEngineContext(dataMetaClient.getStorageConf()), dataWriteConfig.getMetadataConfig(), dataWriteConfig.getViewStorageConfig(), dataWriteConfig.getCommonConfig(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java index 9cb9b8ca7df5f..377c71ea40344 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.engine.ReaderContextFactory; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.metadata.model.FileSliceAndPartition; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieIndexDefinition; @@ -235,7 +236,7 @@ public static Map getRecordKeyToSecondaryKey(HoodieTableMeta } public static HoodieData readSecondaryKeysFromFileSlices(HoodieEngineContext engineContext, - List> partitionFileSlicePairs, + List partitionFileSlicePairs, int secondaryIndexMaxParallelism, String activeModule, HoodieTableMetaClient metaClient, HoodieIndexDefinition indexDefinition, @@ -255,8 +256,8 @@ public static HoodieData readSecondaryKeysFromFileSlices(Hoodi engineContext.setJobStatus(activeModule, "Secondary Index: reading secondary keys from " + partitionFileSlicePairs.size() + " file slices"); HoodieFileFormat baseFileFormat = metaClient.getTableConfig().getBaseFileFormat(); return engineContext.parallelize(partitionFileSlicePairs, parallelism).flatMap(partitionAndBaseFile -> { - final String partition = partitionAndBaseFile.getKey(); - final FileSlice fileSlice = partitionAndBaseFile.getValue(); + final String partition = partitionAndBaseFile.partitionPath(); + final FileSlice fileSlice = partitionAndBaseFile.fileSlice(); Option dataFilePath = Option.ofNullable(fileSlice.getBaseFile().map(baseFile -> FSUtils.getAbsoluteFilePath(basePath, partition, baseFile.getFileName())).orElseGet(null)); HoodieSchema readerSchema; if (dataFilePath.isPresent()) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/BaseIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/BaseIndexer.java new file mode 100644 index 0000000000000..dcbee7f332b93 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/BaseIndexer.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; + +import lombok.extern.slf4j.Slf4j; + +/** + * Base implementation of {@link Indexer} that handles common metadata-partition bootstrap flow, + * including file-group initialization, commit, and partition state update. + */ +@Slf4j +public abstract class BaseIndexer implements Indexer { + protected final HoodieEngineContext engineContext; + protected final HoodieWriteConfig dataTableWriteConfig; + protected final HoodieTableMetaClient dataTableMetaClient; + + protected BaseIndexer( + HoodieEngineContext engineContext, + HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient) { + this.engineContext = engineContext; + this.dataTableWriteConfig = dataTableWriteConfig; + this.dataTableMetaClient = dataTableMetaClient; + } + + /** + * Hook invoked after the bootstrap bulk commit for an index partition succeeds. + *

+ * The default implementation marks the index partition as available in the data table config. + * Subclasses can override this to perform index-specific follow-up work (for example, index-definition + * registration or post-commit validation). + * + * @param metadataMetaClient metadata table meta client used during initialization + * @param records records committed during index partition initialization + * @param fileGroupCount number of file groups created for the index partition + * @param relativePartitionPath metadata table relative partition path being initialized + */ + @Override + public void postInitialization(HoodieTableMetaClient metadataMetaClient, HoodieData records, int fileGroupCount, String relativePartitionPath) { + dataTableMetaClient.getTableConfig().setMetadataPartitionState(dataTableMetaClient, relativePartitionPath, true); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/ExpressionIndexRecordGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/ExpressionIndexRecordGenerator.java new file mode 100644 index 0000000000000..1fbf3dfcc5175 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/ExpressionIndexRecordGenerator.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.metadata.model.FileToIndex; +import org.apache.hudi.storage.StorageConfiguration; + +import java.util.List; + +/** + * Engine-specific generator for expression index records used during metadata index bootstrap. + */ +public interface ExpressionIndexRecordGenerator { + EngineType getEngineType(); + + /** + * Generates expression index records. + * + * @param filesToIndex triplet of partition, (file path, file size) + * @param indexDefinition definition of the expression index + * @param metaClient {@link HoodieTableMetaClient} instance + * @param parallelism parallelism to use for engine operations + * @param tableSchema table schema + * @param readerSchema reader schema + * @param storageConf storage config + * @param instantTime instant time + * @return expression index records + */ + HoodieData generate( + List filesToIndex, + HoodieIndexDefinition indexDefinition, + HoodieTableMetaClient metaClient, + int parallelism, + HoodieSchema tableSchema, + HoodieSchema readerSchema, + StorageConfiguration storageConf, + String instantTime); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/Indexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/Indexer.java new file mode 100644 index 0000000000000..94a450fc3e788 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/Indexer.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.util.Lazy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * Interface for initializing and updating a type of metadata or index + * in the metadata table. + *

+ * When a new type of index is added to MetadataPartitionType, an + * implementation of the {@link Indexer} interface is required, and it + * must be added to {@link IndexerFactory}. + */ +public interface Indexer { + /** + * Generates records for initializing the index. + * + * @param dataTableInstantTime instant time of the data table that the metadata table is initialized on + * @param instantTimeForPartition instant time used for initializing a specific metadata partition + * @param partitionIdToAllFilesMap map of partition to files + * @param lazyLatestMergedPartitionFileSliceList lazily-evaluated list of file slices for the indexer that needs it + * @return zero or more {@link IndexPartitionInitialization} entries to be initialized. + * Returning an empty list means no metadata partition needs initialization in this invocation. + * @throws IOException upon IO error + */ + List buildInitializationData( + String dataTableInstantTime, + String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException; + + /** + * Hook invoked after the bootstrap bulk commit for an index partition succeeds. + * Implementations can use this to perform index-specific follow-up work. + * + * @param metadataMetaClient metadata table meta client used during initialization + * @param records records committed during index partition initialization + * @param fileGroupCount number of file groups created for the index partition + * @param relativePartitionPath metadata table relative partition path being initialized + */ + void postInitialization( + HoodieTableMetaClient metadataMetaClient, + HoodieData records, + int fileGroupCount, + String relativePartitionPath); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/IndexerFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/IndexerFactory.java new file mode 100644 index 0000000000000..49dc979bc11ef --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/IndexerFactory.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata.index; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.bloomfilters.BloomFiltersIndexer; +import org.apache.hudi.metadata.index.columnstats.ColumnStatsIndexer; +import org.apache.hudi.metadata.index.expression.ExpressionIndexer; +import org.apache.hudi.metadata.index.files.FilesIndexer; +import org.apache.hudi.metadata.index.partitionstats.PartitionStatsIndexer; +import org.apache.hudi.metadata.index.record.PartitionedRecordIndexer; +import org.apache.hudi.metadata.index.record.RecordIndexer; +import org.apache.hudi.metadata.index.secondary.SecondaryIndexer; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Factory for creating {@link Indexer} implementations and resolving enabled indexers + * based on table and metadata configuration. + */ +public class IndexerFactory { + private static Indexer getIndexer(MetadataPartitionType partitionType, + HoodieEngineContext engineContext, + HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient, + ExpressionIndexRecordGenerator expressionIndexRecordGenerator) { + switch (partitionType) { + case FILES: + return new FilesIndexer(engineContext, dataTableWriteConfig, dataTableMetaClient); + case BLOOM_FILTERS: + return new BloomFiltersIndexer(engineContext, dataTableWriteConfig, dataTableMetaClient); + case COLUMN_STATS: + return new ColumnStatsIndexer(engineContext, dataTableWriteConfig, dataTableMetaClient); + case RECORD_INDEX: + return dataTableWriteConfig.isRecordLevelIndexEnabled() + ? new PartitionedRecordIndexer(engineContext, dataTableWriteConfig, dataTableMetaClient) + : new RecordIndexer(engineContext, dataTableWriteConfig, dataTableMetaClient); + case EXPRESSION_INDEX: + return new ExpressionIndexer(engineContext, dataTableWriteConfig, dataTableMetaClient, expressionIndexRecordGenerator); + case PARTITION_STATS: + return new PartitionStatsIndexer(engineContext, dataTableWriteConfig, dataTableMetaClient); + case SECONDARY_INDEX: + return new SecondaryIndexer(engineContext, dataTableWriteConfig, dataTableMetaClient); + default: + throw new HoodieNotSupportedException("Unsupported metadata partition type for indexing: " + partitionType); + } + } + + /** + * Returns the map of metadata partition type to the indexer for the enabled metadata + * partition types based on the metadata config and table config. + */ + public static Map getEnabledIndexerMap( + HoodieEngineContext engineContext, + HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient, + ExpressionIndexRecordGenerator expressionIndexRecordGenerator) { + if (!dataTableWriteConfig.getMetadataConfig().isEnabled()) { + return Collections.emptyMap(); + } + return Collections.unmodifiableMap(Arrays.stream(MetadataPartitionType.getValidValues(dataTableMetaClient.getTableConfig().getTableVersion())) + .filter(partitionType -> partitionType.isMetadataPartitionSupported(dataTableMetaClient) + && (partitionType.isMetadataPartitionEnabled(dataTableWriteConfig.getMetadataConfig(), dataTableMetaClient.getTableConfig()) + || partitionType.isMetadataPartitionAvailable(dataTableMetaClient))) + .collect(Collectors.toMap( + Function.identity(), + type -> IndexerFactory.getIndexer(type, engineContext, dataTableWriteConfig, dataTableMetaClient, expressionIndexRecordGenerator)))); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/UnsupportedExpressionIndexRecordGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/UnsupportedExpressionIndexRecordGenerator.java new file mode 100644 index 0000000000000..55fd2928b17ad --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/UnsupportedExpressionIndexRecordGenerator.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.metadata.model.FileToIndex; +import org.apache.hudi.storage.StorageConfiguration; + +import java.util.List; + +/** + * Fallback {@link ExpressionIndexRecordGenerator} that throws a not-supported exception + * when expression index bootstrap is requested for unsupported engines. + */ +public class UnsupportedExpressionIndexRecordGenerator implements ExpressionIndexRecordGenerator { + + private final EngineType engineType; + + public UnsupportedExpressionIndexRecordGenerator(EngineType engineType) { + this.engineType = engineType; + } + + @Override + public EngineType getEngineType() { + return engineType; + } + + @Override + public HoodieData generate( + List filesToIndex, + HoodieIndexDefinition indexDefinition, + HoodieTableMetaClient metaClient, + int parallelism, + HoodieSchema tableSchema, + HoodieSchema readerSchema, + StorageConfiguration storageConf, + String instantTime) { + if (metaClient.getTableConfig().getTableVersion().lesserThan(HoodieTableVersion.EIGHT)) { + throw new HoodieNotSupportedException("Table version 6 and below does not support expression index"); + } + throw new HoodieNotSupportedException(engineType + " engine does not support building expression index yet"); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/bloomfilters/BloomFiltersIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/bloomfilters/BloomFiltersIndexer.java new file mode 100644 index 0000000000000..0e644212a7090 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/bloomfilters/BloomFiltersIndexer.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.bloomfilters; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.BaseIndexer; +import org.apache.hudi.util.Lazy; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link MetadataPartitionType#BLOOM_FILTERS} index + */ +@Slf4j +public class BloomFiltersIndexer extends BaseIndexer { + + public BloomFiltersIndexer(HoodieEngineContext engineContext, + HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + @Override + public List buildInitializationData( + String dataTableInstantTime, + String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + HoodieData records = HoodieTableMetadataUtil.convertFilesToBloomFilterRecords( + engineContext, Collections.emptyMap(), partitionIdToAllFilesMap, dataTableInstantTime, dataTableMetaClient, + dataTableWriteConfig.getBloomIndexParallelism(), dataTableWriteConfig.getBloomFilterType()); + final int fileGroupCount = dataTableWriteConfig.getMetadataConfig().getBloomFilterIndexFileGroupCount(); + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath(), records)); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/columnstats/ColumnStatsIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/columnstats/ColumnStatsIndexer.java new file mode 100644 index 0000000000000..3a25023932d91 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/columnstats/ColumnStatsIndexer.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.columnstats; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.BaseIndexer; +import org.apache.hudi.util.Lazy; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.index.HoodieIndexUtils.register; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.existingIndexVersionOrDefault; +import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS; + +/** + * Implementation of {@link MetadataPartitionType#COLUMN_STATS} metadata + */ +@Slf4j +public class ColumnStatsIndexer extends BaseIndexer { + private Lazy> columnsToIndex; + + public ColumnStatsIndexer(HoodieEngineContext engineContext, + HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + + this.columnsToIndex = Lazy.lazily(() -> + new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex( + dataTableMetaClient.getTableConfig(), + dataTableWriteConfig.getMetadataConfig(), + Lazy.lazily(() -> HoodieTableMetadataUtil.tryResolveSchemaForTable(dataTableMetaClient)), + true, + Option.of(dataTableWriteConfig.getRecordMerger().getRecordType()), + existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, dataTableMetaClient)).keySet())); + } + + @Override + public List buildInitializationData( + String dataTableInstantTime, + String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + final int fileGroupCount = dataTableWriteConfig.getMetadataConfig().getColumnStatsIndexFileGroupCount(); + if (partitionIdToAllFilesMap.isEmpty()) { + this.columnsToIndex = Lazy.lazily(Collections::emptyList); + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, COLUMN_STATS.getPartitionPath(), engineContext.emptyHoodieData())); + } + + if (columnsToIndex.get().isEmpty()) { + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, COLUMN_STATS.getPartitionPath(), engineContext.emptyHoodieData())); + } + + log.info("Indexing {} columns for column stats index", columnsToIndex.get().size()); + // during initialization, we need stats for base and log files. + HoodieData records = HoodieTableMetadataUtil.convertFilesToColumnStatsRecords( + engineContext, Collections.emptyMap(), partitionIdToAllFilesMap, + dataTableMetaClient, dataTableWriteConfig.getColumnStatsIndexParallelism(), + dataTableWriteConfig.getMetadataConfig().getMaxReaderBufferSize(), + columnsToIndex.get()); + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, COLUMN_STATS.getPartitionPath(), records)); + } + + @Override + public void postInitialization(HoodieTableMetaClient metadataMetaClient, HoodieData records, int fileGroupCount, String relativePartitionPath) { + HoodieIndexDefinition indexDefinition = HoodieIndexDefinition.newBuilder() + .withIndexName(PARTITION_NAME_COLUMN_STATS) + .withIndexType(PARTITION_NAME_COLUMN_STATS) + .withIndexFunction(PARTITION_NAME_COLUMN_STATS) + .withSourceFields(columnsToIndex.get()) + // Use the existing version if exists, otherwise fall back to the default version. + .withVersion(existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, dataTableMetaClient)) + .withIndexOptions(Collections.EMPTY_MAP) + .build(); + log.info("Registering or updating index: {} of type: {}", indexDefinition.getIndexName(), indexDefinition.getIndexType()); + register(dataTableMetaClient, indexDefinition); + + super.postInitialization(metadataMetaClient, records, fileGroupCount, relativePartitionPath); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/expression/ExpressionIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/expression/ExpressionIndexer.java new file mode 100644 index 0000000000000..34707dffa99b0 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/expression/ExpressionIndexer.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.expression; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.metadata.index.ExpressionIndexRecordGenerator; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.metadata.model.FileToIndex; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.BaseIndexer; +import org.apache.hudi.util.Lazy; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getExpressionIndexPartitionsToInit; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getProjectedSchemaForExpressionIndex; +import static org.apache.hudi.metadata.MetadataPartitionType.EXPRESSION_INDEX; + +/** + * Implementation of {@link MetadataPartitionType#EXPRESSION_INDEX} index + */ +@Slf4j +public class ExpressionIndexer extends BaseIndexer { + + private final ExpressionIndexRecordGenerator expressionIndexRecordGenerator; + + public ExpressionIndexer( + HoodieEngineContext engineContext, + HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient, + ExpressionIndexRecordGenerator expressionIndexRecordGenerator) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + + this.expressionIndexRecordGenerator = expressionIndexRecordGenerator; + } + + @Override + public List buildInitializationData( + String dataTableInstantTime, + String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + Set expressionIndexPartitionsToInit = getExpressionIndexPartitionsToInit( + EXPRESSION_INDEX, dataTableWriteConfig.getMetadataConfig(), dataTableMetaClient); + if (expressionIndexPartitionsToInit.size() != 1) { + if (expressionIndexPartitionsToInit.size() > 1) { + log.warn("Skipping expression index initialization as only one expression index " + + "bootstrap at a time is supported for now. Provided: {}", expressionIndexPartitionsToInit); + } + return Collections.emptyList(); + } + + String indexName = expressionIndexPartitionsToInit.iterator().next(); + HoodieIndexDefinition indexDefinition = HoodieTableMetadataUtil.getHoodieIndexDefinition(indexName, dataTableMetaClient); + ValidationUtils.checkState(indexDefinition != null, "Expression Index definition is not present for index " + indexName); + + List partitionFileSlicePairs = lazyLatestMergedPartitionFileSliceList.get(); + List filesToIndex = new ArrayList<>(); + partitionFileSlicePairs.forEach(fsp -> { + if (fsp.fileSlice().getBaseFile().isPresent()) { + filesToIndex.add(FileToIndex.of(fsp.partitionPath(), fsp.fileSlice().getBaseFile().get().getPath(), fsp.fileSlice().getBaseFile().get().getFileSize())); + } + fsp.fileSlice().getLogFiles() + .forEach(hoodieLogFile + -> filesToIndex.add(FileToIndex.of(fsp.partitionPath(), hoodieLogFile.getPath().toString(), hoodieLogFile.getFileSize()))); + }); + + int fileGroupCount = dataTableWriteConfig.getMetadataConfig().getExpressionIndexFileGroupCount(); + if (partitionFileSlicePairs.isEmpty()) { + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, indexName, engineContext.emptyHoodieData())); + } + + int parallelism = Math.min(filesToIndex.size(), dataTableWriteConfig.getMetadataConfig().getExpressionIndexParallelism()); + Lazy tableSchemaOpt = Lazy.lazily(() -> + HoodieTableMetadataUtil.tryResolveSchemaForTable(dataTableMetaClient) + .orElseThrow(() -> new HoodieMetadataException("Table schema is not available for expression index initialization"))); + HoodieSchema tableSchema = tableSchemaOpt.get(); + HoodieSchema readerSchema = getProjectedSchemaForExpressionIndex(indexDefinition, dataTableMetaClient, tableSchema); + + HoodieData records = expressionIndexRecordGenerator.generate( + filesToIndex, indexDefinition, dataTableMetaClient, parallelism, + tableSchema, readerSchema, engineContext.getStorageConf(), dataTableInstantTime); + + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, indexName, records)); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/files/FilesIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/files/FilesIndexer.java new file mode 100644 index 0000000000000..b4234423f3f56 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/files/FilesIndexer.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata.index.files; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.BaseIndexer; +import org.apache.hudi.util.Lazy; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.hudi.metadata.MetadataPartitionType.FILES; + +/** + * Implementation of {@link MetadataPartitionType#FILES} metadata + */ +@Slf4j +public class FilesIndexer extends BaseIndexer { + public FilesIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + @Override + public List buildInitializationData(String dataTableInstantTime, String instantTimeForPartition, Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + // FILES partition uses a single file group + final int fileGroupCount = 1; + + Set partitions = partitionIdToAllFilesMap.keySet(); + final int totalDataFilesCount = partitionIdToAllFilesMap.values().stream().mapToInt(Map::size).sum(); + log.info("Committing total {} partitions and {} files to metadata", partitions.size(), totalDataFilesCount); + + // Record which saves the list of all partitions + HoodieRecord record = HoodieMetadataPayload.createPartitionListRecord(partitions); + HoodieData allPartitionsRecord = engineContext.parallelize(Collections.singletonList(record), 1); + if (partitionIdToAllFilesMap.isEmpty()) { + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, FILES.getPartitionPath(), allPartitionsRecord)); + } + + // Records which save the file listing of each partition + engineContext.setJobStatus(this.getClass().getSimpleName(), "Creating records for metadata FILES partition"); + HoodieData fileListRecords = engineContext.parallelize( + new ArrayList<>(partitionIdToAllFilesMap.entrySet()), partitionIdToAllFilesMap.size()) + .map(partitionInfo -> { + Map fileNameToSizeMap = partitionInfo.getValue(); + return HoodieMetadataPayload.createPartitionFilesRecord( + partitionInfo.getKey(), fileNameToSizeMap, Collections.emptyList()); + }); + ValidationUtils.checkState(fileListRecords.count() == partitions.size()); + + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, FILES.getPartitionPath(), allPartitionsRecord.union(fileListRecords))); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/model/DataPartitionAndRecords.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/model/DataPartitionAndRecords.java new file mode 100644 index 0000000000000..4ac2ac71ab661 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/model/DataPartitionAndRecords.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. + */ + +package org.apache.hudi.metadata.index.model; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.experimental.Accessors; + +/** + * Holds the initialization records and data partition if the index is a partitioned index. + *

+ * For non-partitioned indexes, {@code dataPartition} is empty and the records represent the entire index records. + */ +@AllArgsConstructor +@Getter +@Accessors(fluent = true) +public class DataPartitionAndRecords { + private final int numFileGroups; + private final Option dataPartition; + private final HoodieData indexRecords; +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/model/IndexPartitionInitialization.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/model/IndexPartitionInitialization.java new file mode 100644 index 0000000000000..b6505513d3d7b --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/model/IndexPartitionInitialization.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.model; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.metadata.DefaultMetadataTableFileGroupIndexParser; +import org.apache.hudi.metadata.MetadataTableFileGroupIndexParser; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.experimental.Accessors; + +import java.util.Collections; +import java.util.List; + +/** + * Represents the initialization data for a single metadata index partition, + * including file-group count, optional source data partition, destination metadata + * partition path, and records to commit. + */ +@AllArgsConstructor +@Getter +@Accessors(fluent = true) +public class IndexPartitionInitialization { + private final int totalFileGroups; + private final String indexPartitionName; + private final MetadataTableFileGroupIndexParser indexParser; + private final List dataPartitionAndRecords; + + /** + * Creates initialization data for a non-partitioned index from a single {@link DataPartitionAndRecords} + * payload and a default file-group parser. + * + * @param indexPartitionName metadata index partition path to initialize + * @param dataPartitionAndRecords records and metadata for one data partition (or non-partitioned payload) + * @return an {@link IndexPartitionInitialization} instance configured with file-group count inferred + * from {@code dataPartitionAndRecords} + */ + public static IndexPartitionInitialization of(String indexPartitionName, DataPartitionAndRecords dataPartitionAndRecords) { + return of(dataPartitionAndRecords.numFileGroups(), indexPartitionName, dataPartitionAndRecords.indexRecords()); + } + + /** + * Creates initialization data for a non-partitioned index using a single records payload and the default + * {@link MetadataTableFileGroupIndexParser} implementation. + * + * @param totalFileGroups total number of file groups to pre-create for this index partition + * @param indexPartitionName metadata index partition path/name to initialize + * @param indexRecords records to be written into the target index partition + * @return an {@link IndexPartitionInitialization} containing one {@link DataPartitionAndRecords} + * entry without source data-partition binding + */ + public static IndexPartitionInitialization of(int totalFileGroups, String indexPartitionName, HoodieData indexRecords) { + return of(totalFileGroups, indexPartitionName, new DefaultMetadataTableFileGroupIndexParser(totalFileGroups), + Collections.singletonList(new DataPartitionAndRecords(totalFileGroups, Option.empty(), indexRecords))); + } + + /** + * Creates initialization data for a partitioned index with a provided file-group parser and + * one or more per-data-partition record payloads. + * + * @param totalFileGroups total number of file groups to pre-create for this index partition + * @param indexPartitionName metadata index partition path to initialize + * @param indexParser parser used to map records to metadata table file-group indexes + * @param dataPartitionAndRecords input record payloads grouped by source data partition + * @return an {@link IndexPartitionInitialization} with the provided parser and partitioned payloads + */ + public static IndexPartitionInitialization of(int totalFileGroups, String indexPartitionName, MetadataTableFileGroupIndexParser indexParser, List dataPartitionAndRecords) { + return new IndexPartitionInitialization(totalFileGroups, indexPartitionName, indexParser, dataPartitionAndRecords); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/partitionstats/PartitionStatsIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/partitionstats/PartitionStatsIndexer.java new file mode 100644 index 0000000000000..a9f1972195994 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/partitionstats/PartitionStatsIndexer.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.partitionstats; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.BaseIndexer; +import org.apache.hudi.util.Lazy; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.metadata.MetadataPartitionType.PARTITION_STATS; + +/** + * Implementation of {@link MetadataPartitionType#PARTITION_STATS} metadata + */ +@Slf4j +public class PartitionStatsIndexer extends BaseIndexer { + public PartitionStatsIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + @Override + public List buildInitializationData(String dataTableInstantTime, String instantTimeForPartition, Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + // Partition stats index cannot be enabled for a non-partitioned table + if (!dataTableMetaClient.getTableConfig().isTablePartitioned()) { + return Collections.emptyList(); + } + + // For PARTITION_STATS, COLUMN_STATS should also be enabled + if (!dataTableWriteConfig.isMetadataColumnStatsIndexEnabled()) { + log.debug("Skipping partition stats initialization as column stats index is not enabled. Please enable {}", + HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key()); + return Collections.emptyList(); + } + + Lazy> tableSchemaOpt = Lazy.lazily(() -> HoodieTableMetadataUtil.tryResolveSchemaForTable(dataTableMetaClient)); + HoodieData records = HoodieTableMetadataUtil.convertFilesToPartitionStatsRecords( + engineContext, lazyLatestMergedPartitionFileSliceList.get(), dataTableWriteConfig.getMetadataConfig(), + dataTableMetaClient, tableSchemaOpt, Option.of(dataTableWriteConfig.getRecordMerger().getRecordType())); + final int fileGroupCount = dataTableWriteConfig.getMetadataConfig().getPartitionStatsIndexFileGroupCount(); + + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, PARTITION_STATS.getPartitionPath(), records)); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/BaseRecordIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/BaseRecordIndexer.java new file mode 100644 index 0000000000000..d7d3f9d878d71 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/BaseRecordIndexer.java @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.record; + +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.engine.ReaderContextFactory; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.HoodieAvroFileReader; +import org.apache.hudi.io.storage.HoodieIOFactory; +import org.apache.hudi.metadata.index.model.DataPartitionAndRecords; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCache; +import org.apache.hudi.common.schema.HoodieSchemaUtils; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.read.HoodieFileGroupReader; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.CloseableMappingIterator; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.utils.SerDeHelper; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.BaseIndexer; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.HoodieStorageUtils; +import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.storage.StoragePath; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.schema.HoodieSchemaUtils.getRecordKeySchema; + +/** + * Base implementation for record-index. + */ +@Slf4j +public abstract class BaseRecordIndexer extends BaseIndexer { + + private static final int RECORD_INDEX_AVERAGE_RECORD_SIZE = 48; + + protected BaseRecordIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + protected DataPartitionAndRecords initializeRecordIndexPartition( + List latestMergedPartitionFileSliceList, + int recordIndexMaxParallelism) { + return initializeRecordIndexPartition(null, latestMergedPartitionFileSliceList, recordIndexMaxParallelism); + } + + protected DataPartitionAndRecords initializeRecordIndexPartition( + String dataPartition, + List latestMergedPartitionFileSliceList, + int recordIndexMaxParallelism) { + log.info("Initializing record index from {} file slices", latestMergedPartitionFileSliceList.size()); + HoodieData records = readRecordKeysFromFileSliceSnapshot( + engineContext, + latestMergedPartitionFileSliceList, + recordIndexMaxParallelism, + this.getClass().getSimpleName(), + dataTableMetaClient, + dataTableWriteConfig); + + // Initialize the file groups + final int fileGroupCount = estimateFileGroupCount(records); + log.info("Initializing record index with {} file groups.", fileGroupCount); + return new DataPartitionAndRecords(fileGroupCount, Option.ofNullable(dataPartition), records); + } + + @Override + public void postInitialization(HoodieTableMetaClient metadataMetaClient, HoodieData records, int fileGroupCount, String relativePartitionPath) { + super.postInitialization(metadataMetaClient, records, fileGroupCount, relativePartitionPath); + // Validate record index after commit if validation is enabled + if (dataTableWriteConfig.getMetadataConfig().isRecordIndexInitializationValidationEnabled()) { + validateRecordIndex(records, fileGroupCount, metadataMetaClient); + } + records.unpersist(); + } + + /** + * Validates the record index after bootstrap by comparing the expected record count with the actual + * record count stored in the metadata table. The validation is performed in a distributed manner + * using the engine context to count records from HFiles in parallel. + * + * @param recordIndexRecords the HoodieData containing the expected records + * @param fileGroupCount the expected number of file groups + * @param metadataMetaClient meta client for the metadata table + */ + protected void validateRecordIndex(HoodieData recordIndexRecords, int fileGroupCount, HoodieTableMetaClient metadataMetaClient) { + String partitionName = MetadataPartitionType.RECORD_INDEX.getPartitionPath(); + HoodieTableFileSystemView fsView = HoodieTableMetadataUtil.getFileSystemViewForMetadataTable(metadataMetaClient); + try { + // Use merged file slices to handle cases with pending compactions + List fileSlices = HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, fsView, partitionName); + + // Filter to only file slices with base files and extract their storage paths + List baseFilePaths = fileSlices.stream() + .filter(fs -> fs.getBaseFile().isPresent()) + .map(fs -> fs.getBaseFile().get().getStoragePath()) + .collect(Collectors.toList()); + + // Count records in a distributed manner using the engine context + long totalRecords = countRecordsInHFiles(baseFilePaths, metadataMetaClient); + long expectedRecordCount = recordIndexRecords.count(); + + ValidationUtils.checkArgument(totalRecords == expectedRecordCount, "Record Count Validation failed with " + + totalRecords + " present in record index vs the expected " + expectedRecordCount); + log.info(String.format("Record index initialized on %d shards (expected = %d) with %d records (expected = %d)", + fileSlices.size(), fileGroupCount, totalRecords, expectedRecordCount)); + } finally { + fsView.close(); + } + } + + /** + * Counts the total number of records in HFiles in a distributed manner. + * + * @param baseFilePaths list of storage paths to HFiles + * @param metadataMetaClient meta client for the metadata table + * @return total number of records across all HFiles + */ + private long countRecordsInHFiles(List baseFilePaths, HoodieTableMetaClient metadataMetaClient) { + if (baseFilePaths.isEmpty()) { + return 0L; + } + + int parallelism = Math.min(baseFilePaths.size(), dataTableWriteConfig.getMetadataConfig().getRecordIndexMaxParallelism()); + StorageConfiguration storageConfBroadcast = metadataMetaClient.getStorageConf(); + HoodieFileFormat baseFileFormat = metadataMetaClient.getTableConfig().getBaseFileFormat(); + + return engineContext.parallelize(baseFilePaths, parallelism) + .mapPartitions(pathIterator -> { + long count = 0L; + while (pathIterator.hasNext()) { + StoragePath path = pathIterator.next(); + try { + HoodieStorage storage = HoodieStorageUtils.getStorage(path, storageConfBroadcast); + HoodieConfig readerConfig = new HoodieConfig(); + HoodieAvroFileReader reader = (HoodieAvroFileReader) HoodieIOFactory.getIOFactory(storage) + .getReaderFactory(HoodieRecord.HoodieRecordType.AVRO) + .getFileReader(readerConfig, path, baseFileFormat, Option.empty()); + try { + count += reader.getTotalRecords(); + } finally { + reader.close(); + } + } catch (IOException e) { + throw new HoodieIOException("Error reading total records from file " + path, e); + } + } + return Collections.singletonList(count).iterator(); + }, true) + .collectAsList() + .stream() + .mapToLong(Long::longValue) + .sum(); + } + + /** + * Fetch record locations from FileSlice snapshot. + * + * @param engineContext context ot use. + * @param partitionFileSlicePairs list of pairs of partition and file slice. + * @param recordIndexMaxParallelism parallelism to use. + * @param activeModule active module of interest. + * @param metaClient metaclient instance to use. + * @param dataWriteConfig write config to use. + * @return metadata records for initializing record index entries. + */ + protected HoodieData readRecordKeysFromFileSliceSnapshot( + HoodieEngineContext engineContext, + List partitionFileSlicePairs, + int recordIndexMaxParallelism, + String activeModule, + HoodieTableMetaClient metaClient, + HoodieWriteConfig dataWriteConfig) { + if (partitionFileSlicePairs.isEmpty()) { + return engineContext.emptyHoodieData(); + } + + Option instantTime = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants() + .lastInstant() + .map(HoodieInstant::requestedTime); + if (!instantTime.isPresent()) { + return engineContext.emptyHoodieData(); + } + + engineContext.setJobStatus(activeModule, "Record Index: reading record keys from " + partitionFileSlicePairs.size() + " file slices"); + final int parallelism = Math.min(partitionFileSlicePairs.size(), recordIndexMaxParallelism); + ReaderContextFactory readerContextFactory = engineContext.getReaderContextFactory(metaClient); + return engineContext.parallelize(partitionFileSlicePairs, parallelism).flatMap(partitionAndFileSlice -> { + final String partition = partitionAndFileSlice.partitionPath(); + final FileSlice fileSlice = partitionAndFileSlice.fileSlice(); + final String fileId = fileSlice.getFileId(); + HoodieReaderContext readerContext = readerContextFactory.getContext(); + HoodieSchema dataSchema = HoodieSchemaCache.intern(HoodieSchemaUtils.addMetadataFields(HoodieSchema.parse(dataWriteConfig.getWriteSchema()), dataWriteConfig.allowOperationMetadataField())); + HoodieSchema requestedSchema = metaClient.getTableConfig().populateMetaFields() ? getRecordKeySchema() + : HoodieSchemaUtils.projectSchema(dataSchema, Arrays.asList(metaClient.getTableConfig().getRecordKeyFields().orElse(new String[0]))); + Option internalSchemaOption = SerDeHelper.fromJson(dataWriteConfig.getInternalSchema()); + HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() + .withReaderContext(readerContext) + .withHoodieTableMetaClient(metaClient) + .withFileSlice(fileSlice) + .withLatestCommitTime(instantTime.get()) + .withDataSchema(dataSchema) + .withRequestedSchema(requestedSchema) + .withInternalSchema(internalSchemaOption) + .withShouldUseRecordPosition(false) + .withProps(metaClient.getTableConfig().getProps()) + .build(); + String baseFileInstantTime = fileSlice.getBaseInstantTime(); + return new CloseableMappingIterator<>(fileGroupReader.getClosableIterator(), record -> { + String recordKey = readerContext.getRecordContext().getRecordKey(record, requestedSchema); + return HoodieMetadataPayload.createRecordIndexUpdate(recordKey, partition, fileId, + baseFileInstantTime, 0); + }); + }); + } + + protected int estimateFileGroupCount(HoodieData records) { + int minFileGroupCount; + int maxFileGroupCount; + if (dataTableWriteConfig.isRecordLevelIndexEnabled()) { + minFileGroupCount = dataTableWriteConfig.getRecordLevelIndexMinFileGroupCount(); + maxFileGroupCount = dataTableWriteConfig.getRecordLevelIndexMaxFileGroupCount(); + } else { + minFileGroupCount = dataTableWriteConfig.getGlobalRecordLevelIndexMinFileGroupCount(); + maxFileGroupCount = dataTableWriteConfig.getGlobalRecordLevelIndexMaxFileGroupCount(); + } + Supplier recordCountSupplier = () -> { + records.persist("MEMORY_AND_DISK_SER"); + long count = records.count(); + log.info("Initializing record index with {} mappings", count); + return count; + }; + return HoodieTableMetadataUtil.estimateFileGroupCount( + MetadataPartitionType.RECORD_INDEX, + recordCountSupplier, + RECORD_INDEX_AVERAGE_RECORD_SIZE, + minFileGroupCount, + maxFileGroupCount, + dataTableWriteConfig.getRecordIndexGrowthFactor(), + dataTableWriteConfig.getRecordIndexMaxFileGroupSizeBytes() + ); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/PartitionedRecordIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/PartitionedRecordIndexer.java new file mode 100644 index 0000000000000..149e3419fb563 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/PartitionedRecordIndexer.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.record; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.model.DataPartitionAndRecords; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.record.HoodieRecordIndex; +import org.apache.hudi.metadata.BucketizedMetadataTableFileGroupIndexParser; +import org.apache.hudi.util.Lazy; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.createRecordIndexDefinition; +import static org.apache.hudi.metadata.MetadataPartitionType.RECORD_INDEX; + +/** + * Implementation of the global {@link MetadataPartitionType#RECORD_INDEX} index + */ +@Slf4j +public class PartitionedRecordIndexer extends BaseRecordIndexer { + public PartitionedRecordIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + @Override + public List buildInitializationData(String dataTableInstantTime, String instantTimeForPartition, Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + createRecordIndexDefinition(dataTableMetaClient, Collections.singletonMap(HoodieRecordIndex.IS_PARTITIONED_OPTION, "true")); + Map> partitionFileSlicePairsMap = lazyLatestMergedPartitionFileSliceList.get().stream() + .collect(Collectors.groupingBy(FileSliceAndPartition::partitionPath)); + Map fileGroupCountAndRecordsPairMap = new HashMap<>(partitionFileSlicePairsMap.size()); + int maxParallelismPerHudiPartition = partitionFileSlicePairsMap.isEmpty() + ? 1 : Math.max(1, dataTableWriteConfig.getMetadataConfig().getRecordIndexMaxParallelism() / partitionFileSlicePairsMap.size()); + int totalFileGroupCount = 0; + for (String partition : partitionFileSlicePairsMap.keySet()) { + log.info("Initializing partitioned record index from data partition {}", partition); + DataPartitionAndRecords dataPartitionAndRecords = initializeRecordIndexPartition(partition, partitionFileSlicePairsMap.get(partition), maxParallelismPerHudiPartition); + fileGroupCountAndRecordsPairMap.put(partition, dataPartitionAndRecords); + totalFileGroupCount += dataPartitionAndRecords.numFileGroups(); + } + log.info("Initializing partitioned record index with {} mappings", totalFileGroupCount); + + List initializationList = new ArrayList<>(); + // Generate the file groups + TreeMap partitionSizes = new TreeMap<>(); + for (Map.Entry entry: fileGroupCountAndRecordsPairMap.entrySet()) { + String dataPartition = entry.getKey(); + DataPartitionAndRecords dataPartitionAndRecords = entry.getValue(); + ValidationUtils.checkArgument(dataPartitionAndRecords.numFileGroups() > 0, "FileGroup count for partitioned RLI data partition " + dataPartition + " should be > 0"); + partitionSizes.put(dataPartition, dataPartitionAndRecords.numFileGroups()); + initializationList.add(dataPartitionAndRecords); + } + + return Collections.singletonList(IndexPartitionInitialization.of(totalFileGroupCount, + RECORD_INDEX.getPartitionPath(), new BucketizedMetadataTableFileGroupIndexParser(partitionSizes), initializationList)); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/RecordIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/RecordIndexer.java new file mode 100644 index 0000000000000..553c6d7d16e4a --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/record/RecordIndexer.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.record; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.model.DataPartitionAndRecords; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.record.HoodieRecordIndex; +import org.apache.hudi.util.Lazy; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.createRecordIndexDefinition; +import static org.apache.hudi.metadata.MetadataPartitionType.RECORD_INDEX; + +/** + * Implementation of the global {@link MetadataPartitionType#RECORD_INDEX} index + */ +@Slf4j +public class RecordIndexer extends BaseRecordIndexer { + + public RecordIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + @Override + public List buildInitializationData(String dataTableInstantTime, String instantTimeForPartition, Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + createRecordIndexDefinition(dataTableMetaClient, Collections.singletonMap(HoodieRecordIndex.IS_PARTITIONED_OPTION, "false")); + DataPartitionAndRecords dataPartitionAndRecords = initializeRecordIndexPartition( + lazyLatestMergedPartitionFileSliceList.get(), dataTableWriteConfig.getMetadataConfig().getRecordIndexMaxParallelism()); + return Collections.singletonList(IndexPartitionInitialization.of(RECORD_INDEX.getPartitionPath(), dataPartitionAndRecords)); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/secondary/SecondaryIndexer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/secondary/SecondaryIndexer.java new file mode 100644 index 0000000000000..6c7a57762429a --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/index/secondary/SecondaryIndexer.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index.secondary; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.BaseIndexer; +import org.apache.hudi.util.Lazy; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getSecondaryIndexPartitionsToInit; +import static org.apache.hudi.metadata.MetadataPartitionType.RECORD_INDEX; +import static org.apache.hudi.metadata.MetadataPartitionType.SECONDARY_INDEX; +import static org.apache.hudi.metadata.SecondaryIndexRecordGenerationUtils.readSecondaryKeysFromFileSlices; + +/** + * Implementation of {@link MetadataPartitionType#SECONDARY_INDEX} index + */ +@Slf4j +public class SecondaryIndexer extends BaseIndexer { + + private static final int RECORD_INDEX_AVERAGE_RECORD_SIZE = 48; + + public SecondaryIndexer( + HoodieEngineContext engineContext, + HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + @Override + public List buildInitializationData(String dataTableInstantTime, String instantTimeForPartition, Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + Set secondaryIndexPartitionsToInit = getSecondaryIndexPartitionsToInit(SECONDARY_INDEX, dataTableWriteConfig.getMetadataConfig(), dataTableMetaClient); + if (secondaryIndexPartitionsToInit.size() != 1) { + if (secondaryIndexPartitionsToInit.size() > 1) { + log.warn("Skipping secondary index initialization as only one secondary index bootstrap at a time is supported for now. Provided: {}", secondaryIndexPartitionsToInit); + } + return Collections.emptyList(); + } + + String indexName = secondaryIndexPartitionsToInit.iterator().next(); + HoodieIndexDefinition indexDefinition = HoodieTableMetadataUtil.getHoodieIndexDefinition(indexName, dataTableMetaClient); + ValidationUtils.checkState(indexDefinition != null, "Secondary Index definition is not present for index " + indexName); + + List partitionFileSlicePairs = lazyLatestMergedPartitionFileSliceList.get(); + + int parallelism = Math.min(partitionFileSlicePairs.size(), dataTableWriteConfig.getMetadataConfig().getSecondaryIndexParallelism()); + HoodieData records = readSecondaryKeysFromFileSlices( + engineContext, + partitionFileSlicePairs, + parallelism, + this.getClass().getSimpleName(), + dataTableMetaClient, + indexDefinition, + dataTableWriteConfig.getProps()); + + // Initialize the file groups - using the same estimation logic as that of record index + final int fileGroupCount = HoodieTableMetadataUtil.estimateFileGroupCount(RECORD_INDEX, records::count, + RECORD_INDEX_AVERAGE_RECORD_SIZE, dataTableWriteConfig.getGlobalRecordLevelIndexMinFileGroupCount(), + dataTableWriteConfig.getGlobalRecordLevelIndexMaxFileGroupCount(), dataTableWriteConfig.getRecordIndexGrowthFactor(), + dataTableWriteConfig.getRecordIndexMaxFileGroupSizeBytes()); + + return Collections.singletonList(IndexPartitionInitialization.of(fileGroupCount, indexName, records)); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/TestHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/TestHoodieBackedTableMetadataWriter.java index 5b69c1af65285..c1e17108a7dd2 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/TestHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/TestHoodieBackedTableMetadataWriter.java @@ -201,7 +201,7 @@ void testConvertToColumnStatsRecordWithValidColumns() { // Mock convertFilesToColumnStatsRecords to return empty HoodieData HoodieData mockHoodieData = mock(HoodieData.class); mockedUtil.when(() -> HoodieTableMetadataUtil.convertFilesToColumnStatsRecords( - any(), any(), any(), any(), any(), anyInt(), anyInt(), any())) + any(), any(), any(), any(), anyInt(), anyInt(), any())) .thenReturn(mockHoodieData); Map> partitionFilesToAdd = new HashMap<>(); @@ -223,7 +223,6 @@ void testConvertToColumnStatsRecordWithValidColumns() { eq(partitionFilesToDelete), eq(partitionFilesToAdd), eq(dataMetaClient), - eq(metadataConfig), eq(4), eq(1024), any() @@ -246,7 +245,7 @@ void testConvertToColumnStatsRecordWithMixedInputs() { // Mock convertFilesToColumnStatsRecords to return empty HoodieData HoodieData mockHoodieData = mock(HoodieData.class); mockedUtil.when(() -> HoodieTableMetadataUtil.convertFilesToColumnStatsRecords( - any(), any(), any(), any(), any(), anyInt(), anyInt(), any())) + any(), any(), any(), any(), anyInt(), anyInt(), any())) .thenReturn(mockHoodieData); Map> partitionFilesToAdd = new HashMap<>(); @@ -275,7 +274,6 @@ void testConvertToColumnStatsRecordWithMixedInputs() { eq(partitionFilesToDelete), eq(partitionFilesToAdd), eq(dataMetaClient), - eq(metadataConfig), eq(4), eq(1024), any() diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/TestIndexerFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/TestIndexerFactory.java new file mode 100644 index 0000000000000..f76e56cf92061 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/TestIndexerFactory.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. + */ + +package org.apache.hudi.metadata.index; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.record.PartitionedRecordIndexer; +import org.apache.hudi.metadata.index.record.RecordIndexer; + +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class TestIndexerFactory { + + @Test + void testMetadataDisabledReturnsEmptyMap() { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metadataConfig.isEnabled()).thenReturn(false); + + Map enabled = + IndexerFactory.getEnabledIndexerMap(engineContext, writeConfig, mock(HoodieTableMetaClient.class), mock(ExpressionIndexRecordGenerator.class)); + assertTrue(enabled.isEmpty()); + } + + @Test + void testRecordIndexerSelectionByConfig() { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metadataConfig.isEnabled()).thenReturn(true); + when(writeConfig.getRecordMerger()).thenReturn(mock(HoodieRecordMerger.class)); + + when(metaClient.getTableConfig()).thenReturn(tableConfig); + when(metaClient.getIndexMetadata()).thenReturn(Option.empty()); + when(tableConfig.getTableVersion()).thenReturn(HoodieTableVersion.EIGHT); + when(tableConfig.isTablePartitioned()).thenReturn(true); + when(tableConfig.isMetadataPartitionAvailable(any(MetadataPartitionType.class))).thenReturn(false); + when(tableConfig.isMetadataPartitionAvailable(MetadataPartitionType.RECORD_INDEX)).thenReturn(true); + + when(writeConfig.isRecordLevelIndexEnabled()).thenReturn(false); + Map globalIndexers = + IndexerFactory.getEnabledIndexerMap(engineContext, writeConfig, metaClient, mock(ExpressionIndexRecordGenerator.class)); + assertInstanceOf(RecordIndexer.class, globalIndexers.get(MetadataPartitionType.RECORD_INDEX)); + + when(writeConfig.isRecordLevelIndexEnabled()).thenReturn(true); + Map partitionedIndexers = + IndexerFactory.getEnabledIndexerMap(engineContext, writeConfig, metaClient, mock(ExpressionIndexRecordGenerator.class)); + assertInstanceOf(PartitionedRecordIndexer.class, partitionedIndexers.get(MetadataPartitionType.RECORD_INDEX)); + } + + @Test + void testTableVersionSixGatesExpressionAndSecondary() { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withPath("file:///tmp") + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .withExpressionIndexEnabled(true) + .withSecondaryIndexEnabled(true) + .withSecondaryIndexForColumn("col1") + .build()) + .build(); + + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); + when(metaClient.getTableConfig()).thenReturn(tableConfig); + when(metaClient.getIndexMetadata()).thenReturn(Option.empty()); + when(tableConfig.getTableVersion()).thenReturn(HoodieTableVersion.SIX); + when(tableConfig.isTablePartitioned()).thenReturn(true); + when(tableConfig.isMetadataPartitionAvailable(any(MetadataPartitionType.class))).thenReturn(false); + + Map enabled = + IndexerFactory.getEnabledIndexerMap(engineContext, writeConfig, metaClient, mock(ExpressionIndexRecordGenerator.class)); + + assertFalse(enabled.containsKey(MetadataPartitionType.EXPRESSION_INDEX)); + assertFalse(enabled.containsKey(MetadataPartitionType.SECONDARY_INDEX)); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/bloomfilters/TestBloomFiltersIndexer.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/bloomfilters/TestBloomFiltersIndexer.java new file mode 100644 index 0000000000000..c85a3bbdc7031 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/bloomfilters/TestBloomFiltersIndexer.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + */ + +package org.apache.hudi.metadata.index.bloomfilters; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.util.Lazy; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +class TestBloomFiltersIndexer { + + @SuppressWarnings("unchecked") + @Test + void testInitializeDataWithRealEngineContextAndIndexDataContent() throws IOException { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(getDefaultStorageConf()); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(writeConfig.getBloomIndexParallelism()).thenReturn(8); + when(writeConfig.getBloomFilterType()).thenReturn("DYNAMIC_V0"); + when(metadataConfig.getBloomFilterIndexFileGroupCount()).thenReturn(3); + + HoodieData records = (HoodieData) (HoodieData) engineContext.parallelize( + Collections.singletonList(HoodieMetadataPayload.createPartitionFilesRecord("p_bloom", + Collections.singletonMap("f1.parquet", 11L), Collections.emptyList())), + 1); + + try (MockedStatic mockedUtil = mockStatic(HoodieTableMetadataUtil.class)) { + mockedUtil.when(() -> HoodieTableMetadataUtil.convertFilesToBloomFilterRecords(any(), any(), any(), any(), any(), anyInt(), any())) + .thenReturn(records); + + ExposedBloomFiltersIndexer indexer = new ExposedBloomFiltersIndexer(engineContext, writeConfig, metaClient); + List initializationList = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertEquals(1, initializationList.size()); + + assertEquals(MetadataPartitionType.BLOOM_FILTERS.getPartitionPath(), initializationList.get(0).indexPartitionName()); + assertEquals(3, initializationList.get(0).totalFileGroups()); + List collected = initializationList.get(0).dataPartitionAndRecords().get(0).indexRecords().collectAsList(); + assertEquals(1, collected.size()); + assertEquals("p_bloom", collected.get(0).getRecordKey()); + } + } + + private static class ExposedBloomFiltersIndexer extends BloomFiltersIndexer { + ExposedBloomFiltersIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + List callGetData(String dataTableInstantTime, String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + return buildInitializationData(dataTableInstantTime, instantTimeForPartition, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/columnstats/TestColumnStatsIndexer.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/columnstats/TestColumnStatsIndexer.java new file mode 100644 index 0000000000000..739a0c9925e0f --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/columnstats/TestColumnStatsIndexer.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. + */ + +package org.apache.hudi.metadata.index.columnstats; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.util.Lazy; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +class TestColumnStatsIndexer { + + @Test + void testInitializeDataWithEmptyInputUsesEmptyHoodieData() throws IOException { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieData emptyData = mock(HoodieData.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metadataConfig.getColumnStatsIndexFileGroupCount()).thenReturn(2); + when(engineContext.emptyHoodieData()).thenReturn((HoodieData) emptyData); + + ExposedColumnStatsIndexer indexer = new ExposedColumnStatsIndexer(engineContext, writeConfig, metaClient); + List initializationList = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertEquals(1, initializationList.size()); + + assertEquals(MetadataPartitionType.COLUMN_STATS.getPartitionPath(), initializationList.get(0).indexPartitionName()); + assertSame(emptyData, initializationList.get(0).dataPartitionAndRecords().get(0).indexRecords()); + } + + @SuppressWarnings("unchecked") + @Test + void testInitializeDataWithRealEngineContextAndIndexDataContent() throws IOException { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(getDefaultStorageConf()); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); + HoodieRecordMerger recordMerger = mock(HoodieRecordMerger.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(writeConfig.getColumnStatsIndexParallelism()).thenReturn(4); + when(writeConfig.getRecordMerger()).thenReturn(recordMerger); + when(recordMerger.getRecordType()).thenReturn(HoodieRecord.HoodieRecordType.AVRO); + when(metadataConfig.getColumnStatsIndexFileGroupCount()).thenReturn(5); + when(metadataConfig.getMaxReaderBufferSize()).thenReturn(4096); + when(metaClient.getTableConfig()).thenReturn(tableConfig); + + Map> files = new HashMap<>(); + files.put("p1", Collections.singletonMap("f1.parquet", 1L)); + + HoodieData records = (HoodieData) (HoodieData) engineContext.parallelize( + Collections.singletonList(HoodieMetadataPayload.createPartitionFilesRecord("p_col", + Collections.singletonMap("f_col.parquet", 22L), Collections.emptyList())), + 1); + + try (MockedStatic mockedUtil = mockStatic(HoodieTableMetadataUtil.class)) { + Map columns = new HashMap<>(); + columns.put("c1", new Object()); + mockedUtil.when(() -> HoodieTableMetadataUtil.getColumnsToIndex(any(), any(), any(), eq(true), eq(Option.of(HoodieRecord.HoodieRecordType.AVRO)), any())) + .thenReturn(columns); + mockedUtil.when(() -> HoodieTableMetadataUtil.convertFilesToColumnStatsRecords(any(), any(), any(), any(), anyInt(), anyInt(), any())) + .thenReturn(records); + + ExposedColumnStatsIndexer indexer = new ExposedColumnStatsIndexer(engineContext, writeConfig, metaClient); + List initializationList = indexer.callGetData("001", "002", files, Lazy.lazily(Collections::emptyList)); + assertEquals(1, initializationList.size()); + + assertEquals(5, initializationList.get(0).totalFileGroups()); + List collected = initializationList.get(0).dataPartitionAndRecords().get(0).indexRecords().collectAsList(); + assertEquals(1, collected.size()); + assertEquals("p_col", collected.get(0).getRecordKey()); + } + } + + private static class ExposedColumnStatsIndexer extends ColumnStatsIndexer { + ExposedColumnStatsIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + List callGetData(String dataTableInstantTime, String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + return buildInitializationData(dataTableInstantTime, instantTimeForPartition, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/expression/TestExpressionIndexer.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/expression/TestExpressionIndexer.java new file mode 100644 index 0000000000000..7d8424a2059cd --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/expression/TestExpressionIndexer.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. + */ + +package org.apache.hudi.metadata.index.expression; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.index.ExpressionIndexRecordGenerator; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.util.Lazy; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +class TestExpressionIndexer { + + @Test + void testSkipWhenMultipleExpressionPartitions() throws IOException { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + + try (MockedStatic mockedUtil = mockStatic(HoodieTableMetadataUtil.class)) { + mockedUtil.when(() -> HoodieTableMetadataUtil.getExpressionIndexPartitionsToInit(any(), any(), any())) + .thenReturn(Set.of("expr1", "expr2")); + + ExposedExpressionIndexer indexer = new ExposedExpressionIndexer(engineContext, writeConfig, metaClient, mock(ExpressionIndexRecordGenerator.class)); + List result = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertTrue(result.isEmpty()); + } + } + + @SuppressWarnings("unchecked") + @Test + void testInitializeWithRealEngineContextAndIndexDataContent() throws IOException { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(getDefaultStorageConf()); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + ExpressionIndexRecordGenerator generator = mock(ExpressionIndexRecordGenerator.class); + HoodieIndexDefinition definition = mock(HoodieIndexDefinition.class); + HoodieSchema tableSchema = mock(HoodieSchema.class); + HoodieSchema projectedSchema = mock(HoodieSchema.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metadataConfig.getExpressionIndexFileGroupCount()).thenReturn(6); + when(metadataConfig.getExpressionIndexParallelism()).thenReturn(10); + + HoodieData records = (HoodieData) (HoodieData) engineContext.parallelize( + Collections.singletonList(HoodieMetadataPayload.createPartitionFilesRecord("p_expr", + Collections.singletonMap("f_expr.parquet", 55L), Collections.emptyList())), + 1); + when(generator.generate(any(), any(), any(), anyInt(), any(), any(), any(), any())).thenReturn(records); + + FileSlice fileSlice = new FileSlice("p1", "001", "f1"); + fileSlice.setBaseFile(new HoodieBaseFile("file:///tmp/p1/f1.parquet")); + List fileSlices = Collections.singletonList(FileSliceAndPartition.of("p1", fileSlice)); + + try (MockedStatic mockedUtil = mockStatic(HoodieTableMetadataUtil.class)) { + mockedUtil.when(() -> HoodieTableMetadataUtil.getExpressionIndexPartitionsToInit(any(), any(), any())) + .thenReturn(Collections.singleton("expr_idx")); + mockedUtil.when(() -> HoodieTableMetadataUtil.getHoodieIndexDefinition("expr_idx", metaClient)).thenReturn(definition); + mockedUtil.when(() -> HoodieTableMetadataUtil.tryResolveSchemaForTable(metaClient)).thenReturn(Option.of(tableSchema)); + mockedUtil.when(() -> HoodieTableMetadataUtil.getProjectedSchemaForExpressionIndex(definition, metaClient, tableSchema)).thenReturn(projectedSchema); + + ExposedExpressionIndexer indexer = new ExposedExpressionIndexer(engineContext, writeConfig, metaClient, generator); + List initializationList = indexer.callGetData("001", "002", new HashMap<>(), Lazy.lazily(() -> fileSlices)); + assertEquals(1, initializationList.size()); + + assertEquals("expr_idx", initializationList.get(0).indexPartitionName()); + List collected = initializationList.get(0).dataPartitionAndRecords().get(0).indexRecords().collectAsList(); + assertEquals(1, collected.size()); + assertEquals("p_expr", collected.get(0).getRecordKey()); + } + } + + private static class ExposedExpressionIndexer extends ExpressionIndexer { + ExposedExpressionIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient, ExpressionIndexRecordGenerator expressionIndexRecordGenerator) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient, expressionIndexRecordGenerator); + } + + List callGetData(String dataTableInstantTime, String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + return buildInitializationData(dataTableInstantTime, instantTimeForPartition, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/files/TestFilesIndexer.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/files/TestFilesIndexer.java new file mode 100644 index 0000000000000..062b2c36b2160 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/files/TestFilesIndexer.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + */ + +package org.apache.hudi.metadata.index.files; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.util.Lazy; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; + +class TestFilesIndexer { + + @SuppressWarnings("unchecked") + @Test + void testInitializeDataEmptyInput() throws IOException { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieData allPartitions = mock(HoodieData.class); + + when(engineContext.parallelize(any(List.class), org.mockito.ArgumentMatchers.eq(1))).thenReturn(allPartitions); + + ExposedFilesIndexer indexer = new ExposedFilesIndexer(engineContext, writeConfig, metaClient); + List initializationList = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertEquals(1, initializationList.size()); + + assertEquals(MetadataPartitionType.FILES.getPartitionPath(), initializationList.get(0).indexPartitionName()); + assertSame(allPartitions, initializationList.get(0).dataPartitionAndRecords().get(0).indexRecords()); + } + + @Test + void testInitializeDataWithRealEngineContext() throws IOException { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(getDefaultStorageConf()); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + + Map> files = new HashMap<>(); + files.put("p1", Collections.singletonMap("f1.parquet", 100L)); + files.put("p2", Collections.singletonMap("f2.parquet", 200L)); + + ExposedFilesIndexer indexer = new ExposedFilesIndexer(engineContext, writeConfig, metaClient); + List initializationList = indexer.callGetData("001", "002", files, Lazy.lazily(Collections::emptyList)); + assertEquals(1, initializationList.size()); + + assertEquals(MetadataPartitionType.FILES.getPartitionPath(), initializationList.get(0).indexPartitionName()); + // 1 partition-list record + 2 partition-file records + HoodieData indexRecords = initializationList.get(0).dataPartitionAndRecords().get(0).indexRecords(); + assertEquals(3L, indexRecords.count()); + + List records = indexRecords.collectAsList(); + Set keys = records.stream().map(HoodieRecord::getRecordKey).collect(Collectors.toSet()); + assertEquals(new HashSet<>(Arrays.asList(HoodieTableMetadata.RECORDKEY_PARTITION_LIST, "p1", "p2")), keys); + + HoodieMetadataPayload partitionListPayload = (HoodieMetadataPayload) records.stream() + .filter(r -> HoodieTableMetadata.RECORDKEY_PARTITION_LIST.equals(r.getRecordKey())) + .findFirst().orElseThrow(() -> new AssertionError("partition-list record not found")) + .getData(); + assertEquals(new HashSet<>(Arrays.asList("p1", "p2")), new HashSet<>(partitionListPayload.getFilenames())); + + Map filesPayloadByPartition = records.stream() + .filter(r -> !HoodieTableMetadata.RECORDKEY_PARTITION_LIST.equals(r.getRecordKey())) + .collect(Collectors.toMap(HoodieRecord::getRecordKey, r -> (HoodieMetadataPayload) r.getData())); + assertEquals(Collections.singletonList("f1.parquet"), filesPayloadByPartition.get("p1").getFilenames()); + assertEquals(Collections.singletonList("f2.parquet"), filesPayloadByPartition.get("p2").getFilenames()); + } + + private static class ExposedFilesIndexer extends FilesIndexer { + ExposedFilesIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + List callGetData(String dataTableInstantTime, String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + return buildInitializationData(dataTableInstantTime, instantTimeForPartition, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/partitionstats/TestPartitionStatsIndexer.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/partitionstats/TestPartitionStatsIndexer.java new file mode 100644 index 0000000000000..2bee124553986 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/partitionstats/TestPartitionStatsIndexer.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. + */ + +package org.apache.hudi.metadata.index.partitionstats; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.util.Lazy; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +class TestPartitionStatsIndexer { + + @Test + void testSkipForNonPartitionedTable() throws IOException { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); + + when(metaClient.getTableConfig()).thenReturn(tableConfig); + when(tableConfig.isTablePartitioned()).thenReturn(false); + + ExposedPartitionStatsIndexer indexer = new ExposedPartitionStatsIndexer(engineContext, writeConfig, metaClient); + List result = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertTrue(result.isEmpty()); + } + + @Test + void testSkipWhenColumnStatsDisabled() throws IOException { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); + + when(metaClient.getTableConfig()).thenReturn(tableConfig); + when(tableConfig.isTablePartitioned()).thenReturn(true); + when(writeConfig.isMetadataColumnStatsIndexEnabled()).thenReturn(false); + + ExposedPartitionStatsIndexer indexer = new ExposedPartitionStatsIndexer(engineContext, writeConfig, metaClient); + List result = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertTrue(result.isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + void testInitializeWithRealEngineContextAndIndexDataContent() throws IOException { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(getDefaultStorageConf()); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieRecordMerger recordMerger = mock(HoodieRecordMerger.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); + + when(metaClient.getTableConfig()).thenReturn(tableConfig); + when(tableConfig.isTablePartitioned()).thenReturn(true); + when(writeConfig.isMetadataColumnStatsIndexEnabled()).thenReturn(true); + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(writeConfig.getRecordMerger()).thenReturn(recordMerger); + when(recordMerger.getRecordType()).thenReturn(HoodieRecord.HoodieRecordType.AVRO); + when(metadataConfig.getPartitionStatsIndexFileGroupCount()).thenReturn(4); + + HoodieData records = (HoodieData) (HoodieData) engineContext.parallelize( + Collections.singletonList(HoodieMetadataPayload.createPartitionFilesRecord("p_part", + Collections.singletonMap("f_part.parquet", 33L), Collections.emptyList())), + 1); + + try (MockedStatic mockedUtil = mockStatic(HoodieTableMetadataUtil.class)) { + mockedUtil.when(() -> HoodieTableMetadataUtil.convertFilesToPartitionStatsRecords(any(), any(), any(), any(), any(), any())) + .thenReturn(records); + + ExposedPartitionStatsIndexer indexer = new ExposedPartitionStatsIndexer(engineContext, writeConfig, metaClient); + List initializationList = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertEquals(1, initializationList.size()); + + assertEquals(4, initializationList.get(0).totalFileGroups()); + List collected = initializationList.get(0).dataPartitionAndRecords().get(0).indexRecords().collectAsList(); + assertEquals(1, collected.size()); + assertEquals("p_part", collected.get(0).getRecordKey()); + } + } + + private static class ExposedPartitionStatsIndexer extends PartitionStatsIndexer { + ExposedPartitionStatsIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + List callGetData(String dataTableInstantTime, String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + return buildInitializationData(dataTableInstantTime, instantTimeForPartition, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/record/TestPartitionedRecordIndexer.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/record/TestPartitionedRecordIndexer.java new file mode 100644 index 0000000000000..d27fe1e3d1b0b --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/record/TestPartitionedRecordIndexer.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. + */ + +package org.apache.hudi.metadata.index.record; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.index.model.DataPartitionAndRecords; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.util.Lazy; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +class TestPartitionedRecordIndexer { + + @SuppressWarnings("unchecked") + @Test + void testInitializeWithRealEngineContextAndIndexDataContent() throws IOException { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(getDefaultStorageConf()); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient dataMetaClient = mock(HoodieTableMetaClient.class); + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metadataConfig.getRecordIndexMaxParallelism()).thenReturn(8); + when(dataMetaClient.getTableConfig()).thenReturn(tableConfig); + + HoodieData p1Data = (HoodieData) (HoodieData) engineContext.parallelize( + Collections.singletonList(HoodieMetadataPayload.createPartitionFilesRecord("p1_data", + Collections.singletonMap("f1.parquet", 1L), Collections.emptyList())), + 1); + HoodieData p2Data = (HoodieData) (HoodieData) engineContext.parallelize( + Collections.singletonList(HoodieMetadataPayload.createPartitionFilesRecord("p2_data", + Collections.singletonMap("f2.parquet", 2L), Collections.emptyList())), + 1); + + DataPartitionAndRecords p1Init = new DataPartitionAndRecords(1, Option.of("p1"), p1Data); + DataPartitionAndRecords p2Init = new DataPartitionAndRecords(2, Option.of("p2"), p2Data); + + FileSliceAndPartition fs1 = FileSliceAndPartition.of("p1", new FileSlice("p1", "001", "f1")); + FileSliceAndPartition fs2 = FileSliceAndPartition.of("p2", new FileSlice("p2", "001", "f2")); + List input = Arrays.asList(fs1, fs2); + + ExposedPartitionedRecordIndexer indexer = new ExposedPartitionedRecordIndexer(engineContext, writeConfig, dataMetaClient, p1Init, p2Init); + + try (MockedStatic mockedUtil = mockStatic(org.apache.hudi.metadata.HoodieTableMetadataUtil.class)) { + List initializationList = indexer.buildInitializationData("001", "002", Map.of(), Lazy.lazily(() -> input)); + + mockedUtil.verify(() -> org.apache.hudi.metadata.HoodieTableMetadataUtil.createRecordIndexDefinition(any(), any())); + assertEquals(1, initializationList.size()); + assertEquals(2, initializationList.get(0).dataPartitionAndRecords().size()); + assertEquals(2, indexer.initializePartitionCalls); + } + } + + private static class ExposedPartitionedRecordIndexer extends PartitionedRecordIndexer { + private final DataPartitionAndRecords p1; + private final DataPartitionAndRecords p2; + private int initializePartitionCalls; + + ExposedPartitionedRecordIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient, + DataPartitionAndRecords p1, + DataPartitionAndRecords p2) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + this.p1 = p1; + this.p2 = p2; + } + + @Override + protected DataPartitionAndRecords initializeRecordIndexPartition(String dataPartition, List latestMergedPartitionFileSliceList, + int recordIndexMaxParallelism) { + initializePartitionCalls++; + if ("p1".equals(dataPartition)) { + return p1; + } + return p2; + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/record/TestRecordIndexer.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/record/TestRecordIndexer.java new file mode 100644 index 0000000000000..d387aea774d23 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/record/TestRecordIndexer.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. + */ + +package org.apache.hudi.metadata.index.record; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.index.model.DataPartitionAndRecords; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.util.Lazy; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +class TestRecordIndexer { + + @Test + void testGetDataCreatesDefinitionAndReturnsInitialization() throws IOException { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieData records = mock(HoodieData.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metadataConfig.getRecordIndexMaxParallelism()).thenReturn(4); + + DataPartitionAndRecords init = new DataPartitionAndRecords(2, Option.empty(), records); + ExposedRecordIndexer indexer = new ExposedRecordIndexer(engineContext, writeConfig, metaClient, init); + + try (MockedStatic mockedUtil = mockStatic(org.apache.hudi.metadata.HoodieTableMetadataUtil.class)) { + List result = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertEquals(1, result.size()); + assertEquals(2, result.get(0).totalFileGroups()); + mockedUtil.verify(() -> org.apache.hudi.metadata.HoodieTableMetadataUtil.createRecordIndexDefinition(any(), any()), times(1)); + } + } + + @Test + void testPostInitializationValidationAndUnpersist() { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); + HoodieData records = mock(HoodieData.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metaClient.getTableConfig()).thenReturn(tableConfig); + + ExposedRecordIndexer indexer = new ExposedRecordIndexer( + engineContext, writeConfig, metaClient, new DataPartitionAndRecords(1, Option.empty(), records)); + + when(metadataConfig.isRecordIndexInitializationValidationEnabled()).thenReturn(false); + indexer.callPost(metaClient, IndexPartitionInitialization.of(1, "record_index", records), "record_index"); + assertFalse(indexer.validateCalled); + verify(records, times(1)).unpersist(); + + when(metadataConfig.isRecordIndexInitializationValidationEnabled()).thenReturn(true); + indexer.callPost(metaClient, IndexPartitionInitialization.of(1, "record_index", records), "record_index"); + assertTrue(indexer.validateCalled); + } + + @SuppressWarnings("unchecked") + @Test + void testGetDataWithRealEngineContextAndIndexDataContent() throws IOException { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(getDefaultStorageConf()); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metadataConfig.getRecordIndexMaxParallelism()).thenReturn(4); + + HoodieData records = (HoodieData) (HoodieData) engineContext.parallelize( + Collections.singletonList(HoodieMetadataPayload.createPartitionFilesRecord("p_record", + Collections.singletonMap("f_record.parquet", 66L), Collections.emptyList())), + 1); + + ExposedRecordIndexer indexer = new ExposedRecordIndexer( + engineContext, writeConfig, metaClient, new DataPartitionAndRecords(2, Option.empty(), records)); + + try (MockedStatic mockedUtil = mockStatic(org.apache.hudi.metadata.HoodieTableMetadataUtil.class)) { + List result = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertEquals(1, result.size()); + assertEquals(1, result.get(0).dataPartitionAndRecords().get(0).indexRecords().collectAsList().size()); + assertEquals("p_record", result.get(0).dataPartitionAndRecords().get(0).indexRecords().collectAsList().get(0).getRecordKey()); + mockedUtil.verify(() -> org.apache.hudi.metadata.HoodieTableMetadataUtil.createRecordIndexDefinition(any(), any()), times(1)); + } + } + + private static class ExposedRecordIndexer extends RecordIndexer { + private final DataPartitionAndRecords predefined; + private boolean validateCalled; + + ExposedRecordIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, + HoodieTableMetaClient dataTableMetaClient, DataPartitionAndRecords predefined) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + this.predefined = predefined; + } + + @Override + protected DataPartitionAndRecords initializeRecordIndexPartition(List latestMergedPartitionFileSliceList, + int recordIndexMaxParallelism) { + return predefined; + } + + @Override + protected void validateRecordIndex(HoodieData recordIndexRecords, int fileGroupCount, HoodieTableMetaClient metadataMetaClient) { + validateCalled = true; + } + + List callGetData(String dataTableInstantTime, String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + return buildInitializationData(dataTableInstantTime, instantTimeForPartition, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + } + + void callPost(HoodieTableMetaClient metadataMetaClient, IndexPartitionInitialization indexPartitionInitialization, String relativePartitionPath) { + postInitialization(metadataMetaClient, indexPartitionInitialization.dataPartitionAndRecords().get(0).indexRecords(), + indexPartitionInitialization.totalFileGroups(), relativePartitionPath); + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/secondary/TestSecondaryIndexer.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/secondary/TestSecondaryIndexer.java new file mode 100644 index 0000000000000..a88c136754f6e --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metadata/index/secondary/TestSecondaryIndexer.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. + */ + +package org.apache.hudi.metadata.index.secondary; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.index.model.IndexPartitionInitialization; +import org.apache.hudi.metadata.model.FileSliceAndPartition; +import org.apache.hudi.util.Lazy; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyFloat; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +class TestSecondaryIndexer { + + @Test + void testSkipWhenMultipleSecondaryPartitions() throws IOException { + HoodieEngineContext engineContext = mock(HoodieEngineContext.class); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + + try (MockedStatic mockedUtil = mockStatic(HoodieTableMetadataUtil.class)) { + mockedUtil.when(() -> HoodieTableMetadataUtil.getSecondaryIndexPartitionsToInit(any(), any(), any())) + .thenReturn(Set.of("sec1", "sec2")); + + ExposedSecondaryIndexer indexer = new ExposedSecondaryIndexer(engineContext, writeConfig, metaClient); + List result = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertTrue(result.isEmpty()); + } + } + + @SuppressWarnings("unchecked") + @Test + void testInitializeWithRealEngineContextAndIndexDataContent() throws IOException { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(getDefaultStorageConf()); + HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class); + HoodieMetadataConfig metadataConfig = mock(HoodieMetadataConfig.class); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + HoodieIndexDefinition definition = mock(HoodieIndexDefinition.class); + + when(writeConfig.getMetadataConfig()).thenReturn(metadataConfig); + when(metadataConfig.getSecondaryIndexParallelism()).thenReturn(8); + when(writeConfig.getProps()).thenReturn(new TypedProperties()); + + HoodieData records = (HoodieData) (HoodieData) engineContext.parallelize( + Collections.singletonList(HoodieMetadataPayload.createPartitionFilesRecord("p_sec", + Collections.singletonMap("f_sec.parquet", 44L), Collections.emptyList())), + 1); + + try (MockedStatic mockedUtil = mockStatic(HoodieTableMetadataUtil.class); + MockedStatic mockedSecondaryUtil = mockStatic(org.apache.hudi.metadata.SecondaryIndexRecordGenerationUtils.class)) { + mockedUtil.when(() -> HoodieTableMetadataUtil.getSecondaryIndexPartitionsToInit(any(), any(), any())) + .thenReturn(Collections.singleton("sec_idx")); + mockedUtil.when(() -> HoodieTableMetadataUtil.getHoodieIndexDefinition("sec_idx", metaClient)).thenReturn(definition); + mockedSecondaryUtil.when(() -> org.apache.hudi.metadata.SecondaryIndexRecordGenerationUtils.readSecondaryKeysFromFileSlices(any(), any(), anyInt(), any(), any(), any(), any())) + .thenReturn(records); + mockedUtil.when(() -> HoodieTableMetadataUtil.estimateFileGroupCount(any(), any(), anyInt(), anyInt(), anyInt(), anyFloat(), anyLong())) + .thenReturn(7); + + ExposedSecondaryIndexer indexer = new ExposedSecondaryIndexer(engineContext, writeConfig, metaClient); + List initializationList = indexer.callGetData("001", "002", Collections.emptyMap(), Lazy.lazily(Collections::emptyList)); + assertEquals(1, initializationList.size()); + + assertEquals("sec_idx", initializationList.get(0).indexPartitionName()); + assertEquals(7, initializationList.get(0).totalFileGroups()); + List collected = initializationList.get(0).dataPartitionAndRecords().get(0).indexRecords().collectAsList(); + assertEquals(1, collected.size()); + assertEquals("p_sec", collected.get(0).getRecordKey()); + } + } + + private static class ExposedSecondaryIndexer extends SecondaryIndexer { + ExposedSecondaryIndexer(HoodieEngineContext engineContext, HoodieWriteConfig dataTableWriteConfig, HoodieTableMetaClient dataTableMetaClient) { + super(engineContext, dataTableWriteConfig, dataTableMetaClient); + } + + List callGetData(String dataTableInstantTime, String instantTimeForPartition, + Map> partitionIdToAllFilesMap, + Lazy> lazyLatestMergedPartitionFileSliceList) throws IOException { + return buildInitializationData(dataTableInstantTime, instantTimeForPartition, partitionIdToAllFilesMap, lazyLatestMergedPartitionFileSliceList); + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index 268ae942b92e9..c8020a0f25b8e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -27,16 +27,14 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.schema.HoodieSchema; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.metadata.index.UnsupportedExpressionIndexRecordGenerator; import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.table.BulkInsertPartitioner; @@ -82,7 +80,7 @@ public static HoodieTableMetadataWriter create(StorageConfiguration conf, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, Option inFlightInstantTimestamp) { - super(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, inFlightInstantTimestamp); + this(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, inFlightInstantTimestamp, false); } FlinkHoodieBackedTableMetadataWriter(StorageConfiguration storageConf, @@ -91,7 +89,7 @@ public static HoodieTableMetadataWriter create(StorageConfiguration conf, HoodieEngineContext engineContext, Option inFlightInstantTimestamp, boolean streamingWrites) { - super(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, inFlightInstantTimestamp, streamingWrites); + super(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, new UnsupportedExpressionIndexRecordGenerator(EngineType.FLINK), inFlightInstantTimestamp, streamingWrites); } @Override @@ -166,13 +164,6 @@ protected void preWrite(String instantTime) { metadataMetaClient.getActiveTimeline().transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, instantTime); } - @Override - protected HoodieData getExpressionIndexRecords(List>> partitionFilePathAndSizeTriplet, HoodieIndexDefinition indexDefinition, - HoodieTableMetaClient metaClient, int parallelism, HoodieSchema tableSchema, HoodieSchema readerSchema, - StorageConfiguration storageConf, String instantTime) { - throw new HoodieNotSupportedException("Flink metadata table does not support expression index yet."); - } - @Override protected List streamWriteToMetadataTable(Pair, HoodieData> fileGroupIdToTaggedRecords, String instantTime) { return getWriteClient().upsertPreppedRecords(fileGroupIdToTaggedRecords.getValue().collectAsList(), instantTime); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/metadata/JavaHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/metadata/JavaHoodieBackedTableMetadataWriter.java index 2a035f560d5ca..1ea3cdf0bfb87 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/metadata/JavaHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/metadata/JavaHoodieBackedTableMetadataWriter.java @@ -27,15 +27,12 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.schema.HoodieSchema; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.metadata.index.UnsupportedExpressionIndexRecordGenerator; import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.table.BulkInsertPartitioner; @@ -59,7 +56,7 @@ public class JavaHoodieBackedTableMetadataWriter extends HoodieBackedTableMetada protected JavaHoodieBackedTableMetadataWriter(StorageConfiguration storageConf, HoodieWriteConfig writeConfig, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, Option inflightInstantTimestamp) { - super(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, inflightInstantTimestamp); + super(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, new UnsupportedExpressionIndexRecordGenerator(EngineType.JAVA), inflightInstantTimestamp); } public static HoodieTableMetadataWriter create(StorageConfiguration conf, @@ -147,13 +144,6 @@ protected void preWrite(String instantTime) { metadataMetaClient.getActiveTimeline().transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, instantTime); } - @Override - protected HoodieData getExpressionIndexRecords(List>> partitionFilePathAndSizeTriplet, HoodieIndexDefinition indexDefinition, - HoodieTableMetaClient metaClient, int parallelism, HoodieSchema tableSchema, HoodieSchema readerSchema, - StorageConfiguration storageConf, String instantTime) { - throw new HoodieNotSupportedException("Expression index not supported for Java metadata table writer yet."); - } - @Override protected EngineType getEngineType() { return EngineType.JAVA; diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java index e1917cc7cc35d..fdf1b8a135b04 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java @@ -2839,7 +2839,7 @@ private void validateMetadata(HoodieJavaWriteClient testClient, Option i // check if the last instant is restore, then the metadata table should have only the partitions that are not deleted metaClient.reloadActiveTimeline().getReverseOrderedInstants().findFirst().ifPresent(instant -> { if (instant.getAction().equals(HoodieActiveTimeline.RESTORE_ACTION)) { - metadataWriter.getEnabledPartitionTypes().stream().filter(partitionType -> !MetadataPartitionType.shouldDeletePartitionOnRestore(partitionType.getPartitionPath())) + metadataWriter.getEnabledIndexerMap().keySet().stream().filter(partitionType -> !MetadataPartitionType.shouldDeletePartitionOnRestore(partitionType.getPartitionPath())) .forEach(partitionType -> assertTrue(metadataTablePartitions.contains(partitionType.getPartitionPath()))); } }); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java index 47155899e2048..08137261ac5f9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java @@ -65,6 +65,7 @@ import org.apache.hudi.metadata.HoodieMetadataWriteUtils; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.model.FileToIndex; import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.stats.SparkValueMetadataUtils; import org.apache.hudi.stats.ValueMetadata; @@ -260,20 +261,21 @@ private static void setBloomFilterProps(HoodieStorageConfig storageConfig, Map>> partitionFilePathAndSizeTriplet, HoodieIndexDefinition indexDefinition, + List filesToIndex, HoodieIndexDefinition indexDefinition, HoodieTableMetaClient metaClient, int parallelism, HoodieSchema tableSchema, HoodieSchema readerSchema, String instantTime, HoodieEngineContext engineContext, HoodieWriteConfig dataWriteConfig, Option>, HoodieData>> partitionRecordsFunctionOpt) { @@ -290,8 +292,8 @@ public static ExpressionIndexComputationMetadata getExprIndexRecords( ReaderContextFactory readerContextFactory = engineContext.getReaderContextFactory(metaClient); // Read records and append expression index metadata to every row - HoodieData rowData = sparkEngineContext.parallelize(partitionFilePathAndSizeTriplet, parallelism) - .flatMap((SerializableFunction>, Iterator>) entry -> + HoodieData rowData = sparkEngineContext.parallelize(filesToIndex, parallelism) + .flatMap((SerializableFunction>) entry -> getExpressionIndexRecordsIterator(readerContextFactory.getContext(), metaClient, tableSchema, readerSchema, dataWriteConfig, entry)); // Generate dataset with expression index metadata @@ -319,12 +321,11 @@ public static ExpressionIndexComputationMetadata getExprIndexRecords( } private static Iterator getExpressionIndexRecordsIterator(HoodieReaderContext readerContext, HoodieTableMetaClient metaClient, - HoodieSchema tableSchema, HoodieSchema readerSchema, HoodieWriteConfig dataWriteConfig, Pair> entry) { - String partition = entry.getKey(); - Pair filePathSizePair = entry.getValue(); - String filePath = filePathSizePair.getKey(); + HoodieSchema tableSchema, HoodieSchema readerSchema, HoodieWriteConfig dataWriteConfig, FileToIndex entry) { + String partition = entry.partition(); + String filePath = entry.path(); String relativeFilePath = FSUtils.getRelativePartitionPath(metaClient.getBasePath(), new StoragePath(filePath)); - long fileSize = filePathSizePair.getValue(); + long fileSize = entry.size(); boolean isBaseFile = FSUtils.isBaseFile(new StoragePath(filePath.substring(filePath.lastIndexOf("/") + 1))); Stream logFileStream; Option baseFileOption; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index dfb295d5d16a3..1d07d94194f31 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -38,7 +38,6 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.schema.HoodieSchema; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; @@ -47,7 +46,8 @@ import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieSparkIndexClient; import org.apache.hudi.index.expression.HoodieSparkExpressionIndex; -import org.apache.hudi.index.expression.HoodieSparkExpressionIndex.ExpressionIndexComputationMetadata; +import org.apache.hudi.metadata.index.SparkExpressionIndexRecordGenerator; +import org.apache.hudi.metadata.model.FileToIndex; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.hudi.metrics.MetricsReporterType; import org.apache.hudi.stats.HoodieColumnRangeMetadata; @@ -124,7 +124,8 @@ public static HoodieTableMetadataWriter create(StorageConfiguration conf, Hoo HoodieEngineContext engineContext, Option inflightInstantTimestamp, boolean streamingWrites) { - super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, inflightInstantTimestamp, streamingWrites); + super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, + new SparkExpressionIndexRecordGenerator(engineContext, writeConfig), inflightInstantTimestamp, streamingWrites); } @Override @@ -234,7 +235,7 @@ public void deletePartitions(String instantTime, List par */ @Override protected HoodieData getExpressionIndexUpdates(HoodieCommitMetadata commitMetadata, String indexPartition, String instantTime) throws Exception { - HoodieIndexDefinition indexDefinition = getIndexDefinition(indexPartition); + HoodieIndexDefinition indexDefinition = HoodieTableMetadataUtil.getHoodieIndexDefinition(indexPartition, dataMetaClient); boolean isExprIndexUsingColumnStats = indexDefinition.getIndexType().equals(PARTITION_NAME_COLUMN_STATS); Option>, HoodieData>> partitionRecordsFunctionOpt = Option.empty(); if (isExprIndexUsingColumnStats) { @@ -250,10 +251,10 @@ commitMetadata, indexPartition, engineContext, getTableMetadata(), dataMetaClien } // Step 1: Generate partition name, file path and size triplets from the newly created files in the commit metadata - List>> partitionFilePathPairs = new ArrayList<>(); - commitMetadata.getPartitionToWriteStats().forEach((dataPartition, writeStats) -> writeStats.forEach(writeStat -> partitionFilePathPairs.add( - Pair.of(writeStat.getPartitionPath(), Pair.of(new StoragePath(dataMetaClient.getBasePath(), writeStat.getPath()).toString(), writeStat.getFileSizeInBytes()))))); - int parallelism = Math.min(partitionFilePathPairs.size(), dataWriteConfig.getMetadataConfig().getExpressionIndexParallelism()); + List filesToIndex = new ArrayList<>(); + commitMetadata.getPartitionToWriteStats().forEach((dataPartition, writeStats) -> writeStats.forEach(writeStat -> filesToIndex.add( + FileToIndex.of(writeStat.getPartitionPath(), new StoragePath(dataMetaClient.getBasePath(), writeStat.getPath()).toString(), writeStat.getFileSizeInBytes())))); + int parallelism = Math.min(filesToIndex.size(), dataWriteConfig.getMetadataConfig().getExpressionIndexParallelism()); HoodieSchema tableSchema = new TableSchemaResolver(dataMetaClient).getTableSchema(); HoodieSchema readerSchema = getProjectedSchemaForExpressionIndex(indexDefinition, dataMetaClient, tableSchema); // Step 2: Compute the expression index column stat and partition stat records for these newly created files @@ -262,30 +263,13 @@ commitMetadata, indexPartition, engineContext, getTableMetadata(), dataMetaClien // In the partitionRecordsFunctionOpt function we merge the expression index records from the new files created in the commit metadata // with the expression index records from the unmodified files to get the new partition stat records HoodieSparkExpressionIndex.ExpressionIndexComputationMetadata expressionIndexComputationMetadata = - SparkMetadataWriterUtils.getExprIndexRecords(partitionFilePathPairs, indexDefinition, dataMetaClient, parallelism, tableSchema, readerSchema, instantTime, engineContext, dataWriteConfig, + SparkMetadataWriterUtils.getExprIndexRecords(filesToIndex, indexDefinition, dataMetaClient, parallelism, tableSchema, readerSchema, instantTime, engineContext, dataWriteConfig, partitionRecordsFunctionOpt); return expressionIndexComputationMetadata.getPartitionStatRecordsOpt().isPresent() ? expressionIndexComputationMetadata.getExpressionIndexRecords().union(expressionIndexComputationMetadata.getPartitionStatRecordsOpt().get()) : expressionIndexComputationMetadata.getExpressionIndexRecords(); } - @Override - protected HoodieData getExpressionIndexRecords(List>> partitionFilePathAndSizeTriplet, - HoodieIndexDefinition indexDefinition, - HoodieTableMetaClient metaClient, int parallelism, - HoodieSchema tableSchema, HoodieSchema readerSchema, StorageConfiguration storageConf, - String instantTime) { - ExpressionIndexComputationMetadata expressionIndexComputationMetadata = SparkMetadataWriterUtils.getExprIndexRecords(partitionFilePathAndSizeTriplet, indexDefinition, - metaClient, parallelism, tableSchema, readerSchema, instantTime, engineContext, dataWriteConfig, - Option.of(rangeMetadata -> - HoodieTableMetadataUtil.collectAndProcessExprIndexPartitionStatRecords(rangeMetadata, true, Option.of(indexDefinition.getIndexName())))); - HoodieData exprIndexRecords = expressionIndexComputationMetadata.getExpressionIndexRecords(); - if (indexDefinition.getIndexType().equals(PARTITION_NAME_COLUMN_STATS)) { - exprIndexRecords = exprIndexRecords.union(expressionIndexComputationMetadata.getPartitionStatRecordsOpt().get()); - } - return exprIndexRecords; - } - protected SparkRDDMetadataWriteClient getSparkWriteClient(Option, ?, JavaRDD>> writeClientOpt) { return ((SparkRDDMetadataWriteClient) writeClientOpt.orElse(getWriteClient())); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriterTableVersionSix.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriterTableVersionSix.java index 617b4b4e820f9..07a2928f3ce76 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriterTableVersionSix.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriterTableVersionSix.java @@ -30,19 +30,16 @@ import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.schema.HoodieSchema; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.HoodieSparkIndexClient; +import org.apache.hudi.metadata.index.UnsupportedExpressionIndexRecordGenerator; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.hudi.metrics.MetricsReporterType; import org.apache.hudi.storage.StorageConfiguration; @@ -92,7 +89,7 @@ public static HoodieTableMetadataWriter create(StorageConfiguration conf, Hoo HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, Option inflightInstantTimestamp) { - super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, inflightInstantTimestamp); + super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, new UnsupportedExpressionIndexRecordGenerator(EngineType.SPARK), inflightInstantTimestamp); } @Override @@ -187,11 +184,4 @@ protected EngineType getEngineType() { protected void updateColumnsToIndexWithColStats(List columnsToIndex) { new HoodieSparkIndexClient(dataWriteConfig, engineContext).createOrUpdateColumnStatsIndexDefinition(dataMetaClient, columnsToIndex); } - - @Override - protected HoodieData getExpressionIndexRecords(List>> partitionFilePathAndSizeTriplet, HoodieIndexDefinition indexDefinition, - HoodieTableMetaClient metaClient, int parallelism, HoodieSchema tableSchema, HoodieSchema readerSchema, - StorageConfiguration storageConf, String instantTime) { - throw new HoodieNotSupportedException("Expression index not supported for Java metadata table writer yet."); - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/index/SparkExpressionIndexRecordGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/index/SparkExpressionIndexRecordGenerator.java new file mode 100644 index 0000000000000..c5e2327c397b7 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/index/SparkExpressionIndexRecordGenerator.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.index; + +import org.apache.hudi.client.utils.SparkMetadataWriterUtils; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.index.expression.HoodieSparkExpressionIndex; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.model.FileToIndex; +import org.apache.hudi.storage.StorageConfiguration; + +import lombok.extern.slf4j.Slf4j; + +import java.util.List; + +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; + +/** + * Spark implementation of {@link ExpressionIndexRecordGenerator}. + */ +@Slf4j +public class SparkExpressionIndexRecordGenerator implements ExpressionIndexRecordGenerator { + private final HoodieEngineContext engineContext; + private final HoodieWriteConfig dataTableWriteConfig; + + public SparkExpressionIndexRecordGenerator(HoodieEngineContext engineContext, + HoodieWriteConfig dataTableWriteConfig) { + this.engineContext = engineContext; + this.dataTableWriteConfig = dataTableWriteConfig; + } + + @Override + public EngineType getEngineType() { + return EngineType.SPARK; + } + + @Override + public HoodieData generate( + List filesToIndex, + HoodieIndexDefinition indexDefinition, + HoodieTableMetaClient metaClient, + int parallelism, HoodieSchema tableSchema, + HoodieSchema readerSchema, + StorageConfiguration storageConf, + String instantTime) { + if (metaClient.getTableConfig().getTableVersion().lesserThan(HoodieTableVersion.EIGHT)) { + throw new HoodieNotSupportedException("Hudi tables prior to version 8 do not support expression index."); + } + HoodieSparkExpressionIndex.ExpressionIndexComputationMetadata expressionIndexComputationMetadata = SparkMetadataWriterUtils.getExprIndexRecords( + filesToIndex, indexDefinition, metaClient, parallelism, tableSchema, readerSchema, instantTime, engineContext, dataTableWriteConfig, + Option.of(rangeMetadata -> + HoodieTableMetadataUtil.collectAndProcessExprIndexPartitionStatRecords(rangeMetadata, true, Option.of(indexDefinition.getIndexName())))); + HoodieData exprIndexRecords = expressionIndexComputationMetadata.getExpressionIndexRecords(); + if (indexDefinition.getIndexType().equals(PARTITION_NAME_COLUMN_STATS)) { + exprIndexRecords = exprIndexRecords.union(expressionIndexComputationMetadata.getPartitionStatRecordsOpt().get()); + } + return exprIndexRecords; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java index c7244a30d094c..ef6efa4647499 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java @@ -165,7 +165,7 @@ protected void incrementMetric(String action, long value) { gaugeOpt.ifPresent(gauge -> gauge.setValue(gauge.getValue() + value)); } - protected void setMetric(String action, long value) { + public void setMetric(String action, long value) { metrics.registerGauge(action, value); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index bfa0c803fe64e..699bb5f84813f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -56,6 +56,8 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.metadata.model.FileAndPartitionFlag; +import org.apache.hudi.metadata.model.FileSliceAndPartition; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieDeltaWriteStat; @@ -64,7 +66,6 @@ import org.apache.hudi.common.model.HoodieIndexMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieRecordGlobalLocation; @@ -103,7 +104,6 @@ import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.common.util.collection.Tuple3; import org.apache.hudi.common.util.hash.ColumnIndexID; import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieException; @@ -119,11 +119,9 @@ import org.apache.hudi.storage.HoodieStorageUtils; import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.StoragePath; -import org.apache.hudi.storage.StoragePathInfo; import org.apache.hudi.util.Lazy; import lombok.AccessLevel; -import lombok.Getter; import lombok.NoArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.avro.AvroTypeException; @@ -134,7 +132,6 @@ import java.io.FileNotFoundException; import java.io.IOException; -import java.io.Serializable; import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.ByteBuffer; @@ -175,7 +172,6 @@ import static org.apache.hudi.common.model.HoodieRecord.PARTITION_PATH_METADATA_FIELD; import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD; import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; -import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.ConfigUtils.getReaderConfigs; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; @@ -607,42 +603,7 @@ public static HoodieData convertMetadataToBloomFilterRecords(Hoodi }); } - /** - * Convert the clean action to metadata records. - */ - public static Map> convertMetadataToRecords(HoodieEngineContext engineContext, - HoodieCleanMetadata cleanMetadata, - String instantTime, - HoodieTableMetaClient dataMetaClient, - HoodieMetadataConfig metadataConfig, - List enabledPartitionTypes, - int bloomIndexParallelism, - Option recordTypeOpt) { - final Map> partitionToRecordsMap = new HashMap<>(); - final HoodieData filesPartitionRecordsRDD = engineContext.parallelize( - convertMetadataToFilesPartitionRecords(cleanMetadata, instantTime), 1); - partitionToRecordsMap.put(MetadataPartitionType.FILES.getPartitionPath(), filesPartitionRecordsRDD); - if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { - final HoodieData metadataBloomFilterRecordsRDD = - convertMetadataToBloomFilterRecords(cleanMetadata, engineContext, instantTime, bloomIndexParallelism); - partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS.getPartitionPath(), metadataBloomFilterRecordsRDD); - } - - if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { - final HoodieData metadataColumnStatsRDD = - convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, - dataMetaClient, metadataConfig, recordTypeOpt); - partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS.getPartitionPath(), metadataColumnStatsRDD); - } - if (enabledPartitionTypes.contains(MetadataPartitionType.EXPRESSION_INDEX)) { - convertMetadataToExpressionIndexRecords(engineContext, cleanMetadata, instantTime, dataMetaClient, metadataConfig, bloomIndexParallelism, partitionToRecordsMap, - recordTypeOpt); - } - - return partitionToRecordsMap; - } - - private static void convertMetadataToExpressionIndexRecords(HoodieEngineContext engineContext, HoodieCleanMetadata cleanMetadata, + public static void convertMetadataToExpressionIndexRecords(HoodieEngineContext engineContext, HoodieCleanMetadata cleanMetadata, String instantTime, HoodieTableMetaClient dataMetaClient, HoodieMetadataConfig metadataConfig, int bloomIndexParallelism, Map> partitionToRecordsMap, @@ -1314,14 +1275,14 @@ public static HoodieData convertFilesToBloomFilterRecords(HoodieEn int bloomIndexParallelism, String bloomFilterType) { // Create the tuple (partition, filename, isDeleted) to handle both deletes and appends - final List> partitionFileFlagTupleList = fetchPartitionFileInfoTriplets(partitionToDeletedFiles, partitionToAppendedFiles); + final List partitionFileFlagTupleList = fetchPartitionFileInfoTriplets(partitionToDeletedFiles, partitionToAppendedFiles); // Create records MDT int parallelism = Math.max(Math.min(partitionFileFlagTupleList.size(), bloomIndexParallelism), 1); return engineContext.parallelize(partitionFileFlagTupleList, parallelism).flatMap(partitionFileFlagTuple -> { - final String partitionName = partitionFileFlagTuple.f0; - final String filename = partitionFileFlagTuple.f1; - final boolean isDeleted = partitionFileFlagTuple.f2; + final String partitionName = partitionFileFlagTuple.partitionPath(); + final String filename = partitionFileFlagTuple.fileName(); + final boolean isDeleted = partitionFileFlagTuple.flag(); if (!FSUtils.isBaseFile(new StoragePath(filename))) { log.info("Ignoring file {} as it is not a base file", filename); return Stream.empty().iterator(); @@ -1342,7 +1303,7 @@ public static HoodieData convertFilesToBloomFilterRecords(HoodieEn } return Stream.of(HoodieMetadataPayload.createBloomFilterMetadataRecord( - partitionName, filename, instantTime, bloomFilterType, bloomFilterBuffer, partitionFileFlagTuple.f2)) + partitionName, filename, instantTime, bloomFilterType, bloomFilterBuffer, isDeleted)) .iterator(); }); } @@ -1354,7 +1315,6 @@ public static HoodieData convertFilesToColumnStatsRecords(HoodieEn Map> partitionToDeletedFiles, Map> partitionToAppendedFiles, HoodieTableMetaClient dataMetaClient, - HoodieMetadataConfig metadataConfig, int columnStatsIndexParallelism, int maxReaderBufferSize, List columnsToIndex) { @@ -1364,14 +1324,14 @@ public static HoodieData convertFilesToColumnStatsRecords(HoodieEn log.info("Indexing {} columns for column stats index", columnsToIndex.size()); // Create the tuple (partition, filename, isDeleted) to handle both deletes and appends - final List> partitionFileFlagTupleList = fetchPartitionFileInfoTriplets(partitionToDeletedFiles, partitionToAppendedFiles); + final List partitionFileFlagTupleList = fetchPartitionFileInfoTriplets(partitionToDeletedFiles, partitionToAppendedFiles); // Create records MDT int parallelism = Math.max(Math.min(partitionFileFlagTupleList.size(), columnStatsIndexParallelism), 1); return engineContext.parallelize(partitionFileFlagTupleList, parallelism).flatMap(partitionFileFlagTuple -> { - final String partitionPath = partitionFileFlagTuple.f0; - final String filename = partitionFileFlagTuple.f1; - final boolean isDeleted = partitionFileFlagTuple.f2; + final String partitionPath = partitionFileFlagTuple.partitionPath(); + final String filename = partitionFileFlagTuple.fileName(); + final boolean isDeleted = partitionFileFlagTuple.flag(); return getColumnStatsRecords(partitionPath, filename, dataMetaClient, columnsToIndex, isDeleted, maxReaderBufferSize).iterator(); }); } @@ -1388,19 +1348,19 @@ private static ByteBuffer readBloomFilter(HoodieStorage storage, StoragePath fil } } - private static List> fetchPartitionFileInfoTriplets( + private static List fetchPartitionFileInfoTriplets( Map> partitionToDeletedFiles, Map> partitionToAppendedFiles) { // Total number of files which are added or deleted final int totalFiles = partitionToDeletedFiles.values().stream().mapToInt(List::size).sum() + partitionToAppendedFiles.values().stream().mapToInt(Map::size).sum(); - final List> partitionFileFlagTupleList = new ArrayList<>(totalFiles); + final List partitionFileFlagTupleList = new ArrayList<>(totalFiles); partitionToDeletedFiles.entrySet().stream() - .flatMap(entry -> entry.getValue().stream().map(deletedFile -> Tuple3.of(entry.getKey(), deletedFile, true))) + .flatMap(entry -> entry.getValue().stream().map(deletedFile -> FileAndPartitionFlag.of(entry.getKey(), deletedFile, true))) .collect(Collectors.toCollection(() -> partitionFileFlagTupleList)); partitionToAppendedFiles.entrySet().stream() .flatMap( - entry -> entry.getValue().keySet().stream().map(addedFile -> Tuple3.of(entry.getKey(), addedFile, false))) + entry -> entry.getValue().keySet().stream().map(addedFile -> FileAndPartitionFlag.of(entry.getKey(), addedFile, false))) .collect(Collectors.toCollection(() -> partitionFileFlagTupleList)); return partitionFileFlagTupleList; } @@ -2665,7 +2625,7 @@ public static HoodieData collectAndProcessExprIndexPartitionStatRe } public static HoodieData convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext, - List> partitionInfoList, + List partitionInfoList, HoodieMetadataConfig metadataConfig, HoodieTableMetaClient dataTableMetaClient, Lazy> lazyWriterSchemaOpt, @@ -2684,8 +2644,8 @@ public static HoodieData convertFilesToPartitionStatsRecords(Hoodi // Group by partition path and collect file names (BaseFile and LogFiles) List>> partitionToFileNames = partitionInfoList.stream() - .collect(Collectors.groupingBy(Pair::getLeft, - Collectors.mapping(pair -> extractFileNames(pair.getRight()), Collectors.toList()))) + .collect(Collectors.groupingBy(FileSliceAndPartition::partitionPath, + Collectors.mapping(pair -> extractFileNames(pair.fileSlice()), Collectors.toList()))) .entrySet().stream() .map(entry -> Pair.of(entry.getKey(), entry.getValue().stream().flatMap(Set::stream).collect(Collectors.toSet()))) @@ -3058,8 +3018,7 @@ private static Set getIndexPartitionsToInitBasedOnIndexDefinition(Metada return indexPartitions; } - static void createRecordIndexDefinition(HoodieTableMetaClient metaClient, - Map options) { + public static void createRecordIndexDefinition(HoodieTableMetaClient metaClient, Map options) { String indexName = PARTITION_NAME_RECORD_INDEX; HoodieTableVersion tableVersion = metaClient.getTableConfig().getTableVersion(); HoodieIndexVersion indexVersion = HoodieIndexVersion.getCurrentVersion(tableVersion, MetadataPartitionType.RECORD_INDEX); @@ -3073,61 +3032,6 @@ static void createRecordIndexDefinition(HoodieTableMetaClient metaClient, metaClient.buildIndexDefinition(indexDefinition); } - /** - * A class which represents a directory and the files and directories inside it. - *

- * A {@code PartitionFileInfo} object saves the name of the partition and various properties requires of each file - * required for initializing the metadata table. Saving limited properties reduces the total memory footprint when - * a very large number of files are present in the dataset being initialized. - */ - @Getter(AccessLevel.PACKAGE) - public static class DirectoryInfo implements Serializable { - - // Relative path of the directory (relative to the base directory) - private final String relativePath; - // Map of filenames within this partition to their respective sizes - private final Map filenameToSizeMap; - // List of directories within this partition - private final List subDirectories = new ArrayList<>(); - // Is this a hoodie partition - private boolean isHoodiePartition = false; - - public DirectoryInfo(String relativePath, List pathInfos, String maxInstantTime, Set pendingDataInstants) { - this(relativePath, pathInfos, maxInstantTime, pendingDataInstants, true); - } - - /** - * When files are directly fetched from Metadata table we do not need to validate HoodiePartitions. - */ - public DirectoryInfo(String relativePath, List pathInfos, String maxInstantTime, Set pendingDataInstants, - boolean validateHoodiePartitions) { - this.relativePath = relativePath; - - // Pre-allocate with the maximum length possible - filenameToSizeMap = new HashMap<>(pathInfos.size()); - - // Presence of partition meta file implies this is a HUDI partition - // if input files are directly fetched from MDT, it may not contain the HoodiePartitionMetadata file. So, we can ignore the validation for isHoodiePartition. - isHoodiePartition = !validateHoodiePartitions || pathInfos.stream().anyMatch(status -> status.getPath().getName().startsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX)); - for (StoragePathInfo pathInfo : pathInfos) { - // Do not attempt to search for more subdirectories inside directories that are partitions - if (!isHoodiePartition && pathInfo.isDirectory()) { - // Ignore .hoodie directory as there cannot be any partitions inside it - if (!pathInfo.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)) { - this.subDirectories.add(pathInfo.getPath()); - } - } else if (isHoodiePartition && FSUtils.isDataFile(pathInfo.getPath())) { - // Regular HUDI data file (base file or log file) - String dataFileCommitTime = FSUtils.getCommitTime(pathInfo.getPath().getName()); - // Limit the file listings to files which were created by successful commits before the maxInstant time. - if (!pendingDataInstants.contains(dataFileCommitTime) && compareTimestamps(dataFileCommitTime, LESSER_THAN_OR_EQUALS, maxInstantTime)) { - filenameToSizeMap.put(pathInfo.getPath().getName(), pathInfo.getLength()); - } - } - } - } - } - private static TypedProperties getFileGroupReaderPropertiesFromStorageConf(StorageConfiguration storageConf) { TypedProperties properties = new TypedProperties(); properties.setProperty(MAX_MEMORY_FOR_MERGE.key(), diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java index 7944f00181275..1e9f0b603defb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.index.expression.HoodieExpressionIndex; @@ -243,6 +244,12 @@ public boolean isMetadataPartitionEnabled(HoodieMetadataConfig metadataConfig, H return tableConfig.isTablePartitioned() && metadataConfig.isPartitionStatsIndexEnabled(); } + @Override + public boolean isMetadataPartitionSupported(HoodieTableMetaClient metaClient) { + // Partition stats is supported for partitioned tables only + return metaClient.getTableConfig().isTablePartitioned(); + } + @Override public void constructMetadataPayload(HoodieMetadataPayload payload, GenericRecord record) { constructColumnStatsMetadataPayload(payload, record); @@ -368,6 +375,10 @@ public boolean isMetadataPartitionAvailable(HoodieTableMetaClient metaClient) { return metaClient.getTableConfig().isMetadataPartitionAvailable(this); } + public boolean isMetadataPartitionSupported(HoodieTableMetaClient metaClient) { + return true; + } + MetadataPartitionType(final String partitionPath, final String fileIdPrefix, final int recordType) { this.partitionPath = partitionPath; this.fileIdPrefix = fileIdPrefix; @@ -453,8 +464,25 @@ public static Set getAllPartitionPaths() { */ public static MetadataPartitionType[] getValidValues() { // ALL_PARTITIONS is just another record type in FILES partition + return getValidValues(HoodieTableVersion.current()); + } + + /** + * Returns the set of all valid metadata partition types. Prefer using this method over {@link #values()}. + */ + public static MetadataPartitionType[] getValidValues(HoodieTableVersion tableVersion) { + if (tableVersion.greaterThanOrEquals(HoodieTableVersion.EIGHT)) { + // ALL_PARTITIONS is just another record type in FILES partition + return EnumSet.complementOf(EnumSet.of( + ALL_PARTITIONS)).toArray(new MetadataPartitionType[0]); + } return EnumSet.complementOf(EnumSet.of( - ALL_PARTITIONS)).toArray(new MetadataPartitionType[0]); + ALL_PARTITIONS)) + .stream() + .filter(type -> type != SECONDARY_INDEX + && type != EXPRESSION_INDEX + && type != PARTITION_STATS) + .toArray(MetadataPartitionType[]::new); } /** @@ -464,7 +492,7 @@ public static List getEnabledPartitions(HoodieMetadataCon if (!dataMetadataConfig.isEnabled()) { return Collections.emptyList(); } - return Arrays.stream(getValidValues()) + return Arrays.stream(getValidValues(metaClient.getTableConfig().getTableVersion())) .filter(partitionType -> partitionType.isMetadataPartitionEnabled(dataMetadataConfig, metaClient.getTableConfig()) || partitionType.isMetadataPartitionAvailable(metaClient)) .collect(Collectors.toList()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/model/DirectoryInfo.java b/hudi-common/src/main/java/org/apache/hudi/metadata/model/DirectoryInfo.java new file mode 100644 index 0000000000000..5d7161960e28f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/model/DirectoryInfo.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.model; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathInfo; + +import lombok.Getter; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + +/** + * A class which represents a directory and the files and directories inside it. + *

+ * A {@code PartitionFileInfo} object saves the name of the partition and various properties requires of each file + * required for initializing the metadata table. Saving limited properties reduces the total memory footprint when + * a very large number of files are present in the dataset being initialized. + */ +@Getter +public class DirectoryInfo implements Serializable { + + // Relative path of the directory (relative to the base directory) + private final String relativePath; + // Map of filenames within this partition to their respective sizes + private final Map filenameToSizeMap; + // List of directories within this partition + private final List subDirectories = new ArrayList<>(); + // Is this a hoodie partition + private boolean isHoodiePartition = false; + + public DirectoryInfo(String relativePath, List pathInfos, String maxInstantTime, Set pendingDataInstants) { + this(relativePath, pathInfos, maxInstantTime, pendingDataInstants, true); + } + + /** + * When files are directly fetched from Metadata table we do not need to validate HoodiePartitions. + */ + public DirectoryInfo(String relativePath, List pathInfos, String maxInstantTime, Set pendingDataInstants, + boolean validateHoodiePartitions) { + this.relativePath = relativePath; + + // Pre-allocate with the maximum length possible + filenameToSizeMap = new HashMap<>(pathInfos.size()); + + // Presence of partition meta file implies this is a HUDI partition + // if input files are directly fetched from MDT, it may not contain the HoodiePartitionMetadata file. So, we can ignore the validation for isHoodiePartition. + isHoodiePartition = !validateHoodiePartitions || pathInfos.stream().anyMatch(status -> status.getPath().getName().startsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX)); + for (StoragePathInfo pathInfo : pathInfos) { + // Do not attempt to search for more subdirectories inside directories that are partitions + if (!isHoodiePartition && pathInfo.isDirectory()) { + // Ignore .hoodie directory as there cannot be any partitions inside it + if (!pathInfo.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)) { + this.subDirectories.add(pathInfo.getPath()); + } + } else if (isHoodiePartition && FSUtils.isDataFile(pathInfo.getPath())) { + // Regular HUDI data file (base file or log file) + String dataFileCommitTime = FSUtils.getCommitTime(pathInfo.getPath().getName()); + // Limit the file listings to files which were created by successful commits before the maxInstant time. + if (!pendingDataInstants.contains(dataFileCommitTime) && compareTimestamps(dataFileCommitTime, LESSER_THAN_OR_EQUALS, maxInstantTime)) { + filenameToSizeMap.put(pathInfo.getPath().getName(), pathInfo.getLength()); + } + } + } + } + + public static Map> getPartitionToFiles(List partitionInfoList) { + return partitionInfoList.stream() + .map(p -> { + String partitionName = HoodieTableMetadataUtil.getPartitionIdentifierForFilesPartition(p.getRelativePath()); + return Pair.of(partitionName, p.getFilenameToSizeMap()); + }) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileAndPartitionFlag.java b/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileAndPartitionFlag.java new file mode 100644 index 0000000000000..f7ff3b6d4bd18 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileAndPartitionFlag.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.model; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.experimental.Accessors; + +/** + * Data class representing a file and partition with a flag. + */ +@Getter +@AllArgsConstructor +@Accessors(fluent = true) +public class FileAndPartitionFlag { + private final String partitionPath; + private final String fileName; + private final boolean flag; + + public static FileAndPartitionFlag of(String partition, String file, boolean flag) { + return new FileAndPartitionFlag(partition, file, flag); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileSliceAndPartition.java b/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileSliceAndPartition.java new file mode 100644 index 0000000000000..568cca2a2f73e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileSliceAndPartition.java @@ -0,0 +1,46 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.hudi.metadata.model; + +import org.apache.hudi.common.model.FileSlice; + +import lombok.AllArgsConstructor; +import lombok.ToString; +import lombok.Getter; +import lombok.Setter; +import lombok.experimental.Accessors; + +/** + * Holder for a {@link FileSlice} and its partition path. + */ +@AllArgsConstructor +@Getter +@Setter +@ToString +@Accessors(fluent = true) +public class FileSliceAndPartition { + private FileSlice fileSlice; + private String partitionPath; + + public static FileSliceAndPartition of(String partitionPath, FileSlice fileSlice) { + return new FileSliceAndPartition(fileSlice, partitionPath); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileToIndex.java b/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileToIndex.java new file mode 100644 index 0000000000000..59b12f9109ce5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/model/FileToIndex.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata.model; + +import lombok.Getter; +import lombok.experimental.Accessors; + +/** + * Immutable descriptor for a file that should be scanned during index bootstrap. + */ +@Getter +@Accessors(fluent = true) +public class FileToIndex { + private final String partition; + private final String path; + private final long size; + + private FileToIndex(String partition, String path, long size) { + this.partition = partition; + this.path = path; + this.size = size; + } + + public static FileToIndex of(String partition, String path, long size) { + return new FileToIndex(partition, path, size); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/metadata/TestMetadataPartitionType.java b/hudi-common/src/test/java/org/apache/hudi/metadata/TestMetadataPartitionType.java index 2ccddf81add3c..23424b4c9f533 100644 --- a/hudi-common/src/test/java/org/apache/hudi/metadata/TestMetadataPartitionType.java +++ b/hudi-common/src/test/java/org/apache/hudi/metadata/TestMetadataPartitionType.java @@ -40,6 +40,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -62,30 +64,34 @@ private static Stream partitionTypeAndPartitionPairs() { return Stream.of(Arguments.of(type, true)); } return Stream.of(Arguments.of(type, true), Arguments.of(type, false)); - }); + }).flatMap(arguments -> + Stream.of(Arguments.of(arguments.get()[0], arguments.get()[1], HoodieTableVersion.current()), + Arguments.of(arguments.get()[0], arguments.get()[1], HoodieTableVersion.SIX))); } @ParameterizedTest @MethodSource("partitionTypeAndPartitionPairs") - public void testPartitionEnabledByConfigOnly(MetadataPartitionType partitionType, boolean isTablePartitioned) { + public void testPartitionEnabledByConfigOnly(MetadataPartitionType partitionType, boolean isTablePartitioned, HoodieTableVersion tableVersion) { HoodieTableMetaClient metaClient = Mockito.mock(HoodieTableMetaClient.class); HoodieTableConfig tableConfig = Mockito.mock(HoodieTableConfig.class); // Simulate the configuration enabling given partition type, but the meta client not having it available (yet to initialize the partition) Mockito.when(tableConfig.isTablePartitioned()).thenReturn(isTablePartitioned); Mockito.when(metaClient.getTableConfig()).thenReturn(tableConfig); + Mockito.when(tableConfig.getTableVersion()).thenReturn(tableVersion); Mockito.when(tableConfig.isMetadataPartitionAvailable(partitionType)).thenReturn(false); Mockito.when(metaClient.getIndexMetadata()).thenReturn(Option.empty()); HoodieMetadataConfig.Builder metadataConfigBuilder = HoodieMetadataConfig.newBuilder(); int expectedEnabledPartitions; + boolean tableVersionEightOrAbove = tableVersion.greaterThanOrEquals(HoodieTableVersion.EIGHT); switch (partitionType) { case EXPRESSION_INDEX: metadataConfigBuilder.enable(true).withExpressionIndexEnabled(true); - expectedEnabledPartitions = 3; + expectedEnabledPartitions = tableVersionEightOrAbove ? 3 : 2; break; case SECONDARY_INDEX: metadataConfigBuilder.enable(true).withEnableGlobalRecordLevelIndex(true).withSecondaryIndexEnabled(true).withSecondaryIndexForColumn("col1"); - expectedEnabledPartitions = 4; + expectedEnabledPartitions = tableVersionEightOrAbove ? 4 : 3; break; case BLOOM_FILTERS: metadataConfigBuilder.enable(true).withMetadataIndexBloomFilter(true); @@ -100,15 +106,15 @@ public void testPartitionEnabledByConfigOnly(MetadataPartitionType partitionType expectedEnabledPartitions = 2; // by default, FILES, COLUMN_STATS are enabled break; } - if (isTablePartitioned) { + if (isTablePartitioned && tableVersionEightOrAbove) { expectedEnabledPartitions++; // PARTITION_STATS is enabled by default if table is partitioned } List enabledPartitions = MetadataPartitionType.getEnabledPartitions(metadataConfigBuilder.build(), metaClient); - // Verify partition type is enabled due to config assertEquals(expectedEnabledPartitions, enabledPartitions.size()); - assertTrue(enabledPartitions.contains(partitionType) || MetadataPartitionType.ALL_PARTITIONS.equals(partitionType)); + Set validPartitions = Arrays.stream(MetadataPartitionType.getValidValues(tableVersion)).collect(Collectors.toSet()); + assertTrue(!validPartitions.contains(partitionType) || enabledPartitions.contains(partitionType) || MetadataPartitionType.ALL_PARTITIONS.equals(partitionType)); } @Test @@ -119,6 +125,7 @@ public void testPartitionAvailableByMetaClientOnly() { // Simulate the meta client having RECORD_INDEX available but config not enabling it Mockito.when(metaClient.getTableConfig()).thenReturn(tableConfig); Mockito.when(tableConfig.isTablePartitioned()).thenReturn(true); + Mockito.when(tableConfig.getTableVersion()).thenReturn(HoodieTableVersion.current()); Mockito.when(tableConfig.isMetadataPartitionAvailable(MetadataPartitionType.FILES)).thenReturn(true); Mockito.when(metaClient.getIndexMetadata()).thenReturn(Option.empty()); Mockito.when(metaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.RECORD_INDEX)).thenReturn(true); @@ -159,6 +166,7 @@ public void testExpressionIndexPartitionEnabled() { // Simulate the meta client having EXPRESSION_INDEX available Mockito.when(metaClient.getTableConfig()).thenReturn(tableConfig); Mockito.when(tableConfig.isTablePartitioned()).thenReturn(true); + Mockito.when(tableConfig.getTableVersion()).thenReturn(HoodieTableVersion.current()); Mockito.when(tableConfig.isMetadataPartitionAvailable(MetadataPartitionType.FILES)).thenReturn(true); HoodieIndexDefinition expressionIndexDefinition = createIndexDefinition(MetadataPartitionType.EXPRESSION_INDEX, "dummy", "column_stats", "lower", Collections.singletonList("name"), null); HoodieIndexMetadata expressionIndexMetadata = new HoodieIndexMetadata(Collections.singletonMap("expr_index_dummy", expressionIndexDefinition)); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java index 74589886a35a7..cce4d04e91f70 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.metadata.model.FileSliceAndPartition; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieLogFile; @@ -130,7 +131,7 @@ public void testConvertFilesToPartitionStatsRecords() throws Exception { hoodieTestTable = hoodieTestTable.addCommit(instant1); String instant2 = "20230918121110000"; hoodieTestTable = hoodieTestTable.addCommit(instant2); - List> partitionFileSlicePairs = new ArrayList<>(); + List partitionFileSlicePairs = new ArrayList<>(); // Generate 10 inserts for each partition and populate partitionBaseFilePairs and recordKeys. DATE_PARTITIONS.forEach(p -> { try { @@ -156,8 +157,8 @@ public void testConvertFilesToPartitionStatsRecords() throws Exception { engineContext); HoodieBaseFile baseFile2 = new HoodieBaseFile(hoodieTestTable.getBaseFilePath(p, fileId2).toString()); fileSlice2.setBaseFile(baseFile2); - partitionFileSlicePairs.add(Pair.of(p, fileSlice1)); - partitionFileSlicePairs.add(Pair.of(p, fileSlice2)); + partitionFileSlicePairs.add(FileSliceAndPartition.of(p, fileSlice1)); + partitionFileSlicePairs.add(FileSliceAndPartition.of(p, fileSlice2)); } catch (Exception e) { throw new RuntimeException(e); } @@ -239,7 +240,7 @@ public void testGetLogFileColumnRangeMetadata() throws Exception { hoodieTestTable = hoodieTestTable.addCommit(instant1, Option.of(commitMetadata)); String instant2 = "20230918121110000"; hoodieTestTable = hoodieTestTable.addCommit(instant2); - List> partitionFileSlicePairs = new ArrayList<>(); + List partitionFileSlicePairs = new ArrayList<>(); List columnsToIndex = Arrays.asList("rider", "driver"); // Generate 10 inserts for each partition and populate partitionBaseFilePairs and recordKeys. DATE_PARTITIONS.forEach(p -> { @@ -260,8 +261,8 @@ public void testGetLogFileColumnRangeMetadata() throws Exception { writeLogFiles(new StoragePath(metaClient.getBasePath(), p), HOODIE_SCHEMA, HoodieTestDataGenerator.HOODIE_SCHEMA_WITH_METADATA_FIELDS, dataGen.generateInsertsForPartition(instant2, 10, p), 1, metaClient.getStorage(), new Properties(), fileId1, instant2); fileSlice2.addLogFile(new HoodieLogFile(storagePath2.toUri().toString())); - partitionFileSlicePairs.add(Pair.of(p, fileSlice1)); - partitionFileSlicePairs.add(Pair.of(p, fileSlice2)); + partitionFileSlicePairs.add(FileSliceAndPartition.of(p, fileSlice1)); + partitionFileSlicePairs.add(FileSliceAndPartition.of(p, fileSlice2)); // NOTE: we need to set table config as we are not using write client explicitly and these configs are needed for log record reader metaClient.getTableConfig().setValue(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false"); metaClient.getTableConfig().setValue(HoodieTableConfig.RECORDKEY_FIELDS.key(), "_row_key"); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java index b35da0450cfdc..1b7d18668f666 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java @@ -30,7 +30,7 @@ import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; -import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.metadata.model.FileSliceAndPartition; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieIndexDefinition; @@ -49,7 +49,6 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ClosableIterator; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -345,10 +344,10 @@ public void testSecondaryIndexRecordGenerationForMOR() throws IOException { HoodieTableMetadata metadata = metaClient.getTableFormat().getMetadataFactory().create(engineContext, storage, metadataConfig, metaClient.getBasePath().toString()); HoodieTableFileSystemView metadataView = new HoodieTableFileSystemView(metadata, metaClient, metaClient.getActiveTimeline()); metadataView.loadAllPartitions(); - List> partitionFileSlicePairs = new ArrayList<>(); + List partitionFileSlicePairs = new ArrayList<>(); HoodieTableFileSystemView finalMetadataView = metadataView; Arrays.asList(dataGen.getPartitionPaths()).forEach(partition -> finalMetadataView.getLatestMergedFileSlicesBeforeOrOn(partition, firstCommitTime) - .forEach(fs -> partitionFileSlicePairs.add(Pair.of(partition, fs)))); + .forEach(fs -> partitionFileSlicePairs.add(FileSliceAndPartition.of(partition, fs)))); List secondaryIndexRecords = readSecondaryKeysFromFileSlices( engineContext, partitionFileSlicePairs, metadataConfig.getSecondaryIndexParallelism(), this.getClass().getSimpleName(), metaClient, indexDefinition, writeConfig.getProps()).collectAsList(); assertListEquality(expectedSecondaryIndexKeys, secondaryIndexRecords.stream().map(HoodieRecord::getRecordKey).collect(Collectors.toList())); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java index 30aaab80264f3..761699f9a9b01 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java @@ -874,7 +874,7 @@ public void testMetadataTableDeletePartition(HoodieTableType tableType) throws E assertEquals("0000003", completedReplaceInstant.get().requestedTime()); final Map metadataEnabledPartitionTypes = new HashMap<>(); - metadataWriter.getEnabledPartitionTypes().forEach(e -> metadataEnabledPartitionTypes.put(e.getPartitionPath(), e)); + metadataWriter.getEnabledIndexerMap().keySet().forEach(e -> metadataEnabledPartitionTypes.put(e.getPartitionPath(), e)); HoodieTableFileSystemView fsView = HoodieTableFileSystemView.fileListingBasedFileSystemView(engineContext, metadataMetaClient, metadataMetaClient.getActiveTimeline()); metadataTablePartitions.forEach(partition -> { List latestSlices = fsView.getLatestFileSlices(partition).collect(Collectors.toList()); @@ -3867,12 +3867,12 @@ private void validateMetadata(SparkRDDWriteClient testClient, Option ign List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, metadataMetaClient, false); // Secondary index is enabled by default but no MDT partition corresponding to it is available final boolean isPartitionStatsEnabled; - if (!metadataWriter.getEnabledPartitionTypes().contains(COLUMN_STATS)) { + if (!metadataWriter.getEnabledIndexerMap().containsKey(COLUMN_STATS)) { isPartitionStatsEnabled = false; } else { isPartitionStatsEnabled = true; } - long enabledMDTPartitionsSize = metadataWriter.getEnabledPartitionTypes().stream() + long enabledMDTPartitionsSize = metadataWriter.getEnabledIndexerMap().keySet().stream() .filter(partition -> !partition.equals(SECONDARY_INDEX)) // Filter out partition stats if column stats is disabled since it does not get initialized in such a case .filter(partition -> isPartitionStatsEnabled || !partition.equals(PARTITION_STATS)) @@ -3880,7 +3880,7 @@ private void validateMetadata(SparkRDDWriteClient testClient, Option ign assertEquals(enabledMDTPartitionsSize, metadataTablePartitions.size()); final Map metadataEnabledPartitionTypes = new HashMap<>(); - metadataWriter.getEnabledPartitionTypes().forEach(e -> metadataEnabledPartitionTypes.put(e.getPartitionPath(), e)); + metadataWriter.getEnabledIndexerMap().keySet().forEach(e -> metadataEnabledPartitionTypes.put(e.getPartitionPath(), e)); // Metadata table should automatically compact and clean // versions are +1 as autoclean / compaction happens end of commits @@ -4045,7 +4045,8 @@ public static void validateMetadata(HoodieWriteConfig config, Option ign // check if the last instant is restore, then the metadata table should have only the partitions that are not deleted metaClient.reloadActiveTimeline().getReverseOrderedInstants().findFirst().ifPresent(instant -> { if (instant.getAction().equals(HoodieActiveTimeline.RESTORE_ACTION)) { - metadataWriter.getEnabledPartitionTypes().stream().filter(partitionType -> !MetadataPartitionType.shouldDeletePartitionOnRestore(partitionType.getPartitionPath())) + metadataWriter.getEnabledIndexerMap().keySet().stream().filter( + partitionType -> !MetadataPartitionType.shouldDeletePartitionOnRestore(partitionType.getPartitionPath())) .forEach(partitionType -> assertTrue(metadataTablePartitions.contains(partitionType.getPartitionPath()))); } }); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGlobalRecordLevelIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGlobalRecordLevelIndex.scala index e160f80dfb419..cc00fd1e4c7e9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGlobalRecordLevelIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGlobalRecordLevelIndex.scala @@ -595,7 +595,7 @@ class TestGlobalRecordLevelIndex extends RecordLevelIndexTestBase { doWriteAndValidateDataAndRecordIndex(hudiOpts, operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Append) - assertTrue(metadataWriter(getWriteConfig(hudiOpts)).getEnabledPartitionTypes.containsAll(metadataPartitions.asJava)) + assertTrue(metadataWriter(getWriteConfig(hudiOpts)).getEnabledIndexerMap.keySet().containsAll(metadataPartitions.asJava)) } @ParameterizedTest diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGlobalRecordLevelIndexTableVersionSix.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGlobalRecordLevelIndexTableVersionSix.scala index 1b12bd99645b2..c60bb8b9f3a73 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGlobalRecordLevelIndexTableVersionSix.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGlobalRecordLevelIndexTableVersionSix.scala @@ -18,10 +18,21 @@ package org.apache.hudi.functional -import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.DataSourceWriteOptions +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.metadata.MetadataPartitionType +import org.apache.spark.sql.SaveMode +import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Tag +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.EnumSource + +import java.util.Collections @Tag("functional-b") class TestGlobalRecordLevelIndexTableVersionSix extends TestGlobalRecordLevelIndex { @@ -29,4 +40,27 @@ class TestGlobalRecordLevelIndexTableVersionSix extends TestGlobalRecordLevelInd HoodieTableConfig.VERSION.key() -> "6", HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> "6" ) + + @ParameterizedTest + @EnumSource(classOf[HoodieTableType]) + override def testRLIUpsertAndDropIndex(tableType: HoodieTableType): Unit = { + val hudiOpts = commonOpts ++ Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name(), + HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true") + doWriteAndValidateDataAndRecordIndex(hudiOpts, + operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, + saveMode = SaveMode.Overwrite) + + val writeConfig = getWriteConfig(hudiOpts) + writeConfig.setSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + getHoodieWriteClient(writeConfig).dropIndex(Collections.singletonList(MetadataPartitionType.RECORD_INDEX.getPartitionPath)) + assertEquals(0, getFileGroupCountForRecordIndex(writeConfig)) + metaClient = HoodieTableMetaClient.reload(metaClient) + assertEquals(0, metaClient.getTableConfig.getMetadataPartitionsInflight.size()) + // only files, col stats, partition stats partition should be present. + assertEquals(2, metaClient.getTableConfig.getMetadataPartitions.size()) + + doWriteAndValidateDataAndRecordIndex(hudiOpts, + operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, + saveMode = SaveMode.Append) + } }