diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index d8a928584ea09..8a1d046779a61 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1437,6 +1437,14 @@ public boolean useBloomIndexBucketizedChecking() { return getBoolean(HoodieIndexConfig.BLOOM_INDEX_BUCKETIZED_CHECKING); } + public boolean isMetadataBloomFilterIndexEnabled() { + return isMetadataTableEnabled() && getMetadataConfig().isBloomFilterIndexEnabled(); + } + + public boolean isMetadataIndexColumnStatsForAllColumnsEnabled() { + return isMetadataTableEnabled() && getMetadataConfig().isMetadataColumnStatsIndexForAllColumnsEnabled(); + } + public int getBloomIndexKeysPerBucket() { return getInt(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java index e5426ca1161f9..c6a9a14f561a2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java @@ -18,17 +18,30 @@ package org.apache.hudi.index; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Set; +import java.util.TreeSet; import static java.util.stream.Collectors.toList; @@ -37,6 +50,8 @@ */ public class HoodieIndexUtils { + private static final Logger LOG = LogManager.getLogger(HoodieIndexUtils.class); + /** * Fetches Pair of partition path and {@link HoodieBaseFile}s for interested partitions. * @@ -101,4 +116,34 @@ record = new HoodieRecord<>(inputRecord); } return record; } + + /** + * Given a list of row keys and one file, return only row keys existing in that file. + * + * @param filePath - File to filter keys from + * @param candidateRecordKeys - Candidate keys to filter + * @return List of candidate keys that are available in the file + */ + public static List filterKeysFromFile(Path filePath, List candidateRecordKeys, + Configuration configuration) throws HoodieIndexException { + ValidationUtils.checkArgument(FSUtils.isBaseFile(filePath)); + List foundRecordKeys = new ArrayList<>(); + try { + // Load all rowKeys from the file, to double-confirm + if (!candidateRecordKeys.isEmpty()) { + HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieFileReader fileReader = HoodieFileReaderFactory.getFileReader(configuration, filePath); + Set fileRowKeys = fileReader.filterRowKeys(new TreeSet<>(candidateRecordKeys)); + foundRecordKeys.addAll(fileRowKeys); + LOG.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath, + timer.endTimer(), candidateRecordKeys.size(), foundRecordKeys.size())); + if (LOG.isDebugEnabled()) { + LOG.debug("Keys matching for file " + filePath + " => " + foundRecordKeys); + } + } + } catch (Exception e) { + throw new HoodieIndexException("Error checking candidate keys against file.", e); + } + return foundRecordKeys; + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndexCheckFunction.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndexCheckFunction.java index 441a212c59f40..80031f4e8f025 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndexCheckFunction.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndexCheckFunction.java @@ -25,7 +25,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.io.HoodieKeyLookupHandle; -import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult; +import org.apache.hudi.io.HoodieKeyLookupResult; import org.apache.hudi.table.HoodieTable; import java.util.function.Function; @@ -37,7 +37,7 @@ * Function performing actual checking of list containing (fileId, hoodieKeys) against the actual files. */ public class HoodieBaseBloomIndexCheckFunction - implements Function>, Iterator>> { + implements Function>, Iterator>> { private final HoodieTable hoodieTable; @@ -49,11 +49,11 @@ public HoodieBaseBloomIndexCheckFunction(HoodieTable hoodieTable, HoodieWriteCon } @Override - public Iterator> apply(Iterator> filePartitionRecordKeyTripletItr) { + public Iterator> apply(Iterator> filePartitionRecordKeyTripletItr) { return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr); } - class LazyKeyCheckIterator extends LazyIterableIterator, List> { + class LazyKeyCheckIterator extends LazyIterableIterator, List> { private HoodieKeyLookupHandle keyLookupHandle; @@ -66,8 +66,8 @@ protected void start() { } @Override - protected List computeNext() { - List ret = new ArrayList<>(); + protected List computeNext() { + List ret = new ArrayList<>(); try { // process one file in each go. while (inputItr.hasNext()) { @@ -83,7 +83,7 @@ protected List computeNext() { } // if continue on current file - if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) { + if (keyLookupHandle.getPartitionPathFileIDPair().equals(partitionPathFilePair)) { keyLookupHandle.addKey(recordKey); } else { // do the actual checking of file & break out diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java index a223d695cc03a..5b7c7af91cd39 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -19,11 +19,13 @@ package org.apache.hudi.index.bloom; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -33,6 +35,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.MetadataNotFoundException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndexUtils; @@ -46,6 +49,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; import static java.util.stream.Collectors.groupingBy; import static java.util.stream.Collectors.mapping; @@ -111,13 +115,19 @@ record -> new ImmutablePair<>(record.getPartitionPath(), record.getRecordKey())) private HoodiePairData lookupIndex( HoodiePairData partitionRecordKeyPairs, final HoodieEngineContext context, final HoodieTable hoodieTable) { - // Obtain records per partition, in the incoming records + // Step 1: Obtain records per partition, in the incoming records Map recordsPerPartition = partitionRecordKeyPairs.countByKey(); List affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); // Step 2: Load all involved files as pairs - List> fileInfoList = - loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable); + List> fileInfoList; + if (config.getBloomIndexPruneByRanges()) { + fileInfoList = (config.getMetadataConfig().isColumnStatsIndexEnabled() + ? loadColumnRangesFromMetaIndex(affectedPartitionPathList, context, hoodieTable) + : loadColumnRangesFromFiles(affectedPartitionPathList, context, hoodieTable)); + } else { + fileInfoList = getFileInfoForLatestBaseFiles(affectedPartitionPathList, context, hoodieTable); + } final Map> partitionToFileInfo = fileInfoList.stream().collect(groupingBy(Pair::getLeft, mapping(Pair::getRight, toList()))); @@ -133,30 +143,84 @@ private HoodiePairData lookupIndex( /** * Load all involved files as pair List. */ - List> loadInvolvedFiles( + List> loadColumnRangesFromFiles( List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { // Obtain the latest data files from all the partitions. List> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream() .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId())) .collect(toList()); - if (config.getBloomIndexPruneByRanges()) { - // also obtain file ranges, if range pruning is enabled - context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)"); - return context.map(partitionPathFileIDList, pf -> { - try { - HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); - String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys(); - return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1])); - } catch (MetadataNotFoundException me) { - LOG.warn("Unable to find range metadata in file :" + pf); - return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue())); + context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)"); + return context.map(partitionPathFileIDList, pf -> { + try { + HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); + String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys(); + return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1])); + } catch (MetadataNotFoundException me) { + LOG.warn("Unable to find range metadata in file :" + pf); + return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue())); + } + }, Math.max(partitionPathFileIDList.size(), 1)); + } + + /** + * Get BloomIndexFileInfo for all the latest base files for the requested partitions. + * + * @param partitions - List of partitions to get the base files for + * @param context - Engine context + * @param hoodieTable - Hoodie Table + * @return List of partition and file column range info pairs + */ + private List> getFileInfoForLatestBaseFiles( + List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { + List> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, + hoodieTable).stream() + .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId())) + .collect(toList()); + return partitionPathFileIDList.stream() + .map(pf -> Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList()); + } + + /** + * Load the column stats index as BloomIndexFileInfo for all the involved files in the partition. + * + * @param partitions - List of partitions for which column stats need to be loaded + * @param context - Engine context + * @param hoodieTable - Hoodie table + * @return List of partition and file column range info pairs + */ + protected List> loadColumnRangesFromMetaIndex( + List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { + // also obtain file ranges, if range pruning is enabled + context.setJobStatus(this.getClass().getName(), "Load meta index key ranges for file slices"); + + final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp(); + return context.flatMap(partitions, partitionName -> { + // Partition and file name pairs + List> partitionFileNameList = HoodieIndexUtils.getLatestBaseFilesForPartition(partitionName, + hoodieTable).stream().map(baseFile -> Pair.of(partitionName, baseFile.getFileName())) + .sorted() + .collect(toList()); + if (partitionFileNameList.isEmpty()) { + return Stream.empty(); + } + try { + Map, HoodieMetadataColumnStats> fileToColumnStatsMap = hoodieTable + .getMetadataTable().getColumnStats(partitionFileNameList, keyField); + List> result = new ArrayList<>(); + for (Map.Entry, HoodieMetadataColumnStats> entry : fileToColumnStatsMap.entrySet()) { + result.add(Pair.of(entry.getKey().getLeft(), + new BloomIndexFileInfo( + FSUtils.getFileId(entry.getKey().getRight()), + entry.getValue().getMinValue(), + entry.getValue().getMaxValue() + ))); } - }, Math.max(partitionPathFileIDList.size(), 1)); - } else { - return partitionPathFileIDList.stream() - .map(pf -> Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList()); - } + return result.stream(); + } catch (MetadataNotFoundException me) { + throw new HoodieMetadataException("Unable to find column range metadata for partition:" + partitionName, me); + } + }, Math.max(partitions.size(), 1)); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java index 39fa72a329fe3..f063713884ea8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java @@ -55,11 +55,11 @@ public HoodieGlobalBloomIndex(HoodieWriteConfig config, BaseHoodieBloomIndexHelp * Load all involved files as pairs from all partitions in the table. */ @Override - List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, - final HoodieTable hoodieTable) { + List> loadColumnRangesFromFiles(List partitions, final HoodieEngineContext context, + final HoodieTable hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); List allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath()); - return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable); + return super.loadColumnRangesFromFiles(allPartitionPaths, context, hoodieTable); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java index 74191df523659..dde93cf377c77 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java @@ -28,7 +28,7 @@ import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.io.HoodieKeyLookupResult; import org.apache.hudi.table.HoodieTable; import java.util.ArrayList; @@ -63,9 +63,8 @@ public HoodiePairData findMatchingFilesForRecor HoodieList.getList(fileComparisonPairs).stream() .sorted(Comparator.comparing(ImmutablePair::getLeft)).collect(toList()); - List keyLookupResults = new ArrayList<>(); - - Iterator> iterator = new HoodieBaseBloomIndexCheckFunction( + List keyLookupResults = new ArrayList<>(); + Iterator> iterator = new HoodieBaseBloomIndexCheckFunction( hoodieTable, config).apply(fileComparisonPairList.iterator()); while (iterator.hasNext()) { keyLookupResults.addAll(iterator.next()); @@ -77,7 +76,7 @@ public HoodiePairData findMatchingFilesForRecor lookupResult.getMatchingRecordKeys().stream() .map(recordKey -> new ImmutablePair<>(lookupResult, recordKey)).iterator() ).mapToPair(pair -> { - HoodieKeyLookupHandle.KeyLookupResult lookupResult = pair.getLeft(); + HoodieKeyLookupResult lookupResult = pair.getLeft(); String recordKey = pair.getRight(); return new ImmutablePair<>( new HoodieKey(recordKey, lookupResult.getPartitionPath()), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java index c6f9dddef30db..1ad28d14b3a8d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java @@ -19,6 +19,8 @@ package org.apache.hudi.io; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -31,8 +33,8 @@ public abstract class HoodieIOHandle { protected final FileSystem fs; protected final HoodieTable hoodieTable; - HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable) { - this.instantTime = instantTime; + HoodieIOHandle(HoodieWriteConfig config, Option instantTime, HoodieTable hoodieTable) { + this.instantTime = instantTime.orElse(StringUtils.EMPTY_STRING); this.config = config; this.hoodieTable = hoodieTable; this.fs = getFileSystem(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java index c33931f503a36..6796557714221 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java @@ -47,7 +47,7 @@ public class HoodieKeyLocationFetchHandle hoodieTable, Pair partitionPathBaseFilePair, Option keyGeneratorOpt) { - super(config, null, hoodieTable, Pair.of(partitionPathBaseFilePair.getLeft(), partitionPathBaseFilePair.getRight().getFileId())); + super(config, hoodieTable, Pair.of(partitionPathBaseFilePair.getLeft(), partitionPathBaseFilePair.getRight().getFileId())); this.partitionPathBaseFilePair = partitionPathBaseFilePair; this.keyGeneratorOpt = keyGeneratorOpt; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java index ad84e3e974af8..12d075e0cb532 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java @@ -19,25 +19,30 @@ package org.apache.hudi.io; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.table.HoodieTable; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; -import java.util.Set; /** * Takes a bunch of keys and returns ones that are present in the file group. @@ -46,52 +51,58 @@ public class HoodieKeyLookupHandle exten private static final Logger LOG = LogManager.getLogger(HoodieKeyLookupHandle.class); - private final HoodieTableType tableType; - private final BloomFilter bloomFilter; - private final List candidateRecordKeys; - + private final boolean useMetadataTableIndex; + private Option fileName = Option.empty(); private long totalKeysChecked; public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable hoodieTable, - Pair partitionPathFilePair) { - super(config, null, hoodieTable, partitionPathFilePair); - this.tableType = hoodieTable.getMetaClient().getTableType(); + Pair partitionPathFileIDPair) { + this(config, hoodieTable, partitionPathFileIDPair, Option.empty(), false); + } + + public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable hoodieTable, + Pair partitionPathFileIDPair, Option fileName, + boolean useMetadataTableIndex) { + super(config, hoodieTable, partitionPathFileIDPair); this.candidateRecordKeys = new ArrayList<>(); this.totalKeysChecked = 0; - HoodieTimer timer = new HoodieTimer().startTimer(); - - try { - this.bloomFilter = createNewFileReader().readBloomFilter(); - } catch (IOException e) { - throw new HoodieIndexException(String.format("Error reading bloom filter from %s: %s", partitionPathFilePair, e)); + if (fileName.isPresent()) { + ValidationUtils.checkArgument(FSUtils.getFileId(fileName.get()).equals(getFileId()), + "File name '" + fileName.get() + "' doesn't match this lookup handle fileid '" + getFileId() + "'"); + this.fileName = fileName; } - LOG.info(String.format("Read bloom filter from %s in %d ms", partitionPathFilePair, timer.endTimer())); + this.useMetadataTableIndex = useMetadataTableIndex; + this.bloomFilter = getBloomFilter(); } - /** - * Given a list of row keys and one file, return only row keys existing in that file. - */ - public List checkCandidatesAgainstFile(Configuration configuration, List candidateRecordKeys, - Path filePath) throws HoodieIndexException { - List foundRecordKeys = new ArrayList<>(); + private BloomFilter getBloomFilter() { + BloomFilter bloomFilter = null; + HoodieTimer timer = new HoodieTimer().startTimer(); try { - // Load all rowKeys from the file, to double-confirm - if (!candidateRecordKeys.isEmpty()) { - HoodieTimer timer = new HoodieTimer().startTimer(); - Set fileRowKeys = createNewFileReader().filterRowKeys(new HashSet<>(candidateRecordKeys)); - foundRecordKeys.addAll(fileRowKeys); - LOG.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath, - timer.endTimer(), candidateRecordKeys.size(), foundRecordKeys.size())); - if (LOG.isDebugEnabled()) { - LOG.debug("Keys matching for file " + filePath + " => " + foundRecordKeys); + if (this.useMetadataTableIndex) { + ValidationUtils.checkArgument(this.fileName.isPresent(), + "File name not available to fetch bloom filter from the metadata table index."); + Option bloomFilterByteBuffer = + hoodieTable.getMetadataTable().getBloomFilter(partitionPathFileIDPair.getLeft(), fileName.get()); + if (!bloomFilterByteBuffer.isPresent()) { + throw new HoodieIndexException("BloomFilter missing for " + partitionPathFileIDPair.getRight()); + } + bloomFilter = + new HoodieDynamicBoundedBloomFilter(StandardCharsets.UTF_8.decode(bloomFilterByteBuffer.get()).toString(), + BloomFilterTypeCode.DYNAMIC_V0); + } else { + try (HoodieFileReader reader = createNewFileReader()) { + bloomFilter = reader.readBloomFilter(); } } - } catch (Exception e) { - throw new HoodieIndexException("Error checking candidate keys against file.", e); + } catch (IOException e) { + throw new HoodieIndexException(String.format("Error reading bloom filter from %s/%s - %s", + getPartitionPathFileIDPair().getLeft(), this.fileName, e)); } - return foundRecordKeys; + LOG.info(String.format("Read bloom filter from %s in %d ms", partitionPathFileIDPair, timer.endTimer())); + return bloomFilter; } /** @@ -101,7 +112,7 @@ public void addKey(String recordKey) { // check record key against bloom filter of current file & add to possible keys if needed if (bloomFilter.mightContain(recordKey)) { if (LOG.isDebugEnabled()) { - LOG.debug("Record key " + recordKey + " matches bloom filter in " + partitionPathFilePair); + LOG.debug("Record key " + recordKey + " matches bloom filter in " + partitionPathFileIDPair); } candidateRecordKeys.add(recordKey); } @@ -111,53 +122,18 @@ public void addKey(String recordKey) { /** * Of all the keys, that were added, return a list of keys that were actually found in the file group. */ - public KeyLookupResult getLookupResult() { + public HoodieKeyLookupResult getLookupResult() { if (LOG.isDebugEnabled()) { - LOG.debug("#The candidate row keys for " + partitionPathFilePair + " => " + candidateRecordKeys); + LOG.debug("#The candidate row keys for " + partitionPathFileIDPair + " => " + candidateRecordKeys); } HoodieBaseFile dataFile = getLatestDataFile(); - List matchingKeys = - checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys, new Path(dataFile.getPath())); + List matchingKeys = HoodieIndexUtils.filterKeysFromFile(new Path(dataFile.getPath()), candidateRecordKeys, + hoodieTable.getHadoopConf()); LOG.info( String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)", totalKeysChecked, candidateRecordKeys.size(), candidateRecordKeys.size() - matchingKeys.size(), matchingKeys.size())); - return new KeyLookupResult(partitionPathFilePair.getRight(), partitionPathFilePair.getLeft(), + return new HoodieKeyLookupResult(partitionPathFileIDPair.getRight(), partitionPathFileIDPair.getLeft(), dataFile.getCommitTime(), matchingKeys); } - - /** - * Encapsulates the result from a key lookup. - */ - public static class KeyLookupResult { - - private final String fileId; - private final String baseInstantTime; - private final List matchingRecordKeys; - private final String partitionPath; - - public KeyLookupResult(String fileId, String partitionPath, String baseInstantTime, - List matchingRecordKeys) { - this.fileId = fileId; - this.partitionPath = partitionPath; - this.baseInstantTime = baseInstantTime; - this.matchingRecordKeys = matchingRecordKeys; - } - - public String getFileId() { - return fileId; - } - - public String getBaseInstantTime() { - return baseInstantTime; - } - - public String getPartitionPath() { - return partitionPath; - } - - public List getMatchingRecordKeys() { - return matchingRecordKeys; - } - } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupResult.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupResult.java new file mode 100644 index 0000000000000..19096a21d8700 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupResult.java @@ -0,0 +1,57 @@ +/* + * 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.io; + +import java.util.List; + +/** + * Encapsulates the result from a key lookup. + */ +public class HoodieKeyLookupResult { + + private final String fileId; + private final String baseInstantTime; + private final List matchingRecordKeys; + private final String partitionPath; + + public HoodieKeyLookupResult(String fileId, String partitionPath, String baseInstantTime, + List matchingRecordKeys) { + this.fileId = fileId; + this.partitionPath = partitionPath; + this.baseInstantTime = baseInstantTime; + this.matchingRecordKeys = matchingRecordKeys; + } + + public String getFileId() { + return fileId; + } + + public String getBaseInstantTime() { + return baseInstantTime; + } + + public String getPartitionPath() { + return partitionPath; + } + + public List getMatchingRecordKeys() { + return matchingRecordKeys; + } +} + diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java index 78fa9be690367..abe4a9befef9b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.table.HoodieTable; import java.io.IOException; @@ -32,10 +33,12 @@ public class HoodieRangeInfoHandle exten public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable hoodieTable, Pair partitionPathFilePair) { - super(config, null, hoodieTable, partitionPathFilePair); + super(config, hoodieTable, partitionPathFilePair); } public String[] getMinMaxKeys() throws IOException { - return createNewFileReader().readMinMaxRecordKeys(); + try (HoodieFileReader reader = createNewFileReader()) { + return reader.readMinMaxRecordKeys(); + } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java index a771c33c40661..fee75b22decd7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java @@ -18,8 +18,11 @@ package org.apache.hudi.io; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; +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.io.storage.HoodieFileReader; @@ -28,20 +31,17 @@ import java.io.IOException; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - /** * Base class for read operations done logically on the file group. */ public abstract class HoodieReadHandle extends HoodieIOHandle { - protected final Pair partitionPathFilePair; + protected final Pair partitionPathFileIDPair; - public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - Pair partitionPathFilePair) { - super(config, instantTime, hoodieTable); - this.partitionPathFilePair = partitionPathFilePair; + public HoodieReadHandle(HoodieWriteConfig config, HoodieTable hoodieTable, + Pair partitionPathFileIDPair) { + super(config, Option.empty(), hoodieTable); + this.partitionPathFileIDPair = partitionPathFileIDPair; } @Override @@ -49,17 +49,17 @@ protected FileSystem getFileSystem() { return hoodieTable.getMetaClient().getFs(); } - public Pair getPartitionPathFilePair() { - return partitionPathFilePair; + public Pair getPartitionPathFileIDPair() { + return partitionPathFileIDPair; } public String getFileId() { - return partitionPathFilePair.getRight(); + return partitionPathFileIDPair.getRight(); } protected HoodieBaseFile getLatestDataFile() { return hoodieTable.getBaseFileOnlyView() - .getLatestBaseFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get(); + .getLatestBaseFile(partitionPathFileIDPair.getLeft(), partitionPathFileIDPair.getRight()).get(); } protected HoodieFileReader createNewFileReader() throws IOException { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 37721611e2c9a..2afedf2f27aa5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -108,7 +108,7 @@ public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String pa protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId, HoodieTable hoodieTable, Option overriddenSchema, TaskContextSupplier taskContextSupplier) { - super(config, instantTime, hoodieTable); + super(config, Option.of(instantTime), hoodieTable); this.partitionPath = partitionPath; this.fileId = fileId; this.tableSchema = overriddenSchema.orElseGet(() -> getSpecifiedTableSchema(config)); 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 49e2c8f74a1ae..eee676822a8aa 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 @@ -31,6 +31,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -39,6 +40,7 @@ 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.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableConfig; @@ -50,6 +52,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -109,6 +112,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta protected boolean enabled; protected SerializableConfiguration hadoopConf; protected final transient HoodieEngineContext engineContext; + // TODO: HUDI-3258 Support secondary key via multiple partitions within a single type + protected final List enabledPartitionTypes; /** * Hudi backed table metadata writer. @@ -128,6 +133,8 @@ protected HoodieBackedTableMetadataWriter(Configu this.dataWriteConfig = writeConfig; this.engineContext = engineContext; this.hadoopConf = new SerializableConfiguration(hadoopConf); + this.metrics = Option.empty(); + this.enabledPartitionTypes = new ArrayList<>(); if (writeConfig.isMetadataTableEnabled()) { this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX; @@ -145,22 +152,67 @@ protected HoodieBackedTableMetadataWriter(Configu ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(), "File listing cannot be used for Metadata Table"); - initRegistry(); this.dataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build(); + enablePartitions(); + initRegistry(); initialize(engineContext, actionMetadata, inflightInstantTimestamp); initTableMetadata(); } else { enabled = false; - this.metrics = Option.empty(); } } public HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, - HoodieEngineContext engineContext) { + HoodieEngineContext engineContext) { this(hadoopConf, writeConfig, engineContext, Option.empty(), Option.empty()); } + /** + * Enable metadata table partitions based on config. + */ + private void enablePartitions() { + final HoodieMetadataConfig metadataConfig = dataWriteConfig.getMetadataConfig(); + boolean isBootstrapCompleted; + Option metaClient = Option.empty(); + try { + isBootstrapCompleted = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); + if (isBootstrapCompleted) { + metaClient = Option.of(HoodieTableMetaClient.builder().setConf(hadoopConf.get()) + .setBasePath(metadataWriteConfig.getBasePath()).build()); + } + } catch (IOException e) { + throw new HoodieException("Failed to enable metadata partitions!", e); + } + + Option fsView = Option.ofNullable( + metaClient.isPresent() ? HoodieTableMetadataUtil.getFileSystemView(metaClient.get()) : null); + enablePartition(MetadataPartitionType.FILES, metadataConfig, metaClient, fsView, isBootstrapCompleted); + if (metadataConfig.isBloomFilterIndexEnabled()) { + enablePartition(MetadataPartitionType.BLOOM_FILTERS, metadataConfig, metaClient, fsView, isBootstrapCompleted); + } + if (metadataConfig.isColumnStatsIndexEnabled()) { + enablePartition(MetadataPartitionType.COLUMN_STATS, metadataConfig, metaClient, fsView, isBootstrapCompleted); + } + } + + /** + * Enable metadata table partition. + * + * @param partitionType - Metadata table partition type + * @param metadataConfig - Table config + * @param metaClient - Meta client for the metadata table + * @param fsView - Metadata table filesystem view to use + * @param isBootstrapCompleted - Is metadata table bootstrap completed + */ + private void enablePartition(final MetadataPartitionType partitionType, final HoodieMetadataConfig metadataConfig, + final Option metaClient, Option fsView, boolean isBootstrapCompleted) { + final int fileGroupCount = HoodieTableMetadataUtil.getPartitionFileGroupCount(partitionType, metaClient, fsView, + metadataConfig, isBootstrapCompleted); + partitionType.setFileGroupCount(fileGroupCount); + this.enabledPartitionTypes.add(partitionType); + } + protected abstract void initRegistry(); /** @@ -257,10 +309,14 @@ public HoodieWriteConfig getWriteConfig() { return metadataWriteConfig; } - public HoodieBackedTableMetadata metadata() { + public HoodieBackedTableMetadata getTableMetadata() { return metadata; } + public List getEnabledPartitionTypes() { + return this.enabledPartitionTypes; + } + /** * Initialize the metadata table if it does not exist. * @@ -460,7 +516,7 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); initTableMetadata(); - initializeFileGroups(dataMetaClient, MetadataPartitionType.FILES, createInstantTime, 1); + initializeEnabledFileGroups(dataMetaClient, createInstantTime); // List all partitions in the basePath of the containing dataset LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); @@ -529,6 +585,20 @@ private List listAllPartitions(HoodieTableMetaClient datasetMetaC return partitionsToBootstrap; } + /** + * Initialize file groups for all the enabled partition types. + * + * @param dataMetaClient - Meta client for the data table + * @param createInstantTime - Metadata table create instant time + * @throws IOException + */ + private void initializeEnabledFileGroups(HoodieTableMetaClient dataMetaClient, String createInstantTime) throws IOException { + for (MetadataPartitionType enabledPartitionType : this.enabledPartitionTypes) { + initializeFileGroups(dataMetaClient, enabledPartitionType, createInstantTime, + enabledPartitionType.getFileGroupCount()); + } + } + /** * Initialize file groups for a partition. For file listing, we just have one file group. * @@ -550,12 +620,12 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata final HoodieDeleteBlock block = new HoodieDeleteBlock(new HoodieKey[0], blockHeader); LOG.info(String.format("Creating %d file groups for partition %s with base fileId %s at instant time %s", - fileGroupCount, metadataPartition.partitionPath(), metadataPartition.getFileIdPrefix(), instantTime)); + fileGroupCount, metadataPartition.getPartitionPath(), metadataPartition.getFileIdPrefix(), instantTime)); for (int i = 0; i < fileGroupCount; ++i) { final String fileGroupFileId = String.format("%s%04d", metadataPartition.getFileIdPrefix(), i); try { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() - .onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), metadataPartition.partitionPath())) + .onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), metadataPartition.getPartitionPath())) .withFileId(fileGroupFileId).overBaseCommit(instantTime) .withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION) .withFileSize(0L) @@ -567,7 +637,7 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata writer.appendBlock(block); writer.close(); } catch (InterruptedException e) { - throw new HoodieException("Failed to created fileGroup " + fileGroupFileId + " for partition " + metadataPartition.partitionPath(), e); + throw new HoodieException("Failed to created fileGroup " + fileGroupFileId + " for partition " + metadataPartition.getPartitionPath(), e); } } } @@ -577,7 +647,7 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata * Updates of different commit metadata uses the same method to convert to HoodieRecords and hence. */ private interface ConvertMetadataFunction { - List convertMetadata(); + Map> convertMetadata(); } /** @@ -589,8 +659,8 @@ private interface ConvertMetadataFunction { */ private void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction, boolean canTriggerTableService) { if (enabled && metadata != null) { - List records = convertMetadataFunction.convertMetadata(); - commit(engineContext.parallelize(records, 1), MetadataPartitionType.FILES.partitionPath(), instantTime, canTriggerTableService); + Map> partitionRecordsMap = convertMetadataFunction.convertMetadata(); + commit(instantTime, partitionRecordsMap, canTriggerTableService); } } @@ -602,7 +672,8 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co */ @Override public void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean isTableServiceAction) { - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime), !isTableServiceAction); + processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes, + commitMetadata, dataMetaClient, dataWriteConfig.isMetadataIndexColumnStatsForAllColumnsEnabled(), instantTime), !isTableServiceAction); } /** @@ -613,8 +684,8 @@ public void update(HoodieCommitMetadata commitMetadata, String instantTime, bool */ @Override public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime), - false); + processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes, + cleanMetadata, dataMetaClient, instantTime), false); } /** @@ -625,8 +696,9 @@ public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { */ @Override public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), - restoreMetadata, instantTime, metadata.getSyncedInstantTime()), false); + processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, + enabledPartitionTypes, metadataMetaClient.getActiveTimeline(), restoreMetadata, dataMetaClient, instantTime, + metadata.getSyncedInstantTime()), false); } /** @@ -650,9 +722,11 @@ public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) } } - List records = HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), rollbackMetadata, instantTime, - metadata.getSyncedInstantTime(), wasSynced); - commit(engineContext.parallelize(records, 1), MetadataPartitionType.FILES.partitionPath(), instantTime, false); + Map> records = + HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes, + metadataMetaClient.getActiveTimeline(), rollbackMetadata, dataMetaClient, instantTime, + metadata.getSyncedInstantTime(), wasSynced); + commit(instantTime, records, false); } } @@ -665,12 +739,47 @@ public void close() throws Exception { /** * Commit the {@code HoodieRecord}s to Metadata Table as a new delta-commit. - * @param records The HoodieData of records to be written. - * @param partitionName The partition to which the records are to be written. - * @param instantTime The timestamp to use for the deltacommit. + * + * @param instantTime - Action instant time for this commit + * @param partitionRecordsMap - Map of partition name to its records to commit * @param canTriggerTableService true if table services can be scheduled and executed. false otherwise. */ - protected abstract void commit(HoodieData records, String partitionName, String instantTime, boolean canTriggerTableService); + protected abstract void commit( + String instantTime, Map> partitionRecordsMap, + boolean canTriggerTableService); + + /** + * Tag each record with the location in the given partition. + * The record is tagged with respective file slice's location based on its record key. + */ + protected HoodieData prepRecords(Map> partitionRecordsMap) { + // The result set + HoodieData allPartitionRecords = engineContext.emptyHoodieData(); + + HoodieTableFileSystemView fsView = HoodieTableMetadataUtil.getFileSystemView(metadataMetaClient); + for (Map.Entry> entry : partitionRecordsMap.entrySet()) { + final String partitionName = entry.getKey().getPartitionPath(); + final int fileGroupCount = entry.getKey().getFileGroupCount(); + HoodieData records = entry.getValue(); + + List fileSlices = + HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, Option.ofNullable(fsView), partitionName); + ValidationUtils.checkArgument(fileSlices.size() == fileGroupCount, + String.format("Invalid number of file groups for partition:%s, found=%d, required=%d", + partitionName, fileSlices.size(), fileGroupCount)); + + HoodieData rddSinglePartitionRecords = records.map(r -> { + FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), + fileGroupCount)); + r.setCurrentLocation(new HoodieRecordLocation(slice.getBaseInstantTime(), slice.getFileId())); + return r; + }); + + allPartitionRecords = allPartitionRecords.union(rddSinglePartitionRecords); + } + return allPartitionRecords; + } /** * Perform a compaction on the Metadata Table. @@ -735,14 +844,19 @@ protected void bootstrapCommit(List partitionInfoList, String cre List partitions = partitionInfoList.stream().map(p -> p.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : p.getRelativePath()).collect(Collectors.toList()); final int totalFiles = partitionInfoList.stream().mapToInt(p -> p.getTotalFiles()).sum(); + final Map> partitionToRecordsMap = new HashMap<>(); // Record which saves the list of all partitions HoodieRecord allPartitionRecord = HoodieMetadataPayload.createPartitionListRecord(partitions); if (partitions.isEmpty()) { - // in case of boostrapping of a fresh table, there won't be any partitions, but we need to make a boostrap commit - commit(engineContext.parallelize(Collections.singletonList(allPartitionRecord), 1), MetadataPartitionType.FILES.partitionPath(), createInstantTime, false); + // in case of bootstrapping of a fresh table, there won't be any partitions, but we need to make a boostrap commit + final HoodieData allPartitionRecordsRDD = engineContext.parallelize( + Collections.singletonList(allPartitionRecord), 1); + partitionToRecordsMap.put(MetadataPartitionType.FILES, allPartitionRecordsRDD); + commit(createInstantTime, partitionToRecordsMap, false); return; } + HoodieData partitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); if (!partitionInfoList.isEmpty()) { HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { @@ -762,7 +876,8 @@ protected void bootstrapCommit(List partitionInfoList, String cre LOG.info("Committing " + partitions.size() + " partitions and " + totalFiles + " files to metadata"); ValidationUtils.checkState(partitionRecords.count() == (partitions.size() + 1)); - commit(partitionRecords, MetadataPartitionType.FILES.partitionPath(), createInstantTime, false); + partitionToRecordsMap.put(MetadataPartitionType.FILES, partitionRecords); + commit(createInstantTime, partitionToRecordsMap, false); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index d25e95148a01c..d6403240b974c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -768,4 +768,7 @@ public Option getMetad return Option.empty(); } + public HoodieTableMetadata getMetadataTable() { + return this.metadata; + } } 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 f4ce686d2a665..275ab4f5e0a33 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 @@ -22,11 +22,10 @@ import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; -import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; @@ -41,7 +40,7 @@ import java.io.IOException; import java.util.Collections; import java.util.List; -import java.util.stream.Collectors; +import java.util.Map; public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter { @@ -101,10 +100,12 @@ protected void initialize(HoodieEngineContext eng } @Override - protected void commit(HoodieData hoodieDataRecords, String partitionName, String instantTime, boolean canTriggerTableService) { + protected void commit(String instantTime, Map> partitionRecordsMap, + boolean canTriggerTableService) { ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); - List records = (List) hoodieDataRecords.get(); - List recordList = prepRecords(records, partitionName, 1); + ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet."); + HoodieData preppedRecords = prepRecords(partitionRecordsMap); + List preppedRecordList = HoodieList.getList(preppedRecords); try (HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(engineContext, metadataWriteConfig)) { if (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime)) { @@ -119,13 +120,14 @@ protected void commit(HoodieData hoodieDataRecords, String partiti // once rollback is complete, compaction will be retried again, which will eventually hit this code block where the respective commit is // already part of completed commit. So, we have to manually remove the completed instant and proceed. // and it is for the same reason we enabled withAllowMultiWriteOnSameInstant for metadata table. - HoodieInstant alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.getTimestamp().equals(instantTime)).lastInstant().get(); + HoodieInstant alreadyCompletedInstant = + metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.getTimestamp().equals(instantTime)).lastInstant().get(); HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant); metadataMetaClient.reloadActiveTimeline(); } - List statuses = records.size() > 0 - ? writeClient.upsertPreppedRecords(recordList, instantTime) + List statuses = preppedRecordList.size() > 0 + ? writeClient.upsertPreppedRecords(preppedRecordList, instantTime) : Collections.emptyList(); statuses.forEach(writeStatus -> { if (writeStatus.hasErrors()) { @@ -147,21 +149,4 @@ protected void commit(HoodieData hoodieDataRecords, String partiti // Update total size of the metadata and count of base/log files metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata)); } - - /** - * Tag each record with the location in the given partition. - * - * The record is tagged with respective file slice's location based on its record key. - */ - private List prepRecords(List records, String partitionName, int numFileGroups) { - List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, partitionName); - ValidationUtils.checkArgument(fileSlices.size() == numFileGroups, String.format("Invalid number of file groups: found=%d, required=%d", fileSlices.size(), numFileGroups)); - - return records.stream().map(r -> { - FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), numFileGroups)); - final String instantTime = slice.isEmpty() ? "I" : "U"; - r.setCurrentLocation(new HoodieRecordLocation(instantTime, slice.getFileId())); - return r; - }).collect(Collectors.toList()); - } -} +} \ No newline at end of file diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java index 7b4e3b675ea05..b3e89e5ba3105 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java @@ -31,7 +31,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieFlinkClientTestHarness; @@ -130,7 +130,7 @@ public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, b new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List partitions = asList("2016/01/21", "2016/04/01", "2015/03/12"); - List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + List> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); // Still 0, as no valid commit assertEquals(0, filesList.size()); @@ -140,7 +140,7 @@ public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, b .withInserts("2015/03/12", "4", record2, record3, record4); metaClient.reloadActiveTimeline(); - filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); assertEquals(4, filesList.size()); if (rangePruning) { @@ -242,9 +242,8 @@ public void testCheckUUIDsAgainstOneFile() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieFlinkTable table = HoodieFlinkTable.create(config, context, metaClient); - HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId)); - List results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids, - new Path(java.nio.file.Paths.get(basePath, partition, filename).toString())); + List results = HoodieIndexUtils.filterKeysFromFile( + new Path(java.nio.file.Paths.get(basePath, partition, filename).toString()), uuids, hadoopConf); assertEquals(results.size(), 2); assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") || results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java index 148203c9b71f1..e19a429ea7234 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java @@ -25,7 +25,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.io.HoodieKeyLookupHandle; -import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult; +import org.apache.hudi.io.HoodieKeyLookupResult; import org.apache.hudi.table.HoodieTable; import org.apache.spark.api.java.function.Function2; @@ -40,7 +40,7 @@ * Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the actual files. */ public class HoodieBloomIndexCheckFunction - implements Function2>, Iterator>> { + implements Function2>, Iterator>> { private final HoodieTable hoodieTable; @@ -52,12 +52,12 @@ public HoodieBloomIndexCheckFunction(HoodieTable hoodieTable, HoodieWriteConfig } @Override - public Iterator> call(Integer partition, - Iterator> filePartitionRecordKeyTripletItr) { + public Iterator> call(Integer partition, + Iterator> filePartitionRecordKeyTripletItr) { return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr); } - class LazyKeyCheckIterator extends LazyIterableIterator, List> { + class LazyKeyCheckIterator extends LazyIterableIterator, List> { private HoodieKeyLookupHandle keyLookupHandle; @@ -70,9 +70,9 @@ protected void start() { } @Override - protected List computeNext() { + protected List computeNext() { - List ret = new ArrayList<>(); + List ret = new ArrayList<>(); try { // process one file in each go. while (inputItr.hasNext()) { @@ -88,7 +88,7 @@ protected List computeNext() { } // if continue on current file - if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) { + if (keyLookupHandle.getPartitionPathFileIDPair().equals(partitionPathFilePair)) { keyLookupHandle.addKey(recordKey); } else { // do the actual checking of file & break out diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomIndexCheckFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomIndexCheckFunction.java new file mode 100644 index 0000000000000..32bca55099eda --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomIndexCheckFunction.java @@ -0,0 +1,161 @@ +/* + * 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.index.bloom; + +import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.io.HoodieKeyLookupResult; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.function.Function2; +import scala.Tuple2; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Spark Function2 implementation for checking bloom filters for the + * requested keys from the metadata table index. The bloom filter + * checking for keys and the actual file verification for the + * candidate keys is done in an iterative fashion. In each iteration, + * bloom filters are requested for a batch of partition files and the + * keys are checked against them. + */ +public class HoodieMetadataBloomIndexCheckFunction implements + Function2>, Iterator>> { + + private static final Logger LOG = LogManager.getLogger(HoodieMetadataBloomIndexCheckFunction.class); + + // Assuming each file bloom filter takes up 512K, sizing the max file count + // per batch so that the total fetched bloom filters would not cross 128 MB. + private static final long BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH = 256; + private final HoodieTable hoodieTable; + + public HoodieMetadataBloomIndexCheckFunction(HoodieTable hoodieTable) { + this.hoodieTable = hoodieTable; + } + + @Override + public Iterator> call(Integer integer, Iterator> tuple2Iterator) throws Exception { + return new BloomIndexLazyKeyCheckIterator(tuple2Iterator); + } + + private class BloomIndexLazyKeyCheckIterator extends LazyIterableIterator, List> { + public BloomIndexLazyKeyCheckIterator(Iterator> tuple2Iterator) { + super(tuple2Iterator); + } + + @Override + protected void start() { + } + + @Override + protected List computeNext() { + // Partition path and file name pair to list of keys + final Map, List> fileToKeysMap = new HashMap<>(); + final Map fileIDBaseFileMap = new HashMap<>(); + final List resultList = new ArrayList<>(); + + while (inputItr.hasNext()) { + Tuple2 entry = inputItr.next(); + final String partitionPath = entry._2.getPartitionPath(); + final String fileId = entry._1; + if (!fileIDBaseFileMap.containsKey(fileId)) { + Option baseFile = hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId); + if (!baseFile.isPresent()) { + throw new HoodieIndexException("Failed to find the base file for partition: " + partitionPath + + ", fileId: " + fileId); + } + fileIDBaseFileMap.put(fileId, baseFile.get()); + } + fileToKeysMap.computeIfAbsent(Pair.of(partitionPath, fileIDBaseFileMap.get(fileId).getFileName()), + k -> new ArrayList<>()).add(entry._2); + if (fileToKeysMap.size() > BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH) { + break; + } + } + if (fileToKeysMap.isEmpty()) { + return Collections.emptyList(); + } + + List> partitionNameFileNameList = new ArrayList<>(fileToKeysMap.keySet()); + Map, ByteBuffer> fileToBloomFilterMap = + hoodieTable.getMetadataTable().getBloomFilters(partitionNameFileNameList); + + final AtomicInteger totalKeys = new AtomicInteger(0); + fileToKeysMap.forEach((partitionPathFileNamePair, hoodieKeyList) -> { + final String partitionPath = partitionPathFileNamePair.getLeft(); + final String fileName = partitionPathFileNamePair.getRight(); + final String fileId = FSUtils.getFileId(fileName); + ValidationUtils.checkState(!fileId.isEmpty()); + + if (!fileToBloomFilterMap.containsKey(partitionPathFileNamePair)) { + throw new HoodieIndexException("Failed to get the bloom filter for " + partitionPathFileNamePair); + } + final ByteBuffer fileBloomFilterByteBuffer = fileToBloomFilterMap.get(partitionPathFileNamePair); + + HoodieDynamicBoundedBloomFilter fileBloomFilter = + new HoodieDynamicBoundedBloomFilter(StandardCharsets.UTF_8.decode(fileBloomFilterByteBuffer).toString(), + BloomFilterTypeCode.DYNAMIC_V0); + + List candidateRecordKeys = new ArrayList<>(); + hoodieKeyList.forEach(hoodieKey -> { + totalKeys.incrementAndGet(); + if (fileBloomFilter.mightContain(hoodieKey.getRecordKey())) { + candidateRecordKeys.add(hoodieKey.getRecordKey()); + } + }); + + final HoodieBaseFile dataFile = fileIDBaseFileMap.get(fileId); + List matchingKeys = + HoodieIndexUtils.filterKeysFromFile(new Path(dataFile.getPath()), candidateRecordKeys, + hoodieTable.getHadoopConf()); + LOG.debug( + String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)", + hoodieKeyList.size(), candidateRecordKeys.size(), + candidateRecordKeys.size() - matchingKeys.size(), matchingKeys.size())); + + resultList.add(new HoodieKeyLookupResult(fileId, partitionPath, dataFile.getCommitTime(), matchingKeys)); + }); + return resultList; + } + + @Override + protected void end() { + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java index bbb50d5cf6fff..04afc4dd16910 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java @@ -29,20 +29,19 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaPairRDD; import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.io.HoodieKeyLookupResult; import org.apache.hudi.table.HoodieTable; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; +import scala.Tuple2; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import scala.Tuple2; - /** * Helper for {@link HoodieBloomIndex} containing Spark-specific logic. */ @@ -70,26 +69,38 @@ public HoodiePairData findMatchingFilesForRecor JavaRDD> fileComparisonsRDD = HoodieJavaRDD.getJavaRDD(fileComparisonPairs) .map(pair -> new Tuple2<>(pair.getLeft(), pair.getRight())); - Map comparisonsPerFileGroup = computeComparisonsPerFileGroup( - config, recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context); - int inputParallelism = - HoodieJavaPairRDD.getJavaPairRDD(partitionRecordKeyPairs).partitions().size(); + + int inputParallelism = HoodieJavaPairRDD.getJavaPairRDD(partitionRecordKeyPairs).partitions().size(); int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism()); LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${" + config.getBloomIndexParallelism() + "}"); - if (config.useBloomIndexBucketizedChecking()) { + JavaRDD> keyLookupResultRDD; + if (config.isMetadataBloomFilterIndexEnabled()) { + // Step 1: Sort by file id + JavaRDD> sortedFileIdAndKeyPairs = + fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism); + + // Step 2: Use bloom filter to filter and the actual log file to get the record location + keyLookupResultRDD = sortedFileIdAndKeyPairs.mapPartitionsWithIndex( + new HoodieMetadataBloomIndexCheckFunction(hoodieTable), true); + } else if (config.useBloomIndexBucketizedChecking()) { + Map comparisonsPerFileGroup = computeComparisonsPerFileGroup( + config, recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context); Partitioner partitioner = new BucketizedBloomCheckPartitioner(joinParallelism, comparisonsPerFileGroup, config.getBloomIndexKeysPerBucket()); - fileComparisonsRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t)) - .repartitionAndSortWithinPartitions(partitioner).map(Tuple2::_2); + keyLookupResultRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t)) + .repartitionAndSortWithinPartitions(partitioner) + .map(Tuple2::_2) + .mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true); } else { - fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism); + keyLookupResultRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism) + .mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true); } - return HoodieJavaPairRDD.of(fileComparisonsRDD.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true) - .flatMap(List::iterator).filter(lr -> lr.getMatchingRecordKeys().size() > 0) + return HoodieJavaPairRDD.of(keyLookupResultRDD.flatMap(List::iterator) + .filter(lr -> lr.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream() .map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()), new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId()))) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java index 7b2f93ec887e7..521bdb20c58fc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java @@ -200,7 +200,7 @@ public static Dataset buildColumnStatsTableFor( indexRow.add(minMaxValue.getLeft()); // min indexRow.add(minMaxValue.getRight()); // max - indexRow.add(colMetadata.getNumNulls()); + indexRow.add(colMetadata.getNullCount()); }); return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(indexRow)); 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 ccb258a8cdc61..1a32ae5e9aa26 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 @@ -25,14 +25,13 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; -import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.metrics.DistributedRegistry; @@ -43,6 +42,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter { @@ -121,11 +121,12 @@ protected void initialize(HoodieEngineContext eng } } - protected void commit(HoodieData hoodieDataRecords, String partitionName, String instantTime, boolean canTriggerTableService) { + @Override + protected void commit(String instantTime, Map> partitionRecordsMap, boolean canTriggerTableService) { ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet."); ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); - JavaRDD records = (JavaRDD) hoodieDataRecords.get(); - JavaRDD recordRDD = prepRecords(records, partitionName, 1); + HoodieData preppedRecords = prepRecords(partitionRecordsMap); + JavaRDD preppedRecordRDD = HoodieJavaRDD.getJavaRDD(preppedRecords); try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { if (canTriggerTableService) { @@ -150,7 +151,7 @@ protected void commit(HoodieData hoodieDataRecords, String partiti HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant); metadataMetaClient.reloadActiveTimeline(); } - List statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect(); + List statuses = writeClient.upsertPreppedRecords(preppedRecordRDD, instantTime).collect(); statuses.forEach(writeStatus -> { if (writeStatus.hasErrors()) { throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime); @@ -168,20 +169,4 @@ protected void commit(HoodieData hoodieDataRecords, String partiti // Update total size of the metadata and count of base/log files metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata)); } - - /** - * Tag each record with the location in the given partition. - * - * The record is tagged with respective file slice's location based on its record key. - */ - private JavaRDD prepRecords(JavaRDD recordsRDD, String partitionName, int numFileGroups) { - List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, partitionName); - ValidationUtils.checkArgument(fileSlices.size() == numFileGroups, String.format("Invalid number of file groups: found=%d, required=%d", fileSlices.size(), numFileGroups)); - - return recordsRDD.map(r -> { - FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), numFileGroups)); - r.setCurrentLocation(new HoodieRecordLocation(slice.getBaseInstantTime(), slice.getFileId())); - return r; - }); - } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 88a6f02fc3cbc..f175dc5dea021 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -191,6 +191,8 @@ public void testMetadataTableBootstrap(HoodieTableType tableType, boolean addRol // trigger couple of upserts doWriteOperation(testTable, "0000005"); doWriteOperation(testTable, "0000006"); + doWriteOperation(testTable, "0000007"); + doCleanAndValidate(testTable, "0000008", Arrays.asList("0000007")); validateMetadata(testTable, true); } @@ -222,7 +224,7 @@ public void testOnlyValidPartitionsAdded(HoodieTableType tableType) throws Excep testTable.doWriteOperation("0000003", UPSERT, emptyList(), asList("p1", "p2"), 1, true); syncTableMetadata(writeConfig); - List partitions = metadataWriter(writeConfig).metadata().getAllPartitionPaths(); + List partitions = metadataWriter(writeConfig).getTableMetadata().getAllPartitionPaths(); assertFalse(partitions.contains(nonPartitionDirectory), "Must not contain the non-partition " + nonPartitionDirectory); assertTrue(partitions.contains("p1"), "Must contain partition p1"); @@ -345,6 +347,7 @@ public void testInsertUpsertCluster(HoodieTableType tableType) throws Exception if (tableType == MERGE_ON_READ) { doCompaction(testTable, "0000004"); } + doCleanAndValidate(testTable, "0000005", Arrays.asList("0000001")); validateMetadata(testTable, emptyList(), true); } @@ -380,6 +383,32 @@ public void testMetadataTableServices() throws Exception { assertEquals(tableMetadata.getLatestCompactionTime().get(), "0000003001"); } + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testTableOperationsWithMetadataIndex(HoodieTableType tableType) throws Exception { + initPath(); + HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) + .withIndexConfig(HoodieIndexConfig.newBuilder() + .bloomIndexBucketizedChecking(false) + .build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .withMetadataIndexBloomFilter(true) + .withMetadataIndexBloomFilterFileGroups(4) + .withMetadataIndexColumnStats(true) + .withMetadataIndexBloomFilterFileGroups(2) + .withMetadataIndexForAllColumns(true) + .build()) + .build(); + init(tableType, writeConfig); + testTableOperationsForMetaIndexImpl(writeConfig); + } + + private void testTableOperationsForMetaIndexImpl(final HoodieWriteConfig writeConfig) throws Exception { + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + testTableOperationsImpl(engineContext, writeConfig); + } + /** * Tests that virtual key configs are honored in base files after compaction in metadata table. * @@ -619,7 +648,7 @@ private void verifyMetadataRecordKeyExcludeFromPayloadLogFiles(HoodieTable table // Compaction should not be triggered yet. Let's verify no base file // and few log files available. List fileSlices = table.getSliceView() - .getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + .getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList()); if (fileSlices.isEmpty()) { throw new IllegalStateException("LogFile slices are not available!"); } @@ -709,7 +738,7 @@ private void verifyMetadataMergedRecords(HoodieTableMetaClient metadataMetaClien .withBasePath(metadataMetaClient.getBasePath()) .withLogFilePaths(logFilePaths) .withLatestInstantTime(latestCommitTimestamp) - .withPartition(MetadataPartitionType.FILES.partitionPath()) + .withPartition(MetadataPartitionType.FILES.getPartitionPath()) .withReaderSchema(schema) .withMaxMemorySizeInBytes(100000L) .withBufferSize(4096) @@ -739,7 +768,7 @@ private void verifyMetadataMergedRecords(HoodieTableMetaClient metadataMetaClien private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table, boolean enableMetaFields) throws IOException { table.getHoodieView().sync(); List fileSlices = table.getSliceView() - .getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + .getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList()); if (!fileSlices.get(0).getBaseFile().isPresent()) { throw new IllegalStateException("Base file not available!"); } @@ -1058,10 +1087,20 @@ public void testMetadataPayloadSpuriousDeletes(boolean ignoreSpuriousDeletes) th public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exception { init(tableType); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) + .withRollbackUsingMarkers(false).build(); + testTableOperationsImpl(engineContext, writeConfig); + } - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, - getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).build())) { - + /** + * Test all major table operations with the given table, config and context. + * + * @param engineContext - Engine context + * @param writeConfig - Write config + * @throws IOException + */ + private void testTableOperationsImpl(HoodieSparkEngineContext engineContext, HoodieWriteConfig writeConfig) throws IOException { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { // Write 1 (Bulk insert) String newCommitTime = "0000001"; List records = dataGen.generateInserts(newCommitTime, 20); @@ -1738,7 +1777,7 @@ public void testMetadataMetrics() throws Exception { assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".count")); assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".totalDuration")); assertTrue(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_STR + ".count") >= 1L); - final String prefix = MetadataPartitionType.FILES.partitionPath() + "."; + final String prefix = MetadataPartitionType.FILES.getPartitionPath() + "."; assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)); assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_LOG_FILES)); assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)); @@ -1931,7 +1970,10 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException // in the .hoodie folder. List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath), false, false); - assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); + assertEquals(metadataWriter.getEnabledPartitionTypes().size(), metadataTablePartitions.size()); + + final Map metadataEnabledPartitionTypes = new HashMap<>(); + metadataWriter.getEnabledPartitionTypes().forEach(e -> metadataEnabledPartitionTypes.put(e.getPartitionPath(), e)); // Metadata table should automatically compact and clean // versions are +1 as autoclean / compaction happens end of commits @@ -1939,10 +1981,13 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metadataMetaClient, metadataMetaClient.getActiveTimeline()); metadataTablePartitions.forEach(partition -> { List latestSlices = fsView.getLatestFileSlices(partition).collect(Collectors.toList()); - assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count() <= 1, "Should have a single latest base file"); - assertTrue(latestSlices.size() <= 1, "Should have a single latest file slice"); - assertTrue(latestSlices.size() <= numFileVersions, "Should limit file slice to " - + numFileVersions + " but was " + latestSlices.size()); + assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count() + <= metadataEnabledPartitionTypes.get(partition).getFileGroupCount(), "Should have a single latest base file per file group"); + assertTrue(latestSlices.size() + <= metadataEnabledPartitionTypes.get(partition).getFileGroupCount(), "Should have a single latest file slice per file group"); + assertTrue(latestSlices.size() + <= (numFileVersions * metadataEnabledPartitionTypes.get(partition).getFileGroupCount()), "Should limit file slice to " + + numFileVersions + " per file group, but was " + latestSlices.size()); }); LOG.info("Validation time=" + timer.endTimer()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index 1abe15bd008d8..48c2e19b4aff7 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -247,7 +247,7 @@ private void verifyMetadataRecordKeyExcludeFromPayloadLogFiles(HoodieTable table // Compaction should not be triggered yet. Let's verify no base file // and few log files available. List fileSlices = table.getSliceView() - .getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + .getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList()); if (fileSlices.isEmpty()) { throw new IllegalStateException("LogFile slices are not available!"); } @@ -322,7 +322,7 @@ private void verifyMetadataMergedRecords(HoodieTableMetaClient metadataMetaClien .withBasePath(metadataMetaClient.getBasePath()) .withLogFilePaths(logFilePaths) .withLatestInstantTime(latestCommitTimestamp) - .withPartition(MetadataPartitionType.FILES.partitionPath()) + .withPartition(MetadataPartitionType.FILES.getPartitionPath()) .withReaderSchema(schema) .withMaxMemorySizeInBytes(100000L) .withBufferSize(4096) @@ -351,7 +351,7 @@ private void verifyMetadataMergedRecords(HoodieTableMetaClient metadataMetaClien private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table) throws IOException { table.getHoodieView().sync(); List fileSlices = table.getSliceView() - .getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + .getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList()); if (!fileSlices.get(0).getBaseFile().isPresent()) { throw new IllegalStateException("Base file not available!"); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index f419858558690..97572de258d6c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; @@ -84,27 +85,22 @@ public void init(HoodieTableType tableType) throws IOException { init(tableType, true); } + public void init(HoodieTableType tableType, HoodieWriteConfig writeConfig) throws IOException { + init(tableType, Option.of(writeConfig), true, false, false, false); + } + public void init(HoodieTableType tableType, boolean enableMetadataTable) throws IOException { init(tableType, enableMetadataTable, true, false, false); } public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan, boolean enableMetrics, boolean - validateMetadataPayloadStateConsistency) throws IOException { - this.tableType = tableType; - initPath(); - initSparkContexts("TestHoodieMetadata"); - initFileSystem(); - fs.mkdirs(new Path(basePath)); - initTimelineService(); - initMetaClient(tableType); - initTestDataGenerator(); - metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); - writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, enableMetrics, - enableFullScan, true, validateMetadataPayloadStateConsistency).build(); - initWriteConfigAndMetatableWriter(writeConfig, enableMetadataTable); + validateMetadataPayloadStateConsistency) throws IOException { + init(tableType, Option.empty(), enableMetadataTable, enableFullScan, enableMetrics, + validateMetadataPayloadStateConsistency); } - public void init(HoodieTableType tableType, HoodieWriteConfig writeConfig) throws IOException { + public void init(HoodieTableType tableType, Option writeConfig, boolean enableMetadataTable, + boolean enableFullScan, boolean enableMetrics, boolean validateMetadataPayloadStateConsistency) throws IOException { this.tableType = tableType; initPath(); initSparkContexts("TestHoodieMetadata"); @@ -114,8 +110,12 @@ public void init(HoodieTableType tableType, HoodieWriteConfig writeConfig) throw initMetaClient(tableType); initTestDataGenerator(); metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); - this.writeConfig = writeConfig; - initWriteConfigAndMetatableWriter(writeConfig, writeConfig.isMetadataTableEnabled()); + this.writeConfig = writeConfig.isPresent() + ? writeConfig.get() : getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, + enableMetadataTable, enableMetrics, enableFullScan, true, + validateMetadataPayloadStateConsistency) + .build(); + initWriteConfigAndMetatableWriter(this.writeConfig, enableMetadataTable); } protected void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java index 1334adb20d052..b9063a781db3e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java @@ -18,9 +18,13 @@ package org.apache.hudi.index.bloom; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.functional.TestHoodieMetadataBase; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -32,14 +36,10 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaPairRDD; import org.apache.hudi.data.HoodieJavaRDD; -import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; - -import org.apache.avro.Schema; -import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; @@ -48,6 +48,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import scala.Tuple2; import java.nio.file.Paths; import java.util.Arrays; @@ -59,8 +60,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import scala.Tuple2; - import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -69,14 +68,14 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; -public class TestHoodieBloomIndex extends HoodieClientTestHarness { +public class TestHoodieBloomIndex extends TestHoodieMetadataBase { private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBloomIndex.class, "/exampleSchema.avsc", true); private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}"; public static Stream configParams() { Object[][] data = - new Object[][] {{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}}; + new Object[][]{{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}}; return Stream.of(data).map(Arguments::of); } @@ -99,6 +98,10 @@ private HoodieWriteConfig makeConfig(boolean rangePruning, boolean treeFiltering .withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning) .bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking) .bloomIndexKeysPerBucket(2).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .withMetadataIndexBloomFilter(false) + .withMetadataIndexColumnStats(false) + .build()) .build(); } @@ -134,7 +137,7 @@ public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, b new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); - List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + List> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); // Still 0, as no valid commit assertEquals(0, filesList.size()); @@ -143,7 +146,7 @@ public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, b .withInserts("2015/03/12", "3", record1) .withInserts("2015/03/12", "4", record2, record3, record4); - filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); assertEquals(4, filesList.size()); if (rangePruning) { @@ -241,9 +244,9 @@ public void testCheckUUIDsAgainstOneFile() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient); - HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId)); - List results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids, - new Path(Paths.get(basePath, partition, filename).toString())); + List results = HoodieIndexUtils.filterKeysFromFile( + new Path(Paths.get(basePath, partition, filename).toString()), uuids, hadoopConf); + assertEquals(results.size(), 2); assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") || results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java index fa7d586d2dc0a..3f473ba175f4a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java @@ -109,7 +109,7 @@ public void testLoadInvolvedFiles() throws Exception { // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up List partitions = Arrays.asList("2016/01/21", "2016/04/01"); // partitions will NOT be respected by this loadInvolvedFiles(...) call - List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + List> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); // Still 0, as no valid commit assertEquals(0, filesList.size()); @@ -118,7 +118,7 @@ public void testLoadInvolvedFiles() throws Exception { .withInserts("2015/03/12", "3", record1) .withInserts("2015/03/12", "4", record2, record3, record4); - filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); assertEquals(4, filesList.size()); Map filesMap = toFileMap(filesList); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index 906f13d7a63b8..f339f5ed910db 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -59,7 +59,6 @@ import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; -import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -680,7 +679,7 @@ private void runFullValidation(HoodieWriteConfig writeConfig, String metadataTab // in the .hoodie folder. List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath), false, false); - Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); + Assertions.assertEquals(metadataWriter.getEnabledPartitionTypes().size(), metadataTablePartitions.size()); // Metadata table should automatically compact and clean // versions are +1 as autoClean / compaction happens end of commits diff --git a/hudi-common/src/main/avro/HoodieMetadata.avsc b/hudi-common/src/main/avro/HoodieMetadata.avsc index bf85587a3a7ac..6b18f03339bd7 100644 --- a/hudi-common/src/main/avro/HoodieMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieMetadata.avsc @@ -30,27 +30,140 @@ "doc": "Type of the metadata record", "type": "int" }, - { "name": "filesystemMetadata", + { "doc": "Contains information about partitions and files within the dataset", - "type": ["null", { - "type": "map", - "values": { + "name": "filesystemMetadata", + "type": [ + "null", + { + "type": "map", + "values": { + "type": "record", + "name": "HoodieMetadataFileInfo", + "fields": [ + { + "name": "size", + "type": "long", + "doc": "Size of the file" + }, + { + "name": "isDeleted", + "type": "boolean", + "doc": "True if this file has been deleted" + } + ] + } + } + ] + }, + { + "doc": "Metadata Index of bloom filters for all data files in the user table", + "name": "BloomFilterMetadata", + "type": [ + "null", + { + "doc": "Data file bloom filter details", + "name": "HoodieMetadataBloomFilter", "type": "record", - "name": "HoodieMetadataFileInfo", "fields": [ { - "name": "size", - "type": "long", - "doc": "Size of the file" + "doc": "Bloom filter type code", + "name": "type", + "type": "string" + }, + { + "doc": "Instant timestamp when this metadata was created/updated", + "name": "timestamp", + "type": "string" + }, + { + "doc": "Bloom filter binary byte array", + "name": "bloomFilter", + "type": "bytes" + }, + { + "doc": "Bloom filter entry valid/deleted flag", + "name": "isDeleted", + "type": "boolean" + } + ] + } + ] + }, + { + "doc": "Metadata Index of column statistics for all data files in the user table", + "name": "ColumnStatsMetadata", + "type": [ + "null", + { + "doc": "Data file column statistics", + "name": "HoodieMetadataColumnStats", + "type": "record", + "fields": [ + { + "doc": "File name for which this column statistics applies", + "name": "fileName", + "type": [ + "null", + "string" + ] + }, + { + "doc": "Minimum value in the range. Based on user data table schema, we can convert this to appropriate type", + "name": "minValue", + "type": [ + "null", + "string" + ] + }, + { + "doc": "Maximum value in the range. Based on user data table schema, we can convert it to appropriate type", + "name": "maxValue", + "type": [ + "null", + "string" + ] + }, + { + "doc": "Total count of values", + "name": "valueCount", + "type": [ + "null", + "long" + ] + }, + { + "doc": "Total count of null values", + "name": "nullCount", + "type": [ + "null", + "long" + ] + }, + { + "doc": "Total storage size on disk", + "name": "totalSize", + "type": [ + "null", + "long" + ] + }, + { + "doc": "Total uncompressed storage size on disk", + "name": "totalUncompressedSize", + "type": [ + "null", + "long" + ] }, { + "doc": "Column range entry valid/deleted flag", "name": "isDeleted", - "type": "boolean", - "doc": "True if this file has been deleted" + "type": "boolean" } ] } - }] + ] } ] } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java b/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java index 343822b13adec..d4bc287c551c2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java @@ -63,7 +63,7 @@ public class HoodieDynamicBoundedBloomFilter implements BloomFilter { * @param serString the serialized string which represents the {@link HoodieDynamicBoundedBloomFilter} * @param typeCode type code of the bloom filter */ - HoodieDynamicBoundedBloomFilter(String serString, BloomFilterTypeCode typeCode) { + public HoodieDynamicBoundedBloomFilter(String serString, BloomFilterTypeCode typeCode) { // ignoring the type code for now, since we have just one version byte[] bytes = Base64CodecUtil.decode(serString); DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 21ba5f4dd2083..6ce6314bbad2f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -124,6 +124,47 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.10.0") .withDocumentation("Enable full scanning of log files while reading log records. If disabled, hudi does look up of only interested entries."); + public static final ConfigProperty ENABLE_METADATA_INDEX_BLOOM_FILTER = ConfigProperty + .key(METADATA_PREFIX + ".index.bloom.filter.enable") + .defaultValue(false) + .sinceVersion("0.11.0") + .withDocumentation("Enable indexing user data files bloom filters under metadata table. When enabled, " + + "metadata table will have a partition to store the bloom filter index and will be " + + "used during the index lookups."); + + public static final ConfigProperty METADATA_INDEX_BLOOM_FILTER_FILE_GROUP_COUNT = ConfigProperty + .key(METADATA_PREFIX + ".index.bloom.filter.file.group.count") + .defaultValue(4) + .sinceVersion("0.11.0") + .withDocumentation("Metadata bloom filter index partition file group count. This controls the size of the base and " + + "log files and read parallelism in the bloom filter index partition. The recommendation is to size the " + + "file group count such that the base files are under 1GB."); + + public static final ConfigProperty ENABLE_METADATA_INDEX_COLUMN_STATS = ConfigProperty + .key(METADATA_PREFIX + ".index.column.stats.enable") + .defaultValue(false) + .sinceVersion("0.11.0") + .withDocumentation("Enable indexing user data files column ranges under metadata table key lookups. When " + + "enabled, metadata table will have a partition to store the column ranges and will " + + "used for pruning files during the index lookups."); + + public static final ConfigProperty METADATA_INDEX_COLUMN_STATS_FILE_GROUP_COUNT = ConfigProperty + .key(METADATA_PREFIX + ".index.column.stats.file.group.count") + .defaultValue(2) + .sinceVersion("0.11.0") + .withDocumentation("Metadata column stats partition file group count. This controls the size of the base and " + + "log files and read parallelism in the column stats index partition. The recommendation is to size the " + + "file group count such that the base files are under 1GB."); + + public static final ConfigProperty ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS = ConfigProperty + .key(METADATA_PREFIX + ".index.column.stats.all_columns.enable") + .defaultValue(true) + .sinceVersion("0.11.0") + .withDocumentation("Enable indexing user data files column ranges under metadata table key lookups. When " + + "enabled, metadata table will have a partition to store the column ranges and will " + + "used for pruning files during the index lookups. Only applies if " + + ENABLE_METADATA_INDEX_COLUMN_STATS.key() + " is enabled.A"); + public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty .key(METADATA_PREFIX + ".populate.meta.fields") .defaultValue(false) @@ -157,6 +198,26 @@ public boolean enabled() { return getBoolean(ENABLE); } + public boolean isBloomFilterIndexEnabled() { + return getBooleanOrDefault(ENABLE_METADATA_INDEX_BLOOM_FILTER); + } + + public boolean isColumnStatsIndexEnabled() { + return getBooleanOrDefault(ENABLE_METADATA_INDEX_COLUMN_STATS); + } + + public boolean isMetadataColumnStatsIndexForAllColumnsEnabled() { + return getBooleanOrDefault(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS); + } + + public int getBloomFilterIndexFileGroupCount() { + return getIntOrDefault(METADATA_INDEX_BLOOM_FILTER_FILE_GROUP_COUNT); + } + + public int getColumnStatsIndexFileGroupCount() { + return getIntOrDefault(METADATA_INDEX_COLUMN_STATS_FILE_GROUP_COUNT); + } + public boolean enableMetrics() { return getBoolean(METRICS_ENABLE); } @@ -199,6 +260,31 @@ public Builder enable(boolean enable) { return this; } + public Builder withMetadataIndexBloomFilter(boolean enable) { + metadataConfig.setValue(ENABLE_METADATA_INDEX_BLOOM_FILTER, String.valueOf(enable)); + return this; + } + + public Builder withMetadataIndexBloomFilterFileGroups(int fileGroupCount) { + metadataConfig.setValue(METADATA_INDEX_BLOOM_FILTER_FILE_GROUP_COUNT, String.valueOf(fileGroupCount)); + return this; + } + + public Builder withMetadataIndexColumnStats(boolean enable) { + metadataConfig.setValue(ENABLE_METADATA_INDEX_COLUMN_STATS, String.valueOf(enable)); + return this; + } + + public Builder withMetadataIndexColumnStatsFileGroupCount(int fileGroupCount) { + metadataConfig.setValue(METADATA_INDEX_COLUMN_STATS_FILE_GROUP_COUNT, String.valueOf(fileGroupCount)); + return this; + } + + public Builder withMetadataIndexForAllColumns(boolean enable) { + metadataConfig.setValue(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS, String.valueOf(enable)); + return this; + } + public Builder enableMetrics(boolean enableMetrics) { metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics)); return this; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java index 31dcd8765d59c..f25d76813357e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java @@ -28,14 +28,21 @@ public class HoodieColumnRangeMetadata { private final String columnName; private final T minValue; private final T maxValue; - private final long numNulls; + private final long nullCount; + private final long valueCount; + private final long totalSize; + private final long totalUncompressedSize; - public HoodieColumnRangeMetadata(final String filePath, final String columnName, final T minValue, final T maxValue, final long numNulls) { + public HoodieColumnRangeMetadata(final String filePath, final String columnName, final T minValue, final T maxValue, + final long nullCount, long valueCount, long totalSize, long totalUncompressedSize) { this.filePath = filePath; this.columnName = columnName; this.minValue = minValue; this.maxValue = maxValue; - this.numNulls = numNulls; + this.nullCount = nullCount; + this.valueCount = valueCount; + this.totalSize = totalSize; + this.totalUncompressedSize = totalUncompressedSize; } public String getFilePath() { @@ -54,8 +61,20 @@ public T getMaxValue() { return this.maxValue; } - public long getNumNulls() { - return numNulls; + public long getNullCount() { + return nullCount; + } + + public long getValueCount() { + return valueCount; + } + + public long getTotalSize() { + return totalSize; + } + + public long getTotalUncompressedSize() { + return totalUncompressedSize; } @Override @@ -71,12 +90,15 @@ public boolean equals(final Object o) { && Objects.equals(getColumnName(), that.getColumnName()) && Objects.equals(getMinValue(), that.getMinValue()) && Objects.equals(getMaxValue(), that.getMaxValue()) - && Objects.equals(getNumNulls(), that.getNumNulls()); + && Objects.equals(getNullCount(), that.getNullCount()) + && Objects.equals(getValueCount(), that.getValueCount()) + && Objects.equals(getTotalSize(), that.getTotalSize()) + && Objects.equals(getTotalUncompressedSize(), that.getTotalUncompressedSize()); } @Override public int hashCode() { - return Objects.hash(getColumnName(), getMinValue(), getMaxValue(), getNumNulls()); + return Objects.hash(getColumnName(), getMinValue(), getMaxValue(), getNullCount()); } @Override @@ -86,6 +108,10 @@ public String toString() { + "columnName='" + columnName + '\'' + ", minValue=" + minValue + ", maxValue=" + maxValue - + ", numNulls=" + numNulls + '}'; + + ", nullCount=" + nullCount + + ", valueCount=" + valueCount + + ", totalSize=" + totalSize + + ", totalUncompressedSize=" + totalUncompressedSize + + '}'; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index 0ae388150c50d..e8ffecedc7411 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -175,11 +175,11 @@ protected String getKeyField() { return this.simpleKeyGenFields.get().getKey(); } - public void scan() { + public synchronized void scan() { scan(Option.empty()); } - public void scan(Option> keys) { + public synchronized void scan(Option> keys) { currentInstantLogBlocks = new ArrayDeque<>(); progress = 0.0f; totalLogFiles = new AtomicLong(0); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index 66884fe80ec1f..ff85a6fd9d276 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -30,6 +30,8 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.MetadataNotFoundException; import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.avro.AvroSchemaConverter; @@ -62,6 +64,8 @@ */ public class ParquetUtils extends BaseFileUtils { + private static final Logger LOG = LogManager.getLogger(ParquetUtils.class); + /** * Read the rowKey list matching the given filter, from the given parquet file. If the filter is empty, then this will * return all the rowkeys. @@ -300,18 +304,21 @@ public List> readRangeFromParquetMetadata( Map>> columnToStatsListMap = metadata.getBlocks().stream().sequential() .flatMap(blockMetaData -> blockMetaData.getColumns().stream() .filter(f -> cols.contains(f.getPath().toDotString())) - .map(columnChunkMetaData -> - new HoodieColumnRangeMetadata( - parquetFilePath.getName(), - columnChunkMetaData.getPath().toDotString(), - convertToNativeJavaType( - columnChunkMetaData.getPrimitiveType(), - columnChunkMetaData.getStatistics().genericGetMin()), - convertToNativeJavaType( - columnChunkMetaData.getPrimitiveType(), - columnChunkMetaData.getStatistics().genericGetMax()), - columnChunkMetaData.getStatistics().getNumNulls()))) - .collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName)); + .map(columnChunkMetaData -> + new HoodieColumnRangeMetadata( + parquetFilePath.getName(), + columnChunkMetaData.getPath().toDotString(), + convertToNativeJavaType( + columnChunkMetaData.getPrimitiveType(), + columnChunkMetaData.getStatistics().genericGetMin()), + convertToNativeJavaType( + columnChunkMetaData.getPrimitiveType(), + columnChunkMetaData.getStatistics().genericGetMax()), + columnChunkMetaData.getStatistics().getNumNulls(), + columnChunkMetaData.getValueCount(), + columnChunkMetaData.getTotalSize(), + columnChunkMetaData.getTotalUncompressedSize())) + ).collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName)); // Combine those into file-level statistics // NOTE: Inlining this var makes javac (1.8) upset (due to its inability to infer @@ -355,13 +362,17 @@ private > HoodieColumnRangeMetadata combineRanges( maxValue = one.getMaxValue().compareTo(another.getMaxValue()) < 0 ? another.getMaxValue() : one.getMaxValue(); } else if (one.getMaxValue() == null) { maxValue = another.getMaxValue(); - } else { + } else { maxValue = one.getMaxValue(); } return new HoodieColumnRangeMetadata( one.getFilePath(), - one.getColumnName(), minValue, maxValue, one.getNumNulls() + another.getNumNulls()); + one.getColumnName(), minValue, maxValue, + one.getNullCount() + another.getNullCount(), + one.getValueCount() + another.getValueCount(), + one.getTotalSize() + another.getTotalSize(), + one.getTotalUncompressedSize() + another.getTotalUncompressedSize()); } private static Comparable convertToNativeJavaType(PrimitiveType primitiveType, Comparable val) { @@ -408,7 +419,7 @@ private static BigDecimal extractDecimal(Object val, DecimalMetadata decimalMeta return BigDecimal.valueOf((Long) val, scale); } else if (val instanceof Binary) { // NOTE: Unscaled number is stored in BE format (most significant byte is 0th) - return new BigDecimal(new BigInteger(((Binary)val).getBytesUnsafe()), scale); + return new BigDecimal(new BigInteger(((Binary) val).getBytesUnsafe()), scale); } else { throw new UnsupportedOperationException(String.format("Unsupported value type (%s)", val.getClass().getName())); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/ColumnID.java b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/ColumnIndexID.java similarity index 80% rename from hudi-common/src/main/java/org/apache/hudi/common/util/hash/ColumnID.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/hash/ColumnIndexID.java index be4db44ecd961..92e60b30a311f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/ColumnID.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/ColumnIndexID.java @@ -24,14 +24,21 @@ /** * A stateful Hoodie object ID representing any table column. */ -public class ColumnID extends HoodieID { +public class ColumnIndexID extends HoodieIndexID { private static final Type TYPE = Type.COLUMN; - private static final HashID.Size ID_COLUMN_HASH_SIZE = HashID.Size.BITS_64; + public static final HashID.Size ID_COLUMN_HASH_SIZE = HashID.Size.BITS_64; + private final String column; private final byte[] hash; - public ColumnID(final String message) { - this.hash = HashID.hash(message, ID_COLUMN_HASH_SIZE); + public ColumnIndexID(final String column) { + this.column = column; + this.hash = HashID.hash(column, ID_COLUMN_HASH_SIZE); + } + + @Override + public String getName() { + return column; } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/FileID.java b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/FileIndexID.java similarity index 84% rename from hudi-common/src/main/java/org/apache/hudi/common/util/hash/FileID.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/hash/FileIndexID.java index 0cb73c5abf9a8..3f9616908bb39 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/FileID.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/FileIndexID.java @@ -24,14 +24,21 @@ /** * Hoodie object ID representing any file. */ -public class FileID extends HoodieID { +public class FileIndexID extends HoodieIndexID { private static final Type TYPE = Type.FILE; private static final HashID.Size ID_FILE_HASH_SIZE = HashID.Size.BITS_128; + private final String fileName; private final byte[] hash; - public FileID(final String message) { - this.hash = HashID.hash(message, ID_FILE_HASH_SIZE); + public FileIndexID(final String fileName) { + this.fileName = fileName; + this.hash = HashID.hash(fileName, ID_FILE_HASH_SIZE); + } + + @Override + public String getName() { + return fileName; } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/HoodieID.java b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/HoodieIndexID.java similarity index 89% rename from hudi-common/src/main/java/org/apache/hudi/common/util/hash/HoodieID.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/hash/HoodieIndexID.java index e08e254b0a215..139efd17ed0ae 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/HoodieID.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/HoodieIndexID.java @@ -24,9 +24,10 @@ import java.io.Serializable; /** - * A serializable ID that can be used to identify any Hoodie table fields and resources. + * A serializable ID that can be used to identify any Hoodie table fields and + * resources in the on-disk index. */ -public abstract class HoodieID implements Serializable { +public abstract class HoodieIndexID implements Serializable { private static final long serialVersionUID = 1L; @@ -50,6 +51,13 @@ public String toString() { } } + /** + * Get the resource name for which this index id is generated. + * + * @return The resource name + */ + public abstract String getName(); + /** * Get the number of bits representing this ID in memory. *

@@ -74,7 +82,7 @@ public String toString() { public abstract String toString(); /** - * + * Get the Base64 encoded version of the ID. */ public String asBase64EncodedString() { throw new HoodieNotSupportedException("Unsupported hash for " + getType()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/PartitionID.java b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/PartitionIndexID.java similarity index 83% rename from hudi-common/src/main/java/org/apache/hudi/common/util/hash/PartitionID.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/hash/PartitionIndexID.java index f31159faa2a2f..0fbae27b80de8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/PartitionID.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/PartitionIndexID.java @@ -24,14 +24,21 @@ /** * Hoodie object ID representing any partition. */ -public class PartitionID extends HoodieID { +public class PartitionIndexID extends HoodieIndexID { private static final Type TYPE = Type.PARTITION; private static final HashID.Size ID_PARTITION_HASH_SIZE = HashID.Size.BITS_64; + private final String partition; private final byte[] hash; - public PartitionID(final String message) { - this.hash = HashID.hash(message, ID_PARTITION_HASH_SIZE); + public PartitionIndexID(final String partition) { + this.partition = partition; + this.hash = HashID.hash(partition, ID_PARTITION_HASH_SIZE); + } + + @Override + public String getName() { + return partition; } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java index 38e58f63e419d..cb330b81432bf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.Set; import org.apache.avro.Schema; @@ -35,6 +37,10 @@ public interface HoodieFileReader extends AutoCloseable public Set filterRowKeys(Set candidateRowKeys); + default Map getRecordsByKeys(List rowKeys) throws IOException { + throw new UnsupportedOperationException(); + } + public Iterator getRecordIterator(Schema readerSchema) throws IOException; default Iterator getRecordIterator() throws IOException { diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index 893350f3af683..3404d2bd55b55 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -22,12 +22,14 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashSet; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -53,8 +55,11 @@ import org.apache.hudi.common.util.io.ByteBufferBackedInputStream; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; -public class HoodieHFileReader implements HoodieFileReader { +public class HoodieHFileReader implements HoodieFileReader { + private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class); private Path path; private Configuration conf; private HFile.Reader reader; @@ -133,23 +138,50 @@ public BloomFilter readBloomFilter() { } } + /** + * Filter keys by availability. + *

+ * Note: This method is performant when the caller passes in a sorted candidate keys. + * + * @param candidateRowKeys - Keys to check for the availability + * @return Subset of candidate keys that are available + */ @Override - public Set filterRowKeys(Set candidateRowKeys) { - // Current implementation reads all records and filters them. In certain cases, it many be better to: - // 1. Scan a limited subset of keys (min/max range of candidateRowKeys) - // 2. Lookup keys individually (if the size of candidateRowKeys is much less than the total keys in file) - try { - List> allRecords = readAllRecords(); - Set rowKeys = new HashSet<>(); - allRecords.forEach(t -> { - if (candidateRowKeys.contains(t.getFirst())) { - rowKeys.add(t.getFirst()); - } - }); - return rowKeys; - } catch (IOException e) { - throw new HoodieIOException("Failed to read row keys from " + path, e); + public Set filterRowKeys(Set candidateRowKeys) { + return candidateRowKeys.stream().filter(k -> { + try { + return isKeyAvailable(k); + } catch (IOException e) { + LOG.error("Failed to check key availability: " + k); + return false; + } + }).collect(Collectors.toSet()); + } + + @Override + public Map getRecordsByKeys(List rowKeys) throws IOException { + return filterRecordsImpl(new TreeSet<>(rowKeys)); + } + + /** + * Filter records by sorted keys. + *

+ * TODO: Implement single seek and sequential scan till the last candidate key + * instead of repeated seeks. + * + * @param sortedCandidateRowKeys - Sorted set of keys to fetch records for + * @return Map of keys to fetched records + * @throws IOException When the deserialization of records fail + */ + private synchronized Map filterRecordsImpl(TreeSet sortedCandidateRowKeys) throws IOException { + HashMap filteredRecords = new HashMap<>(); + for (String key : sortedCandidateRowKeys) { + Option record = getRecordByKey(key); + if (record.isPresent()) { + filteredRecords.put(key, record.get()); + } } + return filteredRecords; } public List> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException { @@ -246,6 +278,19 @@ public R next() { }; } + private boolean isKeyAvailable(String key) throws IOException { + final KeyValue kv = new KeyValue(key.getBytes(), null, null, null); + synchronized (this) { + if (keyScanner == null) { + keyScanner = reader.getScanner(false, false); + } + if (keyScanner.seekTo(kv) == 0) { + return true; + } + } + return false; + } + @Override public Option getRecordByKey(String key, Schema readerSchema) throws IOException { byte[] value = null; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index 5373e9860b5e7..459397bdcb8ef 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -19,6 +19,8 @@ package org.apache.hudi.metadata; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -30,7 +32,11 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.common.util.hash.ColumnIndexID; +import org.apache.hudi.common.util.hash.FileIndexID; +import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hadoop.fs.FileStatus; @@ -39,12 +45,15 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.stream.Collectors; public abstract class BaseTableMetadata implements HoodieTableMetadata { @@ -63,7 +72,9 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata { // Directory used for Spillable Map when merging records protected final String spillableMapDirectory; - protected boolean enabled; + protected boolean isMetadataTableEnabled; + protected boolean isBloomFilterIndexEnabled = false; + protected boolean isColumnStatsIndexEnabled = false; protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String dataBasePath, String spillableMapDirectory) { @@ -74,7 +85,7 @@ protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataCon this.spillableMapDirectory = spillableMapDirectory; this.metadataConfig = metadataConfig; - this.enabled = metadataConfig.enabled(); + this.isMetadataTableEnabled = metadataConfig.enabled(); if (metadataConfig.enableMetrics()) { this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata"))); } else { @@ -84,16 +95,15 @@ protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataCon /** * Return the list of partitions in the dataset. - * + *

* If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of * partitions is retrieved directly from the underlying {@code FileSystem}. - * + *

* On any errors retrieving the listing from the metadata, defaults to using the file system listings. - * */ @Override public List getAllPartitionPaths() throws IOException { - if (enabled) { + if (isMetadataTableEnabled) { try { return fetchAllPartitionPaths(); } catch (Exception e) { @@ -106,10 +116,10 @@ public List getAllPartitionPaths() throws IOException { /** * Return the list of files in a partition. - * + *

* If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of * partitions is retrieved directly from the underlying {@code FileSystem}. - * + *

* On any errors retrieving the listing from the metadata, defaults to using the file system listings. * * @param partitionPath The absolute path of the partition to list @@ -117,7 +127,7 @@ public List getAllPartitionPaths() throws IOException { @Override public FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException { - if (enabled) { + if (isMetadataTableEnabled) { try { return fetchAllFilesInPartition(partitionPath); } catch (Exception e) { @@ -132,7 +142,7 @@ public FileStatus[] getAllFilesInPartition(Path partitionPath) @Override public Map getAllFilesInPartitions(List partitions) throws IOException { - if (enabled) { + if (isMetadataTableEnabled) { try { List partitionPaths = partitions.stream().map(entry -> new Path(entry)).collect(Collectors.toList()); Map partitionsFilesMap = fetchAllFilesInPartitionPaths(partitionPaths); @@ -146,12 +156,124 @@ public Map getAllFilesInPartitions(List partitions .getAllFilesInPartitions(partitions); } + @Override + public Option getBloomFilter(final String partitionName, final String fileName) + throws HoodieMetadataException { + if (!isBloomFilterIndexEnabled) { + LOG.error("Metadata bloom filter index is disabled!"); + return Option.empty(); + } + + final Pair partitionFileName = Pair.of(partitionName, fileName); + Map, ByteBuffer> bloomFilters = getBloomFilters(Collections.singletonList(partitionFileName)); + if (bloomFilters.isEmpty()) { + LOG.error("Meta index: missing bloom filter for partition: " + partitionName + ", file: " + fileName); + return Option.empty(); + } + + ValidationUtils.checkState(bloomFilters.containsKey(partitionFileName)); + return Option.of(bloomFilters.get(partitionFileName)); + } + + @Override + public Map, ByteBuffer> getBloomFilters(final List> partitionNameFileNameList) + throws HoodieMetadataException { + if (!isBloomFilterIndexEnabled) { + LOG.error("Metadata bloom filter index is disabled!"); + return Collections.emptyMap(); + } + if (partitionNameFileNameList.isEmpty()) { + return Collections.emptyMap(); + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + Set partitionIDFileIDSortedStrings = new TreeSet<>(); + Map> fileToKeyMap = new HashMap<>(); + partitionNameFileNameList.forEach(partitionNameFileNamePair -> { + final String bloomFilterIndexKey = HoodieMetadataPayload.getBloomFilterIndexKey( + new PartitionIndexID(partitionNameFileNamePair.getLeft()), new FileIndexID(partitionNameFileNamePair.getRight())); + partitionIDFileIDSortedStrings.add(bloomFilterIndexKey); + fileToKeyMap.put(bloomFilterIndexKey, partitionNameFileNamePair); + } + ); + + List partitionIDFileIDStrings = new ArrayList<>(partitionIDFileIDSortedStrings); + List>>> hoodieRecordList = + getRecordsByKeys(partitionIDFileIDStrings, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath()); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_BLOOM_FILTERS_METADATA_STR, + (timer.endTimer() / partitionIDFileIDStrings.size()))); + + Map, ByteBuffer> partitionFileToBloomFilterMap = new HashMap<>(); + for (final Pair>> entry : hoodieRecordList) { + if (entry.getRight().isPresent()) { + final Option bloomFilterMetadata = + entry.getRight().get().getData().getBloomFilterMetadata(); + if (bloomFilterMetadata.isPresent()) { + if (!bloomFilterMetadata.get().getIsDeleted()) { + ValidationUtils.checkState(fileToKeyMap.containsKey(entry.getLeft())); + partitionFileToBloomFilterMap.put(fileToKeyMap.get(entry.getLeft()), bloomFilterMetadata.get().getBloomFilter()); + } + } else { + LOG.error("Meta index bloom filter missing for: " + fileToKeyMap.get(entry.getLeft())); + } + } + } + return partitionFileToBloomFilterMap; + } + + @Override + public Map, HoodieMetadataColumnStats> getColumnStats(final List> partitionNameFileNameList, final String columnName) + throws HoodieMetadataException { + if (!isColumnStatsIndexEnabled) { + LOG.error("Metadata column stats index is disabled!"); + return Collections.emptyMap(); + } + + Map> columnStatKeyToFileNameMap = new HashMap<>(); + TreeSet sortedKeys = new TreeSet<>(); + final ColumnIndexID columnIndexID = new ColumnIndexID(columnName); + for (Pair partitionNameFileNamePair : partitionNameFileNameList) { + final String columnStatsIndexKey = HoodieMetadataPayload.getColumnStatsIndexKey( + new PartitionIndexID(partitionNameFileNamePair.getLeft()), + new FileIndexID(partitionNameFileNamePair.getRight()), + columnIndexID); + sortedKeys.add(columnStatsIndexKey); + columnStatKeyToFileNameMap.put(columnStatsIndexKey, partitionNameFileNamePair); + } + + List columnStatKeys = new ArrayList<>(sortedKeys); + HoodieTimer timer = new HoodieTimer().startTimer(); + List>>> hoodieRecordList = + getRecordsByKeys(columnStatKeys, MetadataPartitionType.COLUMN_STATS.getPartitionPath()); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_COLUMN_STATS_METADATA_STR, timer.endTimer())); + + Map, HoodieMetadataColumnStats> fileToColumnStatMap = new HashMap<>(); + for (final Pair>> entry : hoodieRecordList) { + if (entry.getRight().isPresent()) { + final Option columnStatMetadata = + entry.getRight().get().getData().getColumnStatMetadata(); + if (columnStatMetadata.isPresent()) { + if (!columnStatMetadata.get().getIsDeleted()) { + ValidationUtils.checkState(columnStatKeyToFileNameMap.containsKey(entry.getLeft())); + final Pair partitionFileNamePair = columnStatKeyToFileNameMap.get(entry.getLeft()); + ValidationUtils.checkState(!fileToColumnStatMap.containsKey(partitionFileNamePair)); + fileToColumnStatMap.put(partitionFileNamePair, columnStatMetadata.get()); + } + } else { + LOG.error("Meta index column stats missing for: " + entry.getLeft()); + } + } + } + return fileToColumnStatMap; + } + /** * Returns a list of all partitions. */ protected List fetchAllPartitionPaths() throws IOException { HoodieTimer timer = new HoodieTimer().startTimer(); - Option> hoodieRecord = getRecordByKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath()); + Option> hoodieRecord = getRecordByKey(RECORDKEY_PARTITION_LIST, + MetadataPartitionType.FILES.getPartitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer())); List partitions = Collections.emptyList(); @@ -181,7 +303,8 @@ FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { } HoodieTimer timer = new HoodieTimer().startTimer(); - Option> hoodieRecord = getRecordByKey(partitionName, MetadataPartitionType.FILES.partitionPath()); + Option> hoodieRecord = getRecordByKey(partitionName, + MetadataPartitionType.FILES.getPartitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer())); FileStatus[] statuses = {}; @@ -215,7 +338,7 @@ Map fetchAllFilesInPartitionPaths(List partitionPath HoodieTimer timer = new HoodieTimer().startTimer(); List>>> partitionsFileStatus = - getRecordsByKeys(new ArrayList<>(partitionInfo.keySet()), MetadataPartitionType.FILES.partitionPath()); + getRecordsByKeys(new ArrayList<>(partitionInfo.keySet()), MetadataPartitionType.FILES.getPartitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer())); Map result = new HashMap<>(); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java index d05b95dfdb495..a4e5ea3539f17 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java @@ -18,6 +18,7 @@ package org.apache.hudi.metadata; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; @@ -29,8 +30,10 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.exception.HoodieMetadataException; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -139,4 +142,21 @@ public void close() throws Exception { public void reset() { // no-op } + + public Option getBloomFilter(final String partitionName, final String fileName) + throws HoodieMetadataException { + throw new HoodieMetadataException("Unsupported operation: getBloomFilter for " + fileName); + } + + @Override + public Map, ByteBuffer> getBloomFilters(final List> partitionNameFileNameList) + throws HoodieMetadataException { + throw new HoodieMetadataException("Unsupported operation: getBloomFilters!"); + } + + @Override + public Map, HoodieMetadataColumnStats> getColumnStats(final List> partitionNameFileNameList, final String columnName) + throws HoodieMetadataException { + throw new HoodieMetadataException("Unsupported operation: getColumnsStats!"); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index c9e538f72eaa0..301a3e104c772 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -18,6 +18,9 @@ package org.apache.hudi.metadata; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -48,10 +51,6 @@ import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -64,6 +63,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; /** @@ -80,8 +80,9 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { // should we reuse the open file handles, across calls private final boolean reuse; - // Readers for latest file slice corresponding to file groups in the metadata partition of interest - private Map> partitionReaders = new ConcurrentHashMap<>(); + // Readers for the latest file slice corresponding to file groups in the metadata partition + private Map, Pair> partitionReaders = + new ConcurrentHashMap<>(); public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, String spillableMapDirectory) { @@ -97,7 +98,7 @@ public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetada private void initIfNeeded() { this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(dataBasePath); - if (!enabled) { + if (!isMetadataTableEnabled) { if (!HoodieTableMetadata.isMetadataTable(metadataBasePath)) { LOG.info("Metadata table is disabled."); } @@ -105,14 +106,16 @@ private void initIfNeeded() { try { this.metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build(); this.metadataTableConfig = metadataMetaClient.getTableConfig(); + this.isBloomFilterIndexEnabled = metadataConfig.isBloomFilterIndexEnabled(); + this.isColumnStatsIndexEnabled = metadataConfig.isColumnStatsIndexEnabled(); } catch (TableNotFoundException e) { LOG.warn("Metadata table was not found at path " + metadataBasePath); - this.enabled = false; + this.isMetadataTableEnabled = false; this.metadataMetaClient = null; this.metadataTableConfig = null; } catch (Exception e) { LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e); - this.enabled = false; + this.isMetadataTableEnabled = false; this.metadataMetaClient = null; this.metadataTableConfig = null; } @@ -125,30 +128,43 @@ protected Option> getRecordByKey(String key, return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue(); } - protected List>>> getRecordsByKeys(List keys, String partitionName) { - Pair readers = openReadersIfNeeded(keys.get(0), partitionName); - try { - List timings = new ArrayList<>(); - HoodieFileReader baseFileReader = readers.getKey(); - HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight(); + @Override + protected List>>> getRecordsByKeys(List keys, + String partitionName) { + Map, List> partitionFileSliceToKeysMap = getPartitionFileSliceToKeysMapping(partitionName, keys); + List>>> result = new ArrayList<>(); + AtomicInteger fileSlicesKeysCount = new AtomicInteger(); + partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, fileSliceKeys) -> { + Pair readers = openReadersIfNeeded(partitionName, + partitionFileSlicePair.getRight()); + try { + List timings = new ArrayList<>(); + HoodieFileReader baseFileReader = readers.getKey(); + HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight(); - if (baseFileReader == null && logRecordScanner == null) { - return Collections.emptyList(); - } + if (baseFileReader == null && logRecordScanner == null) { + return; + } - // local map to assist in merging with base file records - Map>> logRecords = readLogRecords(logRecordScanner, keys, timings); - List>>> result = readFromBaseAndMergeWithLogRecords( - baseFileReader, keys, logRecords, timings, partitionName); - LOG.info(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", keys.size(), timings)); - return result; - } catch (IOException ioe) { - throw new HoodieIOException("Error merging records from metadata table for " + keys.size() + " key : ", ioe); - } finally { - if (!reuse) { - close(partitionName); + // local map to assist in merging with base file records + Map>> logRecords = readLogRecords(logRecordScanner, + fileSliceKeys, timings); + result.addAll(readFromBaseAndMergeWithLogRecords(baseFileReader, fileSliceKeys, logRecords, + timings, partitionName)); + LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", + fileSliceKeys.size(), timings)); + fileSlicesKeysCount.addAndGet(fileSliceKeys.size()); + } catch (IOException ioe) { + throw new HoodieIOException("Error merging records from metadata table for " + keys.size() + " key : ", ioe); + } finally { + if (!reuse) { + close(Pair.of(partitionFileSlicePair.getLeft(), partitionFileSlicePair.getRight().getFileId())); + } } - } + }); + + ValidationUtils.checkState(keys.size() == fileSlicesKeysCount.get()); + return result; } private Map>> readLogRecords(HoodieMetadataMergedLogRecordReader logRecordScanner, @@ -190,11 +206,11 @@ private List>>> readFrom // Retrieve record from base file if (baseFileReader != null) { HoodieTimer readTimer = new HoodieTimer(); + Map baseFileRecords = baseFileReader.getRecordsByKeys(keys); for (String key : keys) { readTimer.startTimer(); - Option baseRecord = baseFileReader.getRecordByKey(key); - if (baseRecord.isPresent()) { - hoodieRecord = getRecord(baseRecord, partitionName); + if (baseFileRecords.containsKey(key)) { + hoodieRecord = getRecord(Option.of(baseFileRecords.get(key)), partitionName); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer())); // merge base file record w/ log record if present if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) { @@ -233,38 +249,52 @@ private HoodieRecord getRecord(Option base } /** - * Returns a new pair of readers to the base and log files. + * Get the latest file slices for the interested keys in a given partition. + * + * @param partitionName - Partition to get the file slices from + * @param keys - Interested keys + * @return FileSlices for the keys */ - private Pair openReadersIfNeeded(String key, String partitionName) { - return partitionReaders.computeIfAbsent(partitionName, k -> { - try { - final long baseFileOpenMs; - final long logScannerOpenMs; - HoodieFileReader baseFileReader = null; - HoodieMetadataMergedLogRecordReader logRecordScanner = null; + private Map, List> getPartitionFileSliceToKeysMapping(final String partitionName, final List keys) { + // Metadata is in sync till the latest completed instant on the dataset + List latestFileSlices = + HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName); + + Map, List> partitionFileSliceToKeysMap = new HashMap<>(); + for (String key : keys) { + final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key, + latestFileSlices.size())); + final Pair partitionNameFileSlicePair = Pair.of(partitionName, slice); + partitionFileSliceToKeysMap.computeIfAbsent(partitionNameFileSlicePair, k -> new ArrayList<>()).add(key); + } + return partitionFileSliceToKeysMap; + } - // Metadata is in sync till the latest completed instant on the dataset + /** + * Create a file reader and the record scanner for a given partition and file slice + * if readers are not already available. + * + * @param partitionName - Partition name + * @param slice - The file slice to open readers for + * @return File reader and the record scanner pair for the requested file slice + */ + private Pair openReadersIfNeeded(String partitionName, FileSlice slice) { + return partitionReaders.computeIfAbsent(Pair.of(partitionName, slice.getFileId()), k -> { + try { HoodieTimer timer = new HoodieTimer().startTimer(); - List latestFileSlices = HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName); - if (latestFileSlices.size() == 0) { - // empty partition - return Pair.of(null, null); - } - ValidationUtils.checkArgument(latestFileSlices.size() == 1, String.format("Invalid number of file slices: found=%d, required=%d", latestFileSlices.size(), 1)); - final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key, latestFileSlices.size())); // Open base file reader Pair baseFileReaderOpenTimePair = getBaseFileReader(slice, timer); - baseFileReader = baseFileReaderOpenTimePair.getKey(); - baseFileOpenMs = baseFileReaderOpenTimePair.getValue(); + HoodieFileReader baseFileReader = baseFileReaderOpenTimePair.getKey(); + final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue(); // Open the log record scanner using the log files from the latest file slice - Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice, - partitionName); - logRecordScanner = logRecordScannerOpenTimePair.getKey(); - logScannerOpenMs = logRecordScannerOpenTimePair.getValue(); + Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice, partitionName); + HoodieMetadataMergedLogRecordReader logRecordScanner = logRecordScannerOpenTimePair.getKey(); + final long logScannerOpenMs = logRecordScannerOpenTimePair.getValue(); - metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, baseFileOpenMs + logScannerOpenMs)); + metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, + +baseFileOpenMs + logScannerOpenMs)); return Pair.of(baseFileReader, logRecordScanner); } catch (IOException e) { throw new HoodieIOException("Error opening readers for metadata table partition " + partitionName, e); @@ -382,14 +412,20 @@ private List getRollbackedCommits(HoodieInstant instant, HoodieActiveTim @Override public void close() { - for (String partitionName : partitionReaders.keySet()) { - close(partitionName); + for (Pair partitionFileSlicePair : partitionReaders.keySet()) { + close(partitionFileSlicePair); } partitionReaders.clear(); } - private synchronized void close(String partitionName) { - Pair readers = partitionReaders.remove(partitionName); + /** + * Close the file reader and the record scanner for the given file slice. + * + * @param partitionFileSlicePair - Partition and FileSlice + */ + private synchronized void close(Pair partitionFileSlicePair) { + Pair readers = + partitionReaders.remove(partitionFileSlicePair); if (readers != null) { try { if (readers.getKey() != null) { @@ -405,7 +441,7 @@ private synchronized void close(String partitionName) { } public boolean enabled() { - return enabled; + return isMetadataTableEnabled; } public SerializableConfiguration getHadoopConf() { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java index c03bf40c443f4..d85ee5dff901d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java @@ -116,7 +116,7 @@ public static HoodieMetadataMergedLogRecordReader.Builder newBuilder() { * @param key Key of the record to retrieve * @return {@code HoodieRecord} if key was found else {@code Option.empty()} */ - public List>>> getRecordByKey(String key) { + public synchronized List>>> getRecordByKey(String key) { return Collections.singletonList(Pair.of(key, Option.ofNullable((HoodieRecord) records.get(key)))); } 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 2efc96c6f3dee..fe8612c42e802 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 @@ -41,6 +41,8 @@ public class HoodieMetadataMetrics implements Serializable { // Metric names public static final String LOOKUP_PARTITIONS_STR = "lookup_partitions"; public static final String LOOKUP_FILES_STR = "lookup_files"; + public static final String LOOKUP_BLOOM_FILTERS_METADATA_STR = "lookup_meta_index_bloom_filters"; + public static final String LOOKUP_COLUMN_STATS_METADATA_STR = "lookup_meta_index_column_ranges"; public static final String SCAN_STR = "scan"; public static final String BASEFILE_READ_STR = "basefile_read"; public static final String INITIALIZE_STR = "initialize"; @@ -77,7 +79,7 @@ private Map getStats(HoodieTableFileSystemView fsView, boolean d Map stats = new HashMap<>(); // Total size of the metadata and count of base/log files - for (String metadataPartition : MetadataPartitionType.all()) { + for (String metadataPartition : MetadataPartitionType.allPaths()) { List latestSlices = fsView.getLatestFileSlices(metadataPartition).collect(Collectors.toList()); // Total size of the metadata and count of base/log files diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index a80f33b2d756b..5f10fa081a91b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -18,13 +18,22 @@ package org.apache.hudi.metadata; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; import org.apache.hudi.avro.model.HoodieMetadataFileInfo; import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.hash.ColumnIndexID; +import org.apache.hudi.common.util.hash.FileIndexID; + +import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -36,7 +45,9 @@ import org.apache.hudi.io.storage.HoodieHFileReader; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,36 +57,67 @@ import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST; /** - * This is a payload which saves information about a single entry in the Metadata Table. - * - * The type of the entry is determined by the "type" saved within the record. The following types of entries are saved: - * - * 1. List of partitions: There is a single such record - * key="__all_partitions__" - * - * 2. List of files in a Partition: There is one such record for each partition - * key=Partition name - * - * During compaction on the table, the deletions are merged with additions and hence pruned. - * - * Metadata Table records are saved with the schema defined in HoodieMetadata.avsc. This class encapsulates the - * HoodieMetadataRecord for ease of operations. + * MetadataTable records are persisted with the schema defined in HoodieMetadata.avsc. + * This class represents the payload for the MetadataTable. + *

+ * This single metadata payload is shared by all the partitions under the metadata table. + * The partition specific records are determined by the field "type" saved within the record. + * The following types are supported: + *

+ * METADATA_TYPE_PARTITION_LIST (1): + * -- List of all partitions. There is a single such record + * -- key = @{@link HoodieTableMetadata.RECORDKEY_PARTITION_LIST} + *

+ * METADATA_TYPE_FILE_LIST (2): + * -- List of all files in a partition. There is one such record for each partition + * -- key = partition name + *

+ * METADATA_TYPE_COLUMN_STATS (3): + * -- This is an index for column stats in the table + *

+ * METADATA_TYPE_BLOOM_FILTER (4): + * -- This is an index for base file bloom filters. This is a map of FileID to its BloomFilter byte[]. + *

+ * During compaction on the table, the deletions are merged with additions and hence records are pruned. */ public class HoodieMetadataPayload implements HoodieRecordPayload { + // Type of the record. This can be an enum in the schema but Avro1.8 + // has a bug - https://issues.apache.org/jira/browse/AVRO-1810 + protected static final int METADATA_TYPE_PARTITION_LIST = 1; + protected static final int METADATA_TYPE_FILE_LIST = 2; + protected static final int METADATA_TYPE_COLUMN_STATS = 3; + protected static final int METADATA_TYPE_BLOOM_FILTER = 4; + // HoodieMetadata schema field ids public static final String KEY_FIELD_NAME = HoodieHFileReader.KEY_FIELD_NAME; public static final String SCHEMA_FIELD_NAME_TYPE = "type"; public static final String SCHEMA_FIELD_NAME_METADATA = "filesystemMetadata"; + private static final String SCHEMA_FIELD_ID_COLUMN_STATS = "ColumnStatsMetadata"; + private static final String SCHEMA_FIELD_ID_BLOOM_FILTER = "BloomFilterMetadata"; - // Type of the record - // This can be an enum in the schema but Avro 1.8 has a bug - https://issues.apache.org/jira/browse/AVRO-1810 - private static final int PARTITION_LIST = 1; - private static final int FILE_LIST = 2; + // HoodieMetadata bloom filter payload field ids + private static final String FIELD_IS_DELETED = "isDeleted"; + private static final String BLOOM_FILTER_FIELD_TYPE = "type"; + private static final String BLOOM_FILTER_FIELD_TIMESTAMP = "timestamp"; + private static final String BLOOM_FILTER_FIELD_BLOOM_FILTER = "bloomFilter"; + private static final String BLOOM_FILTER_FIELD_IS_DELETED = FIELD_IS_DELETED; + + // HoodieMetadata column stats payload field ids + private static final String COLUMN_STATS_FIELD_MIN_VALUE = "minValue"; + private static final String COLUMN_STATS_FIELD_MAX_VALUE = "maxValue"; + private static final String COLUMN_STATS_FIELD_NULL_COUNT = "nullCount"; + private static final String COLUMN_STATS_FIELD_VALUE_COUNT = "valueCount"; + private static final String COLUMN_STATS_FIELD_TOTAL_SIZE = "totalSize"; + private static final String COLUMN_STATS_FIELD_RESOURCE_NAME = "fileName"; + private static final String COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE = "totalUncompressedSize"; + private static final String COLUMN_STATS_FIELD_IS_DELETED = FIELD_IS_DELETED; private String key = null; private int type = 0; private Map filesystemMetadata = null; + private HoodieMetadataBloomFilter bloomFilterMetadata = null; + private HoodieMetadataColumnStats columnStatMetadata = null; public HoodieMetadataPayload(GenericRecord record, Comparable orderingVal) { this(Option.of(record)); @@ -94,13 +136,60 @@ public HoodieMetadataPayload(Option record) { filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted"))); }); } + + if (type == METADATA_TYPE_BLOOM_FILTER) { + final GenericRecord metadataRecord = (GenericRecord) record.get().get(SCHEMA_FIELD_ID_BLOOM_FILTER); + if (metadataRecord == null) { + throw new HoodieMetadataException("Valid " + SCHEMA_FIELD_ID_BLOOM_FILTER + " record expected for type: " + METADATA_TYPE_BLOOM_FILTER); + } + bloomFilterMetadata = new HoodieMetadataBloomFilter( + (String) metadataRecord.get(BLOOM_FILTER_FIELD_TYPE), + (String) metadataRecord.get(BLOOM_FILTER_FIELD_TIMESTAMP), + (ByteBuffer) metadataRecord.get(BLOOM_FILTER_FIELD_BLOOM_FILTER), + (Boolean) metadataRecord.get(BLOOM_FILTER_FIELD_IS_DELETED) + ); + } + + if (type == METADATA_TYPE_COLUMN_STATS) { + GenericRecord v = (GenericRecord) record.get().get(SCHEMA_FIELD_ID_COLUMN_STATS); + if (v == null) { + throw new HoodieMetadataException("Valid " + SCHEMA_FIELD_ID_COLUMN_STATS + " record expected for type: " + METADATA_TYPE_COLUMN_STATS); + } + columnStatMetadata = new HoodieMetadataColumnStats( + (String) v.get(COLUMN_STATS_FIELD_RESOURCE_NAME), + (String) v.get(COLUMN_STATS_FIELD_MIN_VALUE), + (String) v.get(COLUMN_STATS_FIELD_MAX_VALUE), + (Long) v.get(COLUMN_STATS_FIELD_NULL_COUNT), + (Long) v.get(COLUMN_STATS_FIELD_VALUE_COUNT), + (Long) v.get(COLUMN_STATS_FIELD_TOTAL_SIZE), + (Long) v.get(COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE), + (Boolean) v.get(COLUMN_STATS_FIELD_IS_DELETED) + ); + } } } private HoodieMetadataPayload(String key, int type, Map filesystemMetadata) { + this(key, type, filesystemMetadata, null, null); + } + + private HoodieMetadataPayload(String key, int type, HoodieMetadataBloomFilter metadataBloomFilter) { + this(key, type, null, metadataBloomFilter, null); + } + + private HoodieMetadataPayload(String key, int type, HoodieMetadataColumnStats columnStats) { + this(key, type, null, null, columnStats); + } + + protected HoodieMetadataPayload(String key, int type, + Map filesystemMetadata, + HoodieMetadataBloomFilter metadataBloomFilter, + HoodieMetadataColumnStats columnStats) { this.key = key; this.type = type; this.filesystemMetadata = filesystemMetadata; + this.bloomFilterMetadata = metadataBloomFilter; + this.columnStatMetadata = columnStats; } /** @@ -110,55 +199,97 @@ private HoodieMetadataPayload(String key, int type, Map createPartitionListRecord(List partitions) { Map fileInfo = new HashMap<>(); - partitions.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, false))); + partitions.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, false))); - HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath()); - HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), PARTITION_LIST, fileInfo); + HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.getPartitionPath()); + HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_PARTITION_LIST, + fileInfo); return new HoodieRecord<>(key, payload); } /** * Create and return a {@code HoodieMetadataPayload} to save list of files within a partition. * - * @param partition The name of the partition - * @param filesAdded Mapping of files to their sizes for files which have been added to this partition + * @param partition The name of the partition + * @param filesAdded Mapping of files to their sizes for files which have been added to this partition * @param filesDeleted List of files which have been deleted from this partition */ public static HoodieRecord createPartitionFilesRecord(String partition, - Option> filesAdded, Option> filesDeleted) { + Option> filesAdded, + Option> filesDeleted) { Map fileInfo = new HashMap<>(); filesAdded.ifPresent( m -> m.forEach((filename, size) -> fileInfo.put(filename, new HoodieMetadataFileInfo(size, false)))); filesDeleted.ifPresent( - m -> m.forEach(filename -> fileInfo.put(filename, new HoodieMetadataFileInfo(0L, true)))); + m -> m.forEach(filename -> fileInfo.put(filename, new HoodieMetadataFileInfo(0L, true)))); - HoodieKey key = new HoodieKey(partition, MetadataPartitionType.FILES.partitionPath()); - HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), FILE_LIST, fileInfo); + HoodieKey key = new HoodieKey(partition, MetadataPartitionType.FILES.getPartitionPath()); + HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_FILE_LIST, fileInfo); return new HoodieRecord<>(key, payload); } + /** + * Create bloom filter metadata record. + * + * @param partitionName - Partition name + * @param baseFileName - Base file name for which the bloom filter needs to persisted + * @param timestamp - Instant timestamp responsible for this record + * @param bloomFilter - Bloom filter for the File + * @param isDeleted - Is the bloom filter no more valid + * @return Metadata payload containing the fileID and its bloom filter record + */ + public static HoodieRecord createBloomFilterMetadataRecord(final String partitionName, + final String baseFileName, + final String timestamp, + final ByteBuffer bloomFilter, + final boolean isDeleted) { + ValidationUtils.checkArgument(!baseFileName.contains(Path.SEPARATOR) + && FSUtils.isBaseFile(new Path(baseFileName)), + "Invalid base file '" + baseFileName + "' for MetaIndexBloomFilter!"); + final String bloomFilterIndexKey = new PartitionIndexID(partitionName).asBase64EncodedString() + .concat(new FileIndexID(baseFileName).asBase64EncodedString()); + HoodieKey key = new HoodieKey(bloomFilterIndexKey, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath()); + + // TODO: HUDI-3203 Get the bloom filter type from the file + HoodieMetadataBloomFilter metadataBloomFilter = + new HoodieMetadataBloomFilter(BloomFilterTypeCode.DYNAMIC_V0.name(), + timestamp, bloomFilter, isDeleted); + HoodieMetadataPayload metadataPayload = new HoodieMetadataPayload(key.getRecordKey(), + HoodieMetadataPayload.METADATA_TYPE_BLOOM_FILTER, metadataBloomFilter); + return new HoodieRecord<>(key, metadataPayload); + } + @Override public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { ValidationUtils.checkArgument(previousRecord.type == type, - "Cannot combine " + previousRecord.type + " with " + type); - - Map combinedFileInfo = null; + "Cannot combine " + previousRecord.type + " with " + type); switch (type) { - case PARTITION_LIST: - case FILE_LIST: - combinedFileInfo = combineFilesystemMetadata(previousRecord); - break; + case METADATA_TYPE_PARTITION_LIST: + case METADATA_TYPE_FILE_LIST: + Map combinedFileInfo = combineFilesystemMetadata(previousRecord); + return new HoodieMetadataPayload(key, type, combinedFileInfo); + case METADATA_TYPE_BLOOM_FILTER: + HoodieMetadataBloomFilter combineBloomFilterMetadata = combineBloomFilterMetadata(previousRecord); + return new HoodieMetadataPayload(key, type, combineBloomFilterMetadata); + case METADATA_TYPE_COLUMN_STATS: + return new HoodieMetadataPayload(key, type, combineColumnStatsMetadatat(previousRecord)); default: throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type); } + } + + private HoodieMetadataBloomFilter combineBloomFilterMetadata(HoodieMetadataPayload previousRecord) { + return this.bloomFilterMetadata; + } - return new HoodieMetadataPayload(key, type, combinedFileInfo); + private HoodieMetadataColumnStats combineColumnStatsMetadatat(HoodieMetadataPayload previousRecord) { + return this.columnStatMetadata; } @Override public Option combineAndGetUpdateValue(IndexedRecord oldRecord, Schema schema) throws IOException { - HoodieMetadataPayload anotherPayload = new HoodieMetadataPayload(Option.of((GenericRecord)oldRecord)); + HoodieMetadataPayload anotherPayload = new HoodieMetadataPayload(Option.of((GenericRecord) oldRecord)); HoodieRecordPayload combinedPayload = preCombine(anotherPayload); return combinedPayload.getInsertValue(schema); } @@ -169,7 +300,8 @@ public Option getInsertValue(Schema schema) throws IOException { return Option.empty(); } - HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata); + HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata, bloomFilterMetadata, + columnStatMetadata); return Option.of(record); } @@ -187,6 +319,28 @@ public List getDeletions() { return filterFileInfoEntries(true).map(Map.Entry::getKey).sorted().collect(Collectors.toList()); } + /** + * Get the bloom filter metadata from this payload. + */ + public Option getBloomFilterMetadata() { + if (bloomFilterMetadata == null) { + return Option.empty(); + } + + return Option.of(bloomFilterMetadata); + } + + /** + * Get the bloom filter metadata from this payload. + */ + public Option getColumnStatMetadata() { + if (columnStatMetadata == null) { + return Option.empty(); + } + + return Option.of(columnStatMetadata); + } + /** * Returns the files added as part of this record. */ @@ -235,6 +389,70 @@ private Map combineFilesystemMetadata(HoodieMeta return combinedFileInfo; } + /** + * Get bloom filter index key. + * + * @param partitionIndexID - Partition index id + * @param fileIndexID - File index id + * @return Bloom filter index key + */ + public static String getBloomFilterIndexKey(PartitionIndexID partitionIndexID, FileIndexID fileIndexID) { + return partitionIndexID.asBase64EncodedString() + .concat(fileIndexID.asBase64EncodedString()); + } + + /** + * Get column stats index key. + * + * @param partitionIndexID - Partition index id + * @param fileIndexID - File index id + * @param columnIndexID - Column index id + * @return Column stats index key + */ + public static String getColumnStatsIndexKey(PartitionIndexID partitionIndexID, FileIndexID fileIndexID, ColumnIndexID columnIndexID) { + return columnIndexID.asBase64EncodedString() + .concat(partitionIndexID.asBase64EncodedString()) + .concat(fileIndexID.asBase64EncodedString()); + } + + /** + * Get column stats index key from the column range metadata. + * + * @param partitionName - Partition name + * @param columnRangeMetadata - Column range metadata + * @return Column stats index key + */ + public static String getColumnStatsIndexKey(String partitionName, HoodieColumnRangeMetadata columnRangeMetadata) { + final PartitionIndexID partitionIndexID = new PartitionIndexID(partitionName); + final FileIndexID fileIndexID = new FileIndexID(new Path(columnRangeMetadata.getFilePath()).getName()); + final ColumnIndexID columnIndexID = new ColumnIndexID(columnRangeMetadata.getColumnName()); + return getColumnStatsIndexKey(partitionIndexID, fileIndexID, columnIndexID); + } + + public static Stream createColumnStatsRecords( + String partitionName, Collection> columnRangeMetadataList, boolean isDeleted) { + return columnRangeMetadataList.stream().map(columnRangeMetadata -> { + HoodieKey key = new HoodieKey(getColumnStatsIndexKey(partitionName, columnRangeMetadata), + MetadataPartitionType.COLUMN_STATS.getPartitionPath()); + HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_COLUMN_STATS, + HoodieMetadataColumnStats.newBuilder() + .setFileName(new Path(columnRangeMetadata.getFilePath()).getName()) + .setMinValue(columnRangeMetadata.getMinValue() == null ? null : + columnRangeMetadata.getMinValue().toString()) + .setMaxValue(columnRangeMetadata.getMaxValue() == null ? null : + columnRangeMetadata.getMaxValue().toString()) + .setNullCount(columnRangeMetadata.getNullCount()) + .setValueCount(columnRangeMetadata.getValueCount()) + .setTotalSize(columnRangeMetadata.getTotalSize()) + .setTotalUncompressedSize(columnRangeMetadata.getTotalUncompressedSize()) + .setIsDeleted(isDeleted) + .build()); + return new HoodieRecord<>(key, payload); + }); + + + } + @Override public String toString() { final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {"); @@ -242,6 +460,20 @@ public String toString() { sb.append(SCHEMA_FIELD_NAME_TYPE + "=").append(type).append(", "); sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); + if (type == METADATA_TYPE_BLOOM_FILTER) { + ValidationUtils.checkState(getBloomFilterMetadata().isPresent()); + sb.append("BloomFilter: {"); + sb.append("bloom size: " + getBloomFilterMetadata().get().getBloomFilter().array().length).append(", "); + sb.append("timestamp: " + getBloomFilterMetadata().get().getTimestamp()).append(", "); + sb.append("deleted: " + getBloomFilterMetadata().get().getIsDeleted()); + sb.append("}"); + } + if (type == METADATA_TYPE_COLUMN_STATS) { + ValidationUtils.checkState(getColumnStatMetadata().isPresent()); + sb.append("ColStats: {"); + sb.append(getColumnStatMetadata().get()); + sb.append("}"); + } sb.append('}'); return sb.toString(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java index d981b7085195b..6a5df050a0b29 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -18,6 +18,7 @@ package org.apache.hudi.metadata; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -25,9 +26,12 @@ import org.apache.hudi.common.util.Option; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieMetadataException; import java.io.IOException; import java.io.Serializable; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -104,6 +108,38 @@ static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetad */ Map getAllFilesInPartitions(List partitionPaths) throws IOException; + /** + * Get the bloom filter for the FileID from the metadata table. + * + * @param partitionName - Partition name + * @param fileName - File name for which bloom filter needs to be retrieved + * @return BloomFilter byte buffer if available, otherwise empty + * @throws HoodieMetadataException + */ + Option getBloomFilter(final String partitionName, final String fileName) + throws HoodieMetadataException; + + /** + * Get bloom filters for files from the metadata table index. + * + * @param partitionNameFileNameList - List of partition and file name pair for which bloom filters need to be retrieved + * @return Map of partition file name pair to its bloom filter byte buffer + * @throws HoodieMetadataException + */ + Map, ByteBuffer> getBloomFilters(final List> partitionNameFileNameList) + throws HoodieMetadataException; + + /** + * Get column stats for files from the metadata table index. + * + * @param partitionNameFileNameList - List of partition and file name pair for which bloom filters need to be retrieved + * @param columnName - Column name for which stats are needed + * @return Map of partition and file name pair to its column stats + * @throws HoodieMetadataException + */ + Map, HoodieMetadataColumnStats> getColumnStats(final List> partitionNameFileNameList, final String columnName) + throws HoodieMetadataException; + /** * Get the instant time to which the metadata is synced w.r.t data timeline. */ 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 bb29e4236da0b..5fd59bc932872 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 @@ -18,29 +18,44 @@ package org.apache.hudi.metadata; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.bloom.BloomFilter; +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.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieDeltaWriteStat; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ParquetUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieMetadataException; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -62,12 +77,17 @@ public class HoodieTableMetadataUtil { private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class); + protected static final String PARTITION_NAME_FILES = "files"; + protected static final String PARTITION_NAME_COLUMN_STATS = "column_stats"; + protected static final String PARTITION_NAME_BLOOM_FILTERS = "bloom_filters"; + /** - * Delete the metadata table for the dataset. This will be invoked during upgrade/downgrade operation during which no other + * Delete the metadata table for the dataset. This will be invoked during upgrade/downgrade operation during which + * no other * process should be running. * * @param basePath base path of the dataset - * @param context instance of {@link HoodieEngineContext}. + * @param context instance of {@link HoodieEngineContext}. */ public static void deleteMetadataTable(String basePath, HoodieEngineContext context) { final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); @@ -79,14 +99,53 @@ public static void deleteMetadataTable(String basePath, HoodieEngineContext cont } } + /** + * Convert commit action to metadata records for the enabled partition types. + * + * @param commitMetadata - Commit action metadata + * @param dataMetaClient - Meta client for the data table + * @param isMetaIndexColumnStatsForAllColumns - Do all columns need meta indexing? + * @param instantTime - Action instant time + * @return Map of partition to metadata records for the commit action + */ + public static Map> convertMetadataToRecords( + HoodieEngineContext context, List enabledPartitionTypes, + HoodieCommitMetadata commitMetadata, HoodieTableMetaClient dataMetaClient, + boolean isMetaIndexColumnStatsForAllColumns, String instantTime) { + final Map> partitionToRecordsMap = new HashMap<>(); + final HoodieData filesPartitionRecordsRDD = context.parallelize( + convertMetadataToFilesPartitionRecords(commitMetadata, instantTime), 1); + partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); + + if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { + final List metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(commitMetadata, + dataMetaClient, instantTime); + if (!metadataBloomFilterRecords.isEmpty()) { + final HoodieData metadataBloomFilterRecordsRDD = context.parallelize(metadataBloomFilterRecords, 1); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); + } + } + + if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { + final List metadataColumnStats = convertMetadataToColumnStatsRecords(commitMetadata, context, + dataMetaClient, isMetaIndexColumnStatsForAllColumns, instantTime); + if (!metadataColumnStats.isEmpty()) { + final HoodieData metadataColumnStatsRDD = context.parallelize(metadataColumnStats, 1); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); + } + } + return partitionToRecordsMap; + } + /** * Finds all new files/partitions created as part of commit and creates metadata table records for them. * - * @param commitMetadata - * @param instantTime - * @return a list of metadata table records + * @param commitMetadata - Commit action metadata + * @param instantTime - Commit action instant time + * @return List of metadata table records */ - public static List convertMetadataToRecords(HoodieCommitMetadata commitMetadata, String instantTime) { + public static List convertMetadataToFilesPartitionRecords(HoodieCommitMetadata commitMetadata, + String instantTime) { List records = new LinkedList<>(); List allPartitions = new LinkedList<>(); commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { @@ -124,6 +183,102 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c return records; } + /** + * Convert commit action metadata to bloom filter records. + * + * @param commitMetadata - Commit action metadata + * @param dataMetaClient - Meta client for the data table + * @param instantTime - Action instant time + * @return List of metadata table records + */ + public static List convertMetadataToBloomFilterRecords(HoodieCommitMetadata commitMetadata, + HoodieTableMetaClient dataMetaClient, + String instantTime) { + List records = new LinkedList<>(); + commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { + final String partition = partitionStatName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionStatName; + Map newFiles = new HashMap<>(writeStats.size()); + writeStats.forEach(hoodieWriteStat -> { + // No action for delta logs + if (hoodieWriteStat instanceof HoodieDeltaWriteStat) { + return; + } + + String pathWithPartition = hoodieWriteStat.getPath(); + if (pathWithPartition == null) { + // Empty partition + LOG.error("Failed to find path in write stat to update metadata table " + hoodieWriteStat); + return; + } + int offset = partition.equals(NON_PARTITIONED_NAME) ? (pathWithPartition.startsWith("/") ? 1 : 0) : + partition.length() + 1; + + final String fileName = pathWithPartition.substring(offset); + if (!FSUtils.isBaseFile(new Path(fileName))) { + return; + } + ValidationUtils.checkState(!newFiles.containsKey(fileName), "Duplicate files in HoodieCommitMetadata"); + + final Path writeFilePath = new Path(dataMetaClient.getBasePath(), pathWithPartition); + try { + HoodieFileReader fileReader = + HoodieFileReaderFactory.getFileReader(dataMetaClient.getHadoopConf(), writeFilePath); + try { + final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); + if (fileBloomFilter == null) { + LOG.error("Failed to read bloom filter for " + writeFilePath); + return; + } + ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); + HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( + partition, fileName, instantTime, bloomByteBuffer, false); + records.add(record); + } catch (Exception e) { + LOG.error("Failed to read bloom filter for " + writeFilePath); + return; + } + fileReader.close(); + } catch (IOException e) { + LOG.error("Failed to get bloom filter for file: " + writeFilePath + ", write stat: " + hoodieWriteStat); + } + }); + }); + + return records; + } + + /** + * Convert the clean action to metadata records. + */ + public static Map> convertMetadataToRecords( + HoodieEngineContext engineContext, List enabledPartitionTypes, + HoodieCleanMetadata cleanMetadata, HoodieTableMetaClient dataMetaClient, String instantTime) { + final Map> partitionToRecordsMap = new HashMap<>(); + final HoodieData filesPartitionRecordsRDD = engineContext.parallelize( + convertMetadataToFilesPartitionRecords(cleanMetadata, instantTime), 1); + partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); + + if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { + final List metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(cleanMetadata, + engineContext, instantTime); + if (!metadataBloomFilterRecords.isEmpty()) { + final HoodieData metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); + } + } + + if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { + final List metadataColumnStats = convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, + dataMetaClient); + if (!metadataColumnStats.isEmpty()) { + final HoodieData metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); + } + } + + return partitionToRecordsMap; + } + /** * Finds all files that were deleted as part of a clean and creates metadata table records for them. * @@ -131,7 +286,8 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c * @param instantTime * @return a list of metadata table records */ - public static List convertMetadataToRecords(HoodieCleanMetadata cleanMetadata, String instantTime) { + public static List convertMetadataToFilesPartitionRecords(HoodieCleanMetadata cleanMetadata, + String instantTime) { List records = new LinkedList<>(); int[] fileDeleteCount = {0}; cleanMetadata.getPartitionMetadata().forEach((partitionName, partitionMetadata) -> { @@ -150,48 +306,187 @@ public static List convertMetadataToRecords(HoodieCleanMetadata cl return records; } + /** + * Convert clean metadata to bloom filter index records. + * + * @param cleanMetadata - Clean action metadata + * @param engineContext - Engine context + * @param instantTime - Clean action instant time + * @return List of bloom filter index records for the clean metadata + */ + public static List convertMetadataToBloomFilterRecords(HoodieCleanMetadata cleanMetadata, + HoodieEngineContext engineContext, + String instantTime) { + List> deleteFileList = new ArrayList<>(); + cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { + // Files deleted from a partition + List deletedFiles = partitionMetadata.getDeletePathPatterns(); + deletedFiles.forEach(entry -> { + final Path deletedFilePath = new Path(entry); + if (FSUtils.isBaseFile(deletedFilePath)) { + deleteFileList.add(Pair.of(partition, deletedFilePath.getName())); + } + }); + }); + + return engineContext.map(deleteFileList, deleteFileInfo -> { + return HoodieMetadataPayload.createBloomFilterMetadataRecord( + deleteFileInfo.getLeft(), deleteFileInfo.getRight(), instantTime, ByteBuffer.allocate(0), true); + }, 1).stream().collect(Collectors.toList()); + } + + /** + * Convert clean metadata to column stats index records. + * + * @param cleanMetadata - Clean action metadata + * @param engineContext - Engine context + * @param datasetMetaClient - data table meta client + * @return List of column stats index records for the clean metadata + */ + public static List convertMetadataToColumnStatsRecords(HoodieCleanMetadata cleanMetadata, + HoodieEngineContext engineContext, + HoodieTableMetaClient datasetMetaClient) { + List> deleteFileList = new ArrayList<>(); + cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { + // Files deleted from a partition + List deletedFiles = partitionMetadata.getDeletePathPatterns(); + deletedFiles.forEach(entry -> deleteFileList.add(Pair.of(partition, entry))); + }); + + List latestColumns = getLatestColumns(datasetMetaClient); + return engineContext.flatMap(deleteFileList, + deleteFileInfo -> { + if (deleteFileInfo.getRight().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { + return getColumnStats(deleteFileInfo.getKey(), deleteFileInfo.getValue(), datasetMetaClient, + latestColumns, true); + } + return Stream.empty(); + }, 1).stream().collect(Collectors.toList()); + } + + /** + * Convert restore action metadata to metadata table records. + */ + public static Map> convertMetadataToRecords( + HoodieEngineContext engineContext, List enabledPartitionTypes, + HoodieActiveTimeline metadataTableTimeline, HoodieRestoreMetadata restoreMetadata, + HoodieTableMetaClient dataMetaClient, String instantTime, Option lastSyncTs) { + final Map> partitionToRecordsMap = new HashMap<>(); + final Map> partitionToAppendedFiles = new HashMap<>(); + final Map> partitionToDeletedFiles = new HashMap<>(); + + processRestoreMetadata(metadataTableTimeline, restoreMetadata, + partitionToAppendedFiles, partitionToDeletedFiles, lastSyncTs); + + final HoodieData filesPartitionRecordsRDD = engineContext.parallelize( + convertFilesToFilesPartitionRecords(partitionToDeletedFiles, + partitionToAppendedFiles, instantTime, "Restore"), 1); + partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); + + if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { + final List metadataBloomFilterRecords = convertFilesToBloomFilterRecords( + engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); + if (!metadataBloomFilterRecords.isEmpty()) { + final HoodieData metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); + } + } + + if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { + final List metadataColumnStats = convertFilesToColumnStatsRecords( + engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); + if (!metadataColumnStats.isEmpty()) { + final HoodieData metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); + } + } + + return partitionToRecordsMap; + } + /** * Aggregates all files deleted and appended to from all rollbacks associated with a restore operation then * creates metadata table records for them. * - * @param restoreMetadata - * @param instantTime + * @param restoreMetadata - Restore action metadata * @return a list of metadata table records */ - public static List convertMetadataToRecords(HoodieActiveTimeline metadataTableTimeline, - HoodieRestoreMetadata restoreMetadata, String instantTime, Option lastSyncTs) { - Map> partitionToAppendedFiles = new HashMap<>(); - Map> partitionToDeletedFiles = new HashMap<>(); + private static void processRestoreMetadata(HoodieActiveTimeline metadataTableTimeline, + HoodieRestoreMetadata restoreMetadata, + Map> partitionToAppendedFiles, + Map> partitionToDeletedFiles, + Option lastSyncTs) { restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> { - rms.forEach(rm -> processRollbackMetadata(metadataTableTimeline, rm, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs)); + rms.forEach(rm -> processRollbackMetadata(metadataTableTimeline, rm, + partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs)); }); - - return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore"); } - public static List convertMetadataToRecords(HoodieActiveTimeline metadataTableTimeline, - HoodieRollbackMetadata rollbackMetadata, String instantTime, - Option lastSyncTs, boolean wasSynced) { + /** + * Convert rollback action metadata to metadata table records. + */ + public static Map> convertMetadataToRecords( + HoodieEngineContext engineContext, List enabledPartitionTypes, + HoodieActiveTimeline metadataTableTimeline, HoodieRollbackMetadata rollbackMetadata, + HoodieTableMetaClient dataMetaClient, String instantTime, Option lastSyncTs, boolean wasSynced) { + final Map> partitionToRecordsMap = new HashMap<>(); - Map> partitionToAppendedFiles = new HashMap<>(); Map> partitionToDeletedFiles = new HashMap<>(); - processRollbackMetadata(metadataTableTimeline, rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs); + Map> partitionToAppendedFiles = new HashMap<>(); + List filesPartitionRecords = convertMetadataToRollbackRecords(metadataTableTimeline, rollbackMetadata, + partitionToDeletedFiles, partitionToAppendedFiles, instantTime, lastSyncTs, wasSynced); + final HoodieData rollbackRecordsRDD = engineContext.parallelize(filesPartitionRecords, 1); + partitionToRecordsMap.put(MetadataPartitionType.FILES, rollbackRecordsRDD); + + if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { + final List metadataBloomFilterRecords = convertFilesToBloomFilterRecords( + engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); + if (!metadataBloomFilterRecords.isEmpty()) { + final HoodieData metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); + } + } + + if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { + final List metadataColumnStats = convertFilesToColumnStatsRecords( + engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); + if (!metadataColumnStats.isEmpty()) { + final HoodieData metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); + } + } + + return partitionToRecordsMap; + } + + /** + * Convert rollback action metadata to files partition records. + */ + private static List convertMetadataToRollbackRecords(HoodieActiveTimeline metadataTableTimeline, + HoodieRollbackMetadata rollbackMetadata, + Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, + String instantTime, + Option lastSyncTs, boolean wasSynced) { + processRollbackMetadata(metadataTableTimeline, rollbackMetadata, partitionToDeletedFiles, + partitionToAppendedFiles, lastSyncTs); if (!wasSynced) { // Since the instant-being-rolled-back was never committed to the metadata table, the files added there // need not be deleted. For MOR Table, the rollback appends logBlocks so we need to keep the appended files. partitionToDeletedFiles.clear(); } - return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Rollback"); + return convertFilesToFilesPartitionRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Rollback"); } /** * Extracts information about the deleted and append files from the {@code HoodieRollbackMetadata}. - * + *

* During a rollback files may be deleted (COW, MOR) or rollback blocks be appended (MOR only) to files. This * function will extract this change file for each partition. - * @param metadataTableTimeline Current timeline of the Metdata Table - * @param rollbackMetadata {@code HoodieRollbackMetadata} - * @param partitionToDeletedFiles The {@code Map} to fill with files deleted per partition. + * + * @param metadataTableTimeline Current timeline of the Metadata Table + * @param rollbackMetadata {@code HoodieRollbackMetadata} + * @param partitionToDeletedFiles The {@code Map} to fill with files deleted per partition. * @param partitionToAppendedFiles The {@code Map} to fill with files appended per partition and their sizes. */ private static void processRollbackMetadata(HoodieActiveTimeline metadataTableTimeline, @@ -268,9 +563,12 @@ private static void processRollbackMetadata(HoodieActiveTimeline metadataTableTi }); } - private static List convertFilesToRecords(Map> partitionToDeletedFiles, - Map> partitionToAppendedFiles, String instantTime, - String operation) { + /** + * Convert rollback action metadata to files partition records. + */ + private static List convertFilesToFilesPartitionRecords(Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, + String instantTime, String operation) { List records = new LinkedList<>(); int[] fileChangeCount = {0, 0}; // deletes, appends @@ -309,9 +607,88 @@ private static List convertFilesToRecords(Map return records; } + /** + * Convert rollback action metadata to bloom filter index records. + */ + private static List convertFilesToBloomFilterRecords(HoodieEngineContext engineContext, + HoodieTableMetaClient dataMetaClient, + Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, + String instantTime) { + List records = new LinkedList<>(); + partitionToDeletedFiles.forEach((partitionName, deletedFileList) -> deletedFileList.forEach(deletedFile -> { + if (!FSUtils.isBaseFile(new Path(deletedFile))) { + return; + } + + final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; + records.add(HoodieMetadataPayload.createBloomFilterMetadataRecord( + partition, deletedFile, instantTime, ByteBuffer.allocate(0), true)); + })); + + partitionToAppendedFiles.forEach((partitionName, appendedFileMap) -> { + final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; + appendedFileMap.forEach((appendedFile, length) -> { + if (!FSUtils.isBaseFile(new Path(appendedFile))) { + return; + } + final String pathWithPartition = partitionName + "/" + appendedFile; + final Path appendedFilePath = new Path(dataMetaClient.getBasePath(), pathWithPartition); + try { + HoodieFileReader fileReader = + HoodieFileReaderFactory.getFileReader(dataMetaClient.getHadoopConf(), appendedFilePath); + final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); + if (fileBloomFilter == null) { + LOG.error("Failed to read bloom filter for " + appendedFilePath); + return; + } + ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); + HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( + partition, appendedFile, instantTime, bloomByteBuffer, false); + records.add(record); + fileReader.close(); + } catch (IOException e) { + LOG.error("Failed to get bloom filter for file: " + appendedFilePath); + } + }); + }); + return records; + } + + /** + * Convert rollback action metadata to column stats index records. + */ + private static List convertFilesToColumnStatsRecords(HoodieEngineContext engineContext, + HoodieTableMetaClient datasetMetaClient, + Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, + String instantTime) { + List records = new LinkedList<>(); + List latestColumns = getLatestColumns(datasetMetaClient); + partitionToDeletedFiles.forEach((partitionName, deletedFileList) -> deletedFileList.forEach(deletedFile -> { + final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; + if (deletedFile.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { + final String filePathWithPartition = partitionName + "/" + deletedFile; + records.addAll(getColumnStats(partition, filePathWithPartition, datasetMetaClient, + latestColumns, true).collect(Collectors.toList())); + } + })); + + partitionToAppendedFiles.forEach((partitionName, appendedFileMap) -> appendedFileMap.forEach( + (appendedFile, size) -> { + final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; + if (appendedFile.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { + final String filePathWithPartition = partitionName + "/" + appendedFile; + records.addAll(getColumnStats(partition, filePathWithPartition, datasetMetaClient, + latestColumns, false).collect(Collectors.toList())); + } + })); + return records; + } + /** * Map a record key to a file group in partition of interest. - * + *

* Note: For hashing, the algorithm is same as String.hashCode() but is being defined here as hashCode() * implementation is not guaranteed by the JVM to be consistent across JVM versions and implementations. * @@ -339,7 +716,7 @@ public static int mapRecordKeyToFileGroupIndex(String recordKey, int numFileGrou */ public static List getPartitionLatestMergedFileSlices(HoodieTableMetaClient metaClient, String partition) { LOG.info("Loading latest merged file slices for metadata table partition " + partition); - return getPartitionFileSlices(metaClient, partition, true); + return getPartitionFileSlices(metaClient, Option.empty(), partition, true); } /** @@ -347,27 +724,23 @@ public static List getPartitionLatestMergedFileSlices(HoodieTableMeta * returned is sorted in the correct order of file group name. * * @param metaClient - Instance of {@link HoodieTableMetaClient}. + * @param fsView - Metadata table filesystem view * @param partition - The name of the partition whose file groups are to be loaded. * @return List of latest file slices for all file groups in a given partition. */ - public static List getPartitionLatestFileSlices(HoodieTableMetaClient metaClient, String partition) { + public static List getPartitionLatestFileSlices(HoodieTableMetaClient metaClient, + Option fsView, String partition) { LOG.info("Loading latest file slices for metadata table partition " + partition); - return getPartitionFileSlices(metaClient, partition, false); + return getPartitionFileSlices(metaClient, fsView, partition, false); } /** - * Get the latest file slices for a given partition. + * Get metadata table file system view. * - * @param metaClient - Instance of {@link HoodieTableMetaClient}. - * @param partition - The name of the partition whose file groups are to be loaded. - * @param mergeFileSlices - When enabled, will merge the latest file slices with the last known - * completed instant. This is useful for readers when there are pending - * compactions. MergeFileSlices when disabled, will return the latest file - * slices without any merging, and this is needed for the writers. - * @return List of latest file slices for all file groups in a given partition. + * @param metaClient - Metadata table meta client + * @return Filesystem view for the metadata table */ - private static List getPartitionFileSlices(HoodieTableMetaClient metaClient, String partition, - boolean mergeFileSlices) { + public static HoodieTableFileSystemView getFileSystemView(HoodieTableMetaClient metaClient) { // If there are no commits on the metadata table then the table's // default FileSystemView will not return any file slices even // though we may have initialized them. @@ -377,16 +750,175 @@ private static List getPartitionFileSlices(HoodieTableMetaClient meta HoodieActiveTimeline.createNewInstantTime()); timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails); } + return new HoodieTableFileSystemView(metaClient, timeline); + } - HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, timeline); + /** + * Get the latest file slices for a given partition. + * + * @param metaClient - Instance of {@link HoodieTableMetaClient}. + * @param partition - The name of the partition whose file groups are to be loaded. + * @param mergeFileSlices - When enabled, will merge the latest file slices with the last known + * completed instant. This is useful for readers when there are pending + * compactions. MergeFileSlices when disabled, will return the latest file + * slices without any merging, and this is needed for the writers. + * @return List of latest file slices for all file groups in a given partition. + */ + private static List getPartitionFileSlices(HoodieTableMetaClient metaClient, + Option fileSystemView, + String partition, + boolean mergeFileSlices) { + HoodieTableFileSystemView fsView = fileSystemView.orElse(getFileSystemView(metaClient)); Stream fileSliceStream; if (mergeFileSlices) { fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn( - partition, timeline.filterCompletedInstants().lastInstant().get().getTimestamp()); + partition, metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get().getTimestamp()); } else { fileSliceStream = fsView.getLatestFileSlices(partition); } return fileSliceStream.sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())).collect(Collectors.toList()); } + public static List convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata, + HoodieEngineContext engineContext, + HoodieTableMetaClient dataMetaClient, + boolean isMetaIndexColumnStatsForAllColumns, + String instantTime) { + + try { + List allWriteStats = commitMetadata.getPartitionToWriteStats().values().stream() + .flatMap(entry -> entry.stream()).collect(Collectors.toList()); + return HoodieTableMetadataUtil.createColumnStatsFromWriteStats(engineContext, dataMetaClient, allWriteStats, + isMetaIndexColumnStatsForAllColumns); + } catch (Exception e) { + throw new HoodieException("Failed to generate column stats records for metadata table ", e); + } + } + + /** + * Create column stats from write status. + * + * @param engineContext - Enging context + * @param datasetMetaClient - Dataset meta client + * @param allWriteStats - Write status to convert + * @param isMetaIndexColumnStatsForAllColumns - Are all columns enabled for indexing + */ + public static List createColumnStatsFromWriteStats(HoodieEngineContext engineContext, + HoodieTableMetaClient datasetMetaClient, + List allWriteStats, + boolean isMetaIndexColumnStatsForAllColumns) throws Exception { + if (allWriteStats.isEmpty()) { + return Collections.emptyList(); + } + + List prunedWriteStats = allWriteStats.stream().filter(writeStat -> { + return !(writeStat instanceof HoodieDeltaWriteStat); + }).collect(Collectors.toList()); + if (prunedWriteStats.isEmpty()) { + return Collections.emptyList(); + } + + return engineContext.flatMap(prunedWriteStats, + writeStat -> translateWriteStatToColumnStats(writeStat, datasetMetaClient, + getLatestColumns(datasetMetaClient, isMetaIndexColumnStatsForAllColumns)), + prunedWriteStats.size()); + } + + /** + * Get the latest columns for the table for column stats indexing. + * + * @param datasetMetaClient - Data table meta client + * @param isMetaIndexColumnStatsForAllColumns - Is column stats indexing enabled for all columns + */ + private static List getLatestColumns(HoodieTableMetaClient datasetMetaClient, boolean isMetaIndexColumnStatsForAllColumns) { + if (!isMetaIndexColumnStatsForAllColumns + || datasetMetaClient.getCommitsTimeline().filterCompletedInstants().countInstants() < 1) { + return Collections.singletonList(datasetMetaClient.getTableConfig().getRecordKeyFieldProp()); + } + + TableSchemaResolver schemaResolver = new TableSchemaResolver(datasetMetaClient); + // consider nested fields as well. if column stats is enabled only for a subset of columns, + // directly use them instead of all columns from the latest table schema + try { + return schemaResolver.getTableAvroSchema().getFields().stream() + .map(entry -> entry.name()).collect(Collectors.toList()); + } catch (Exception e) { + throw new HoodieException("Failed to get latest columns for " + datasetMetaClient.getBasePath()); + } + } + + private static List getLatestColumns(HoodieTableMetaClient datasetMetaClient) { + return getLatestColumns(datasetMetaClient, false); + } + + public static Stream translateWriteStatToColumnStats(HoodieWriteStat writeStat, + HoodieTableMetaClient datasetMetaClient, + List latestColumns) { + return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, latestColumns, false); + + } + + private static Stream getColumnStats(final String partitionPath, final String filePathWithPartition, + HoodieTableMetaClient datasetMetaClient, + List columns, boolean isDeleted) { + final String partition = partitionPath.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionPath; + final int offset = partition.equals(NON_PARTITIONED_NAME) ? (filePathWithPartition.startsWith("/") ? 1 : 0) + : partition.length() + 1; + final String fileName = filePathWithPartition.substring(offset); + if (!FSUtils.isBaseFile(new Path(fileName))) { + return Stream.empty(); + } + + if (filePathWithPartition.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { + List> columnRangeMetadataList = new ArrayList<>(); + final Path fullFilePath = new Path(datasetMetaClient.getBasePath(), filePathWithPartition); + if (!isDeleted) { + try { + columnRangeMetadataList = new ParquetUtils().readRangeFromParquetMetadata( + datasetMetaClient.getHadoopConf(), fullFilePath, columns); + } catch (Exception e) { + LOG.error("Failed to read column stats for " + fullFilePath, e); + } + } else { + columnRangeMetadataList = + columns.stream().map(entry -> new HoodieColumnRangeMetadata(fileName, + entry, null, null, 0, 0, 0, 0)) + .collect(Collectors.toList()); + } + return HoodieMetadataPayload.createColumnStatsRecords(partitionPath, columnRangeMetadataList, isDeleted); + } else { + throw new HoodieException("Column range index not supported for filePathWithPartition " + fileName); + } + } + + /** + * Get file group count for a metadata table partition. + * + * @param partitionType - Metadata table partition type + * @param metaClient - Metadata table meta client + * @param fsView - Filesystem view + * @param metadataConfig - Metadata config + * @param isBootstrapCompleted - Is bootstrap completed for the metadata table + * @return File group count for the requested metadata partition type + */ + public static int getPartitionFileGroupCount(final MetadataPartitionType partitionType, + final Option metaClient, + final Option fsView, + final HoodieMetadataConfig metadataConfig, boolean isBootstrapCompleted) { + if (isBootstrapCompleted) { + final List latestFileSlices = HoodieTableMetadataUtil + .getPartitionLatestFileSlices(metaClient.get(), fsView, partitionType.getPartitionPath()); + return Math.max(latestFileSlices.size(), 1); + } + + switch (partitionType) { + case BLOOM_FILTERS: + return metadataConfig.getBloomFilterIndexFileGroupCount(); + case COLUMN_STATS: + return metadataConfig.getColumnStatsIndexFileGroupCount(); + default: + return 1; + } + } + } 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 380f4d04d34a6..9fb268e7de1b0 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 @@ -22,19 +22,23 @@ import java.util.List; public enum MetadataPartitionType { - FILES("files", "files-"); + FILES(HoodieTableMetadataUtil.PARTITION_NAME_FILES, "files-"), + COLUMN_STATS(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS, "col-stats-"), + BLOOM_FILTERS(HoodieTableMetadataUtil.PARTITION_NAME_BLOOM_FILTERS, "bloom-filters-"); - // refers to partition path in metadata table. + // Partition path in metadata table. private final String partitionPath; - // refers to fileId prefix used for all file groups in this partition. + // FileId prefix used for all file groups in this partition. private final String fileIdPrefix; + // Total file groups + private int fileGroupCount = 1; - MetadataPartitionType(String partitionPath, String fileIdPrefix) { + MetadataPartitionType(final String partitionPath, final String fileIdPrefix) { this.partitionPath = partitionPath; this.fileIdPrefix = fileIdPrefix; } - public String partitionPath() { + public String getPartitionPath() { return partitionPath; } @@ -42,7 +46,28 @@ public String getFileIdPrefix() { return fileIdPrefix; } - public static List all() { - return Arrays.asList(MetadataPartitionType.FILES.partitionPath()); + void setFileGroupCount(final int fileGroupCount) { + this.fileGroupCount = fileGroupCount; + } + + public int getFileGroupCount() { + return this.fileGroupCount; + } + + public static List allPaths() { + return Arrays.asList( + FILES.getPartitionPath(), + COLUMN_STATS.getPartitionPath(), + BLOOM_FILTERS.getPartitionPath() + ); + } + + @Override + public String toString() { + return "Metadata partition {" + + "name: " + getPartitionPath() + + ", prefix: " + getFileIdPrefix() + + ", groups: " + getFileGroupCount() + + "}"; } }