[HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups (#4352)
* [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Today, base files have bloom filter at their footers and index lookups have to load the base file to perform any bloom lookups. Though we have interval tree based file purging, we still end up in significant amount of base file read for the bloom filter for the end index lookups for the keys. This index lookup operation can be made more performant by having all the bloom filters in a new metadata partition and doing pointed lookups based on keys. * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Adding indexing support for clean, restore and rollback operations. Each of these operations will now be converted to index records for bloom filter and column stats additionally. * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Making hoodie key consistent for both column stats and bloom index by including fileId instead of fileName, in both read and write paths. - Performance optimization for looking up records in the metadata table. - Avoiding multi column sorting needed for HoodieBloomMetaIndexBatchCheckFunction * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - HoodieBloomMetaIndexBatchCheckFunction cleanup to remove unused classes - Base file checking before reading the file footer for bloom or column stats * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Updating the bloom index and column stats index to have full file name included in the key instead of just file id. - Minor test fixes. * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Fixed flink commit method to handle metadata table all partition update records - TestBloomIndex fixes * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - SparkHoodieBloomIndexHelper code simplification for various config modes - Signature change for getBloomFilters() and getColumnStats(). Callers can just pass in interested partition and file names, the index key is then constructed internally based on the passed in parameters. - KeyLookupHandle and KeyLookupResults code refactoring - Metadata schema changes - removed the reserved field * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Removing HoodieColumnStatsMetadata and using HoodieColumnRangeMetadata instead. Fixed the users of the the removed class. * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Extending meta index test to cover deletes, compactions, clean and restore table operations. Also, fixed the getBloomFilters() and getColumnStats() to account for deleted entries. * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Addressing review comments - java doc for new classes, keys sorting for lookup, index methods renaming. * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Consolidated the bloom filter checking for keys in to one HoodieMetadataBloomIndexCheckFunction instead of a spearate batch and lazy mode. Removed all the configs around it. - Made the metadata table partition file group count configurable. - Fixed the HoodieKeyLookupHandle to have auto closable file reader when checking bloom filter and range keys. - Config property renames. Test fixes. * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Enabling column stats indexing for all columns by default - Handling column stat generation errors and test update * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Metadata table partition file group count taken from the slices when the table is bootstrapped. - Prep records for the commit refactored to the base class - HoodieFileReader interface changes for filtering keys - Multi column and data types support for colums stats index * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - rebase to latest master and merge fixes for the build and test failures * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Extending the metadata column stats type payload schema to include more statistics about the column ranges to help query integration. * [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups - Addressing review comments
This commit is contained in:
committed by
GitHub
parent
d681824982
commit
5927bdd1c0
@@ -1437,6 +1437,14 @@ public class HoodieWriteConfig extends HoodieConfig {
|
|||||||
return getBoolean(HoodieIndexConfig.BLOOM_INDEX_BUCKETIZED_CHECKING);
|
return getBoolean(HoodieIndexConfig.BLOOM_INDEX_BUCKETIZED_CHECKING);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean isMetadataBloomFilterIndexEnabled() {
|
||||||
|
return isMetadataTableEnabled() && getMetadataConfig().isBloomFilterIndexEnabled();
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isMetadataIndexColumnStatsForAllColumnsEnabled() {
|
||||||
|
return isMetadataTableEnabled() && getMetadataConfig().isMetadataColumnStatsIndexForAllColumnsEnabled();
|
||||||
|
}
|
||||||
|
|
||||||
public int getBloomIndexKeysPerBucket() {
|
public int getBloomIndexKeysPerBucket() {
|
||||||
return getInt(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET);
|
return getInt(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,17 +18,30 @@
|
|||||||
|
|
||||||
package org.apache.hudi.index;
|
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.engine.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
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.Option;
|
||||||
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
import org.apache.hudi.common.util.collection.Pair;
|
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.hudi.table.HoodieTable;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.TreeSet;
|
||||||
|
|
||||||
import static java.util.stream.Collectors.toList;
|
import static java.util.stream.Collectors.toList;
|
||||||
|
|
||||||
@@ -37,6 +50,8 @@ import static java.util.stream.Collectors.toList;
|
|||||||
*/
|
*/
|
||||||
public class HoodieIndexUtils {
|
public class HoodieIndexUtils {
|
||||||
|
|
||||||
|
private static final Logger LOG = LogManager.getLogger(HoodieIndexUtils.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Fetches Pair of partition path and {@link HoodieBaseFile}s for interested partitions.
|
* Fetches Pair of partition path and {@link HoodieBaseFile}s for interested partitions.
|
||||||
*
|
*
|
||||||
@@ -101,4 +116,34 @@ public class HoodieIndexUtils {
|
|||||||
}
|
}
|
||||||
return record;
|
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<String> filterKeysFromFile(Path filePath, List<String> candidateRecordKeys,
|
||||||
|
Configuration configuration) throws HoodieIndexException {
|
||||||
|
ValidationUtils.checkArgument(FSUtils.isBaseFile(filePath));
|
||||||
|
List<String> 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<String> 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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -25,7 +25,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieIndexException;
|
import org.apache.hudi.exception.HoodieIndexException;
|
||||||
import org.apache.hudi.io.HoodieKeyLookupHandle;
|
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.hudi.table.HoodieTable;
|
||||||
|
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
@@ -37,7 +37,7 @@ import java.util.List;
|
|||||||
* Function performing actual checking of list containing (fileId, hoodieKeys) against the actual files.
|
* Function performing actual checking of list containing (fileId, hoodieKeys) against the actual files.
|
||||||
*/
|
*/
|
||||||
public class HoodieBaseBloomIndexCheckFunction
|
public class HoodieBaseBloomIndexCheckFunction
|
||||||
implements Function<Iterator<Pair<String, HoodieKey>>, Iterator<List<KeyLookupResult>>> {
|
implements Function<Iterator<Pair<String, HoodieKey>>, Iterator<List<HoodieKeyLookupResult>>> {
|
||||||
|
|
||||||
private final HoodieTable hoodieTable;
|
private final HoodieTable hoodieTable;
|
||||||
|
|
||||||
@@ -49,11 +49,11 @@ public class HoodieBaseBloomIndexCheckFunction
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<List<KeyLookupResult>> apply(Iterator<Pair<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
|
public Iterator<List<HoodieKeyLookupResult>> apply(Iterator<Pair<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
|
||||||
return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr);
|
return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr);
|
||||||
}
|
}
|
||||||
|
|
||||||
class LazyKeyCheckIterator extends LazyIterableIterator<Pair<String, HoodieKey>, List<KeyLookupResult>> {
|
class LazyKeyCheckIterator extends LazyIterableIterator<Pair<String, HoodieKey>, List<HoodieKeyLookupResult>> {
|
||||||
|
|
||||||
private HoodieKeyLookupHandle keyLookupHandle;
|
private HoodieKeyLookupHandle keyLookupHandle;
|
||||||
|
|
||||||
@@ -66,8 +66,8 @@ public class HoodieBaseBloomIndexCheckFunction
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<KeyLookupResult> computeNext() {
|
protected List<HoodieKeyLookupResult> computeNext() {
|
||||||
List<KeyLookupResult> ret = new ArrayList<>();
|
List<HoodieKeyLookupResult> ret = new ArrayList<>();
|
||||||
try {
|
try {
|
||||||
// process one file in each go.
|
// process one file in each go.
|
||||||
while (inputItr.hasNext()) {
|
while (inputItr.hasNext()) {
|
||||||
@@ -83,7 +83,7 @@ public class HoodieBaseBloomIndexCheckFunction
|
|||||||
}
|
}
|
||||||
|
|
||||||
// if continue on current file
|
// if continue on current file
|
||||||
if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) {
|
if (keyLookupHandle.getPartitionPathFileIDPair().equals(partitionPathFilePair)) {
|
||||||
keyLookupHandle.addKey(recordKey);
|
keyLookupHandle.addKey(recordKey);
|
||||||
} else {
|
} else {
|
||||||
// do the actual checking of file & break out
|
// do the actual checking of file & break out
|
||||||
|
|||||||
@@ -19,11 +19,13 @@
|
|||||||
|
|
||||||
package org.apache.hudi.index.bloom;
|
package org.apache.hudi.index.bloom;
|
||||||
|
|
||||||
|
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
import org.apache.hudi.client.WriteStatus;
|
||||||
import org.apache.hudi.common.config.HoodieConfig;
|
import org.apache.hudi.common.config.HoodieConfig;
|
||||||
import org.apache.hudi.common.data.HoodieData;
|
import org.apache.hudi.common.data.HoodieData;
|
||||||
import org.apache.hudi.common.data.HoodiePairData;
|
import org.apache.hudi.common.data.HoodiePairData;
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
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.HoodieKey;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||||
@@ -33,6 +35,7 @@ import org.apache.hudi.common.util.collection.ImmutablePair;
|
|||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieIndexConfig;
|
import org.apache.hudi.config.HoodieIndexConfig;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
import org.apache.hudi.exception.HoodieMetadataException;
|
||||||
import org.apache.hudi.exception.MetadataNotFoundException;
|
import org.apache.hudi.exception.MetadataNotFoundException;
|
||||||
import org.apache.hudi.index.HoodieIndex;
|
import org.apache.hudi.index.HoodieIndex;
|
||||||
import org.apache.hudi.index.HoodieIndexUtils;
|
import org.apache.hudi.index.HoodieIndexUtils;
|
||||||
@@ -46,6 +49,7 @@ import java.util.ArrayList;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static java.util.stream.Collectors.groupingBy;
|
import static java.util.stream.Collectors.groupingBy;
|
||||||
import static java.util.stream.Collectors.mapping;
|
import static java.util.stream.Collectors.mapping;
|
||||||
@@ -111,13 +115,19 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload<T>>
|
|||||||
private HoodiePairData<HoodieKey, HoodieRecordLocation> lookupIndex(
|
private HoodiePairData<HoodieKey, HoodieRecordLocation> lookupIndex(
|
||||||
HoodiePairData<String, String> partitionRecordKeyPairs, final HoodieEngineContext context,
|
HoodiePairData<String, String> partitionRecordKeyPairs, final HoodieEngineContext context,
|
||||||
final HoodieTable hoodieTable) {
|
final HoodieTable hoodieTable) {
|
||||||
// Obtain records per partition, in the incoming records
|
// Step 1: Obtain records per partition, in the incoming records
|
||||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairs.countByKey();
|
Map<String, Long> recordsPerPartition = partitionRecordKeyPairs.countByKey();
|
||||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||||
|
|
||||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||||
List<Pair<String, BloomIndexFileInfo>> fileInfoList =
|
List<Pair<String, BloomIndexFileInfo>> fileInfoList;
|
||||||
loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable);
|
if (config.getBloomIndexPruneByRanges()) {
|
||||||
|
fileInfoList = (config.getMetadataConfig().isColumnStatsIndexEnabled()
|
||||||
|
? loadColumnRangesFromMetaIndex(affectedPartitionPathList, context, hoodieTable)
|
||||||
|
: loadColumnRangesFromFiles(affectedPartitionPathList, context, hoodieTable));
|
||||||
|
} else {
|
||||||
|
fileInfoList = getFileInfoForLatestBaseFiles(affectedPartitionPathList, context, hoodieTable);
|
||||||
|
}
|
||||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo =
|
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo =
|
||||||
fileInfoList.stream().collect(groupingBy(Pair::getLeft, mapping(Pair::getRight, toList())));
|
fileInfoList.stream().collect(groupingBy(Pair::getLeft, mapping(Pair::getRight, toList())));
|
||||||
|
|
||||||
@@ -133,30 +143,84 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload<T>>
|
|||||||
/**
|
/**
|
||||||
* Load all involved files as <Partition, filename> pair List.
|
* Load all involved files as <Partition, filename> pair List.
|
||||||
*/
|
*/
|
||||||
List<Pair<String, BloomIndexFileInfo>> loadInvolvedFiles(
|
List<Pair<String, BloomIndexFileInfo>> loadColumnRangesFromFiles(
|
||||||
List<String> partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) {
|
List<String> partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) {
|
||||||
// Obtain the latest data files from all the partitions.
|
// Obtain the latest data files from all the partitions.
|
||||||
List<Pair<String, String>> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream()
|
List<Pair<String, String>> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream()
|
||||||
.map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId()))
|
.map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId()))
|
||||||
.collect(toList());
|
.collect(toList());
|
||||||
|
|
||||||
if (config.getBloomIndexPruneByRanges()) {
|
context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)");
|
||||||
// also obtain file ranges, if range pruning is enabled
|
return context.map(partitionPathFileIDList, pf -> {
|
||||||
context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)");
|
try {
|
||||||
return context.map(partitionPathFileIDList, pf -> {
|
HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf);
|
||||||
try {
|
String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys();
|
||||||
HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf);
|
return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1]));
|
||||||
String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys();
|
} catch (MetadataNotFoundException me) {
|
||||||
return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1]));
|
LOG.warn("Unable to find range metadata in file :" + pf);
|
||||||
} catch (MetadataNotFoundException me) {
|
return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue()));
|
||||||
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<Pair<String, BloomIndexFileInfo>> getFileInfoForLatestBaseFiles(
|
||||||
|
List<String> partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) {
|
||||||
|
List<Pair<String, String>> 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<Pair<String, BloomIndexFileInfo>> loadColumnRangesFromMetaIndex(
|
||||||
|
List<String> 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<Pair<String, String>> partitionFileNameList = HoodieIndexUtils.getLatestBaseFilesForPartition(partitionName,
|
||||||
|
hoodieTable).stream().map(baseFile -> Pair.of(partitionName, baseFile.getFileName()))
|
||||||
|
.sorted()
|
||||||
|
.collect(toList());
|
||||||
|
if (partitionFileNameList.isEmpty()) {
|
||||||
|
return Stream.empty();
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
Map<Pair<String, String>, HoodieMetadataColumnStats> fileToColumnStatsMap = hoodieTable
|
||||||
|
.getMetadataTable().getColumnStats(partitionFileNameList, keyField);
|
||||||
|
List<Pair<String, BloomIndexFileInfo>> result = new ArrayList<>();
|
||||||
|
for (Map.Entry<Pair<String, String>, 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));
|
return result.stream();
|
||||||
} else {
|
} catch (MetadataNotFoundException me) {
|
||||||
return partitionPathFileIDList.stream()
|
throw new HoodieMetadataException("Unable to find column range metadata for partition:" + partitionName, me);
|
||||||
.map(pf -> Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList());
|
}
|
||||||
}
|
}, Math.max(partitions.size(), 1));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -55,11 +55,11 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload<T>> extends Ho
|
|||||||
* Load all involved files as <Partition, filename> pairs from all partitions in the table.
|
* Load all involved files as <Partition, filename> pairs from all partitions in the table.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
List<Pair<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
|
List<Pair<String, BloomIndexFileInfo>> loadColumnRangesFromFiles(List<String> partitions, final HoodieEngineContext context,
|
||||||
final HoodieTable hoodieTable) {
|
final HoodieTable hoodieTable) {
|
||||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||||
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
|
return super.loadColumnRangesFromFiles(allPartitionPaths, context, hoodieTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|||||||
@@ -28,7 +28,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation;
|
|||||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
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 org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
@@ -63,9 +63,8 @@ public class ListBasedHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper
|
|||||||
HoodieList.getList(fileComparisonPairs).stream()
|
HoodieList.getList(fileComparisonPairs).stream()
|
||||||
.sorted(Comparator.comparing(ImmutablePair::getLeft)).collect(toList());
|
.sorted(Comparator.comparing(ImmutablePair::getLeft)).collect(toList());
|
||||||
|
|
||||||
List<HoodieKeyLookupHandle.KeyLookupResult> keyLookupResults = new ArrayList<>();
|
List<HoodieKeyLookupResult> keyLookupResults = new ArrayList<>();
|
||||||
|
Iterator<List<HoodieKeyLookupResult>> iterator = new HoodieBaseBloomIndexCheckFunction(
|
||||||
Iterator<List<HoodieKeyLookupHandle.KeyLookupResult>> iterator = new HoodieBaseBloomIndexCheckFunction(
|
|
||||||
hoodieTable, config).apply(fileComparisonPairList.iterator());
|
hoodieTable, config).apply(fileComparisonPairList.iterator());
|
||||||
while (iterator.hasNext()) {
|
while (iterator.hasNext()) {
|
||||||
keyLookupResults.addAll(iterator.next());
|
keyLookupResults.addAll(iterator.next());
|
||||||
@@ -77,7 +76,7 @@ public class ListBasedHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper
|
|||||||
lookupResult.getMatchingRecordKeys().stream()
|
lookupResult.getMatchingRecordKeys().stream()
|
||||||
.map(recordKey -> new ImmutablePair<>(lookupResult, recordKey)).iterator()
|
.map(recordKey -> new ImmutablePair<>(lookupResult, recordKey)).iterator()
|
||||||
).mapToPair(pair -> {
|
).mapToPair(pair -> {
|
||||||
HoodieKeyLookupHandle.KeyLookupResult lookupResult = pair.getLeft();
|
HoodieKeyLookupResult lookupResult = pair.getLeft();
|
||||||
String recordKey = pair.getRight();
|
String recordKey = pair.getRight();
|
||||||
return new ImmutablePair<>(
|
return new ImmutablePair<>(
|
||||||
new HoodieKey(recordKey, lookupResult.getPartitionPath()),
|
new HoodieKey(recordKey, lookupResult.getPartitionPath()),
|
||||||
|
|||||||
@@ -19,6 +19,8 @@
|
|||||||
package org.apache.hudi.io;
|
package org.apache.hudi.io;
|
||||||
|
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
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.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
@@ -31,8 +33,8 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload, I, K, O> {
|
|||||||
protected final FileSystem fs;
|
protected final FileSystem fs;
|
||||||
protected final HoodieTable<T, I, K, O> hoodieTable;
|
protected final HoodieTable<T, I, K, O> hoodieTable;
|
||||||
|
|
||||||
HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable) {
|
HoodieIOHandle(HoodieWriteConfig config, Option<String> instantTime, HoodieTable<T, I, K, O> hoodieTable) {
|
||||||
this.instantTime = instantTime;
|
this.instantTime = instantTime.orElse(StringUtils.EMPTY_STRING);
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.hoodieTable = hoodieTable;
|
this.hoodieTable = hoodieTable;
|
||||||
this.fs = getFileSystem();
|
this.fs = getFileSystem();
|
||||||
|
|||||||
@@ -47,7 +47,7 @@ public class HoodieKeyLocationFetchHandle<T extends HoodieRecordPayload, I, K, O
|
|||||||
|
|
||||||
public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
||||||
Pair<String, HoodieBaseFile> partitionPathBaseFilePair, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
Pair<String, HoodieBaseFile> partitionPathBaseFilePair, Option<BaseKeyGenerator> 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.partitionPathBaseFilePair = partitionPathBaseFilePair;
|
||||||
this.keyGeneratorOpt = keyGeneratorOpt;
|
this.keyGeneratorOpt = keyGeneratorOpt;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,25 +19,30 @@
|
|||||||
package org.apache.hudi.io;
|
package org.apache.hudi.io;
|
||||||
|
|
||||||
import org.apache.hudi.common.bloom.BloomFilter;
|
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.HoodieBaseFile;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
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.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.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieIndexException;
|
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.hudi.table.HoodieTable;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Takes a bunch of keys and returns ones that are present in the file group.
|
* Takes a bunch of keys and returns ones that are present in the file group.
|
||||||
@@ -46,52 +51,58 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload, I, K, O> exten
|
|||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(HoodieKeyLookupHandle.class);
|
private static final Logger LOG = LogManager.getLogger(HoodieKeyLookupHandle.class);
|
||||||
|
|
||||||
private final HoodieTableType tableType;
|
|
||||||
|
|
||||||
private final BloomFilter bloomFilter;
|
private final BloomFilter bloomFilter;
|
||||||
|
|
||||||
private final List<String> candidateRecordKeys;
|
private final List<String> candidateRecordKeys;
|
||||||
|
private final boolean useMetadataTableIndex;
|
||||||
|
private Option<String> fileName = Option.empty();
|
||||||
private long totalKeysChecked;
|
private long totalKeysChecked;
|
||||||
|
|
||||||
public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
||||||
Pair<String, String> partitionPathFilePair) {
|
Pair<String, String> partitionPathFileIDPair) {
|
||||||
super(config, null, hoodieTable, partitionPathFilePair);
|
this(config, hoodieTable, partitionPathFileIDPair, Option.empty(), false);
|
||||||
this.tableType = hoodieTable.getMetaClient().getTableType();
|
|
||||||
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));
|
|
||||||
}
|
|
||||||
LOG.info(String.format("Read bloom filter from %s in %d ms", partitionPathFilePair, timer.endTimer()));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
||||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
Pair<String, String> partitionPathFileIDPair, Option<String> fileName,
|
||||||
*/
|
boolean useMetadataTableIndex) {
|
||||||
public List<String> checkCandidatesAgainstFile(Configuration configuration, List<String> candidateRecordKeys,
|
super(config, hoodieTable, partitionPathFileIDPair);
|
||||||
Path filePath) throws HoodieIndexException {
|
this.candidateRecordKeys = new ArrayList<>();
|
||||||
List<String> foundRecordKeys = new ArrayList<>();
|
this.totalKeysChecked = 0;
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
this.useMetadataTableIndex = useMetadataTableIndex;
|
||||||
|
this.bloomFilter = getBloomFilter();
|
||||||
|
}
|
||||||
|
|
||||||
|
private BloomFilter getBloomFilter() {
|
||||||
|
BloomFilter bloomFilter = null;
|
||||||
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
try {
|
try {
|
||||||
// Load all rowKeys from the file, to double-confirm
|
if (this.useMetadataTableIndex) {
|
||||||
if (!candidateRecordKeys.isEmpty()) {
|
ValidationUtils.checkArgument(this.fileName.isPresent(),
|
||||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
"File name not available to fetch bloom filter from the metadata table index.");
|
||||||
Set<String> fileRowKeys = createNewFileReader().filterRowKeys(new HashSet<>(candidateRecordKeys));
|
Option<ByteBuffer> bloomFilterByteBuffer =
|
||||||
foundRecordKeys.addAll(fileRowKeys);
|
hoodieTable.getMetadataTable().getBloomFilter(partitionPathFileIDPair.getLeft(), fileName.get());
|
||||||
LOG.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath,
|
if (!bloomFilterByteBuffer.isPresent()) {
|
||||||
timer.endTimer(), candidateRecordKeys.size(), foundRecordKeys.size()));
|
throw new HoodieIndexException("BloomFilter missing for " + partitionPathFileIDPair.getRight());
|
||||||
if (LOG.isDebugEnabled()) {
|
}
|
||||||
LOG.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
|
bloomFilter =
|
||||||
|
new HoodieDynamicBoundedBloomFilter(StandardCharsets.UTF_8.decode(bloomFilterByteBuffer.get()).toString(),
|
||||||
|
BloomFilterTypeCode.DYNAMIC_V0);
|
||||||
|
} else {
|
||||||
|
try (HoodieFileReader reader = createNewFileReader()) {
|
||||||
|
bloomFilter = reader.readBloomFilter();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieIndexException("Error checking candidate keys against file.", 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 class HoodieKeyLookupHandle<T extends HoodieRecordPayload, I, K, O> exten
|
|||||||
// check record key against bloom filter of current file & add to possible keys if needed
|
// check record key against bloom filter of current file & add to possible keys if needed
|
||||||
if (bloomFilter.mightContain(recordKey)) {
|
if (bloomFilter.mightContain(recordKey)) {
|
||||||
if (LOG.isDebugEnabled()) {
|
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);
|
candidateRecordKeys.add(recordKey);
|
||||||
}
|
}
|
||||||
@@ -111,53 +122,18 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload, I, K, O> exten
|
|||||||
/**
|
/**
|
||||||
* Of all the keys, that were added, return a list of keys that were actually found in the file group.
|
* 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()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("#The candidate row keys for " + partitionPathFilePair + " => " + candidateRecordKeys);
|
LOG.debug("#The candidate row keys for " + partitionPathFileIDPair + " => " + candidateRecordKeys);
|
||||||
}
|
}
|
||||||
|
|
||||||
HoodieBaseFile dataFile = getLatestDataFile();
|
HoodieBaseFile dataFile = getLatestDataFile();
|
||||||
List<String> matchingKeys =
|
List<String> matchingKeys = HoodieIndexUtils.filterKeysFromFile(new Path(dataFile.getPath()), candidateRecordKeys,
|
||||||
checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys, new Path(dataFile.getPath()));
|
hoodieTable.getHadoopConf());
|
||||||
LOG.info(
|
LOG.info(
|
||||||
String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)", totalKeysChecked,
|
String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)", totalKeysChecked,
|
||||||
candidateRecordKeys.size(), candidateRecordKeys.size() - matchingKeys.size(), matchingKeys.size()));
|
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);
|
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<String> matchingRecordKeys;
|
|
||||||
private final String partitionPath;
|
|
||||||
|
|
||||||
public KeyLookupResult(String fileId, String partitionPath, String baseInstantTime,
|
|
||||||
List<String> 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<String> getMatchingRecordKeys() {
|
|
||||||
return matchingRecordKeys;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -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<String> matchingRecordKeys;
|
||||||
|
private final String partitionPath;
|
||||||
|
|
||||||
|
public HoodieKeyLookupResult(String fileId, String partitionPath, String baseInstantTime,
|
||||||
|
List<String> 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<String> getMatchingRecordKeys() {
|
||||||
|
return matchingRecordKeys;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@@ -21,6 +21,7 @@ package org.apache.hudi.io;
|
|||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@@ -32,10 +33,12 @@ public class HoodieRangeInfoHandle<T extends HoodieRecordPayload, I, K, O> exten
|
|||||||
|
|
||||||
public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
||||||
Pair<String, String> partitionPathFilePair) {
|
Pair<String, String> partitionPathFilePair) {
|
||||||
super(config, null, hoodieTable, partitionPathFilePair);
|
super(config, hoodieTable, partitionPathFilePair);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String[] getMinMaxKeys() throws IOException {
|
public String[] getMinMaxKeys() throws IOException {
|
||||||
return createNewFileReader().readMinMaxRecordKeys();
|
try (HoodieFileReader reader = createNewFileReader()) {
|
||||||
|
return reader.readMinMaxRecordKeys();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,8 +18,11 @@
|
|||||||
|
|
||||||
package org.apache.hudi.io;
|
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.HoodieBaseFile;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
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.common.util.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||||
@@ -28,20 +31,17 @@ import org.apache.hudi.table.HoodieTable;
|
|||||||
|
|
||||||
import java.io.IOException;
|
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.
|
* Base class for read operations done logically on the file group.
|
||||||
*/
|
*/
|
||||||
public abstract class HoodieReadHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieIOHandle<T, I, K, O> {
|
public abstract class HoodieReadHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieIOHandle<T, I, K, O> {
|
||||||
|
|
||||||
protected final Pair<String, String> partitionPathFilePair;
|
protected final Pair<String, String> partitionPathFileIDPair;
|
||||||
|
|
||||||
public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
public HoodieReadHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
||||||
Pair<String, String> partitionPathFilePair) {
|
Pair<String, String> partitionPathFileIDPair) {
|
||||||
super(config, instantTime, hoodieTable);
|
super(config, Option.empty(), hoodieTable);
|
||||||
this.partitionPathFilePair = partitionPathFilePair;
|
this.partitionPathFileIDPair = partitionPathFileIDPair;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -49,17 +49,17 @@ public abstract class HoodieReadHandle<T extends HoodieRecordPayload, I, K, O> e
|
|||||||
return hoodieTable.getMetaClient().getFs();
|
return hoodieTable.getMetaClient().getFs();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Pair<String, String> getPartitionPathFilePair() {
|
public Pair<String, String> getPartitionPathFileIDPair() {
|
||||||
return partitionPathFilePair;
|
return partitionPathFileIDPair;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getFileId() {
|
public String getFileId() {
|
||||||
return partitionPathFilePair.getRight();
|
return partitionPathFileIDPair.getRight();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected HoodieBaseFile getLatestDataFile() {
|
protected HoodieBaseFile getLatestDataFile() {
|
||||||
return hoodieTable.getBaseFileOnlyView()
|
return hoodieTable.getBaseFileOnlyView()
|
||||||
.getLatestBaseFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get();
|
.getLatestBaseFile(partitionPathFileIDPair.getLeft(), partitionPathFileIDPair.getRight()).get();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected HoodieFileReader createNewFileReader() throws IOException {
|
protected HoodieFileReader createNewFileReader() throws IOException {
|
||||||
|
|||||||
@@ -108,7 +108,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
|||||||
protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId,
|
protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId,
|
||||||
HoodieTable<T, I, K, O> hoodieTable, Option<Schema> overriddenSchema,
|
HoodieTable<T, I, K, O> hoodieTable, Option<Schema> overriddenSchema,
|
||||||
TaskContextSupplier taskContextSupplier) {
|
TaskContextSupplier taskContextSupplier) {
|
||||||
super(config, instantTime, hoodieTable);
|
super(config, Option.of(instantTime), hoodieTable);
|
||||||
this.partitionPath = partitionPath;
|
this.partitionPath = partitionPath;
|
||||||
this.fileId = fileId;
|
this.fileId = fileId;
|
||||||
this.tableSchema = overriddenSchema.orElseGet(() -> getSpecifiedTableSchema(config));
|
this.tableSchema = overriddenSchema.orElseGet(() -> getSpecifiedTableSchema(config));
|
||||||
|
|||||||
@@ -31,6 +31,7 @@ import org.apache.hudi.common.data.HoodieData;
|
|||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
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.HoodieCleaningPolicy;
|
||||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||||
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
||||||
@@ -39,6 +40,7 @@ import org.apache.hudi.common.model.HoodieKey;
|
|||||||
import org.apache.hudi.common.model.HoodieLogFile;
|
import org.apache.hudi.common.model.HoodieLogFile;
|
||||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
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.HoodieTableType;
|
||||||
import org.apache.hudi.common.model.WriteConcurrencyMode;
|
import org.apache.hudi.common.model.WriteConcurrencyMode;
|
||||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||||
@@ -50,6 +52,7 @@ import org.apache.hudi.common.table.marker.MarkerType;
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
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.HoodieTimer;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
@@ -109,6 +112,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
protected boolean enabled;
|
protected boolean enabled;
|
||||||
protected SerializableConfiguration hadoopConf;
|
protected SerializableConfiguration hadoopConf;
|
||||||
protected final transient HoodieEngineContext engineContext;
|
protected final transient HoodieEngineContext engineContext;
|
||||||
|
// TODO: HUDI-3258 Support secondary key via multiple partitions within a single type
|
||||||
|
protected final List<MetadataPartitionType> enabledPartitionTypes;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Hudi backed table metadata writer.
|
* Hudi backed table metadata writer.
|
||||||
@@ -128,6 +133,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
this.dataWriteConfig = writeConfig;
|
this.dataWriteConfig = writeConfig;
|
||||||
this.engineContext = engineContext;
|
this.engineContext = engineContext;
|
||||||
this.hadoopConf = new SerializableConfiguration(hadoopConf);
|
this.hadoopConf = new SerializableConfiguration(hadoopConf);
|
||||||
|
this.metrics = Option.empty();
|
||||||
|
this.enabledPartitionTypes = new ArrayList<>();
|
||||||
|
|
||||||
if (writeConfig.isMetadataTableEnabled()) {
|
if (writeConfig.isMetadataTableEnabled()) {
|
||||||
this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX;
|
this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX;
|
||||||
@@ -145,22 +152,67 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(),
|
ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(),
|
||||||
"File listing cannot be used for Metadata Table");
|
"File listing cannot be used for Metadata Table");
|
||||||
|
|
||||||
initRegistry();
|
|
||||||
this.dataMetaClient =
|
this.dataMetaClient =
|
||||||
HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build();
|
HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build();
|
||||||
|
enablePartitions();
|
||||||
|
initRegistry();
|
||||||
initialize(engineContext, actionMetadata, inflightInstantTimestamp);
|
initialize(engineContext, actionMetadata, inflightInstantTimestamp);
|
||||||
initTableMetadata();
|
initTableMetadata();
|
||||||
} else {
|
} else {
|
||||||
enabled = false;
|
enabled = false;
|
||||||
this.metrics = Option.empty();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig,
|
public HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig,
|
||||||
HoodieEngineContext engineContext) {
|
HoodieEngineContext engineContext) {
|
||||||
this(hadoopConf, writeConfig, engineContext, Option.empty(), Option.empty());
|
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<HoodieTableMetaClient> 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<HoodieTableFileSystemView> 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<HoodieTableMetaClient> metaClient, Option<HoodieTableFileSystemView> fsView, boolean isBootstrapCompleted) {
|
||||||
|
final int fileGroupCount = HoodieTableMetadataUtil.getPartitionFileGroupCount(partitionType, metaClient, fsView,
|
||||||
|
metadataConfig, isBootstrapCompleted);
|
||||||
|
partitionType.setFileGroupCount(fileGroupCount);
|
||||||
|
this.enabledPartitionTypes.add(partitionType);
|
||||||
|
}
|
||||||
|
|
||||||
protected abstract void initRegistry();
|
protected abstract void initRegistry();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -257,10 +309,14 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
return metadataWriteConfig;
|
return metadataWriteConfig;
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieBackedTableMetadata metadata() {
|
public HoodieBackedTableMetadata getTableMetadata() {
|
||||||
return metadata;
|
return metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public List<MetadataPartitionType> getEnabledPartitionTypes() {
|
||||||
|
return this.enabledPartitionTypes;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize the metadata table if it does not exist.
|
* Initialize the metadata table if it does not exist.
|
||||||
*
|
*
|
||||||
@@ -460,7 +516,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
.initTable(hadoopConf.get(), metadataWriteConfig.getBasePath());
|
.initTable(hadoopConf.get(), metadataWriteConfig.getBasePath());
|
||||||
|
|
||||||
initTableMetadata();
|
initTableMetadata();
|
||||||
initializeFileGroups(dataMetaClient, MetadataPartitionType.FILES, createInstantTime, 1);
|
initializeEnabledFileGroups(dataMetaClient, createInstantTime);
|
||||||
|
|
||||||
// List all partitions in the basePath of the containing dataset
|
// List all partitions in the basePath of the containing dataset
|
||||||
LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath());
|
LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath());
|
||||||
@@ -529,6 +585,20 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
return partitionsToBootstrap;
|
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.
|
* Initialize file groups for a partition. For file listing, we just have one file group.
|
||||||
*
|
*
|
||||||
@@ -550,12 +620,12 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
final HoodieDeleteBlock block = new HoodieDeleteBlock(new HoodieKey[0], blockHeader);
|
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",
|
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) {
|
for (int i = 0; i < fileGroupCount; ++i) {
|
||||||
final String fileGroupFileId = String.format("%s%04d", metadataPartition.getFileIdPrefix(), i);
|
final String fileGroupFileId = String.format("%s%04d", metadataPartition.getFileIdPrefix(), i);
|
||||||
try {
|
try {
|
||||||
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
|
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
|
||||||
.onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), metadataPartition.partitionPath()))
|
.onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), metadataPartition.getPartitionPath()))
|
||||||
.withFileId(fileGroupFileId).overBaseCommit(instantTime)
|
.withFileId(fileGroupFileId).overBaseCommit(instantTime)
|
||||||
.withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION)
|
.withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION)
|
||||||
.withFileSize(0L)
|
.withFileSize(0L)
|
||||||
@@ -567,7 +637,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
writer.appendBlock(block);
|
writer.appendBlock(block);
|
||||||
writer.close();
|
writer.close();
|
||||||
} catch (InterruptedException e) {
|
} 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 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
* Updates of different commit metadata uses the same method to convert to HoodieRecords and hence.
|
* Updates of different commit metadata uses the same method to convert to HoodieRecords and hence.
|
||||||
*/
|
*/
|
||||||
private interface ConvertMetadataFunction {
|
private interface ConvertMetadataFunction {
|
||||||
List<HoodieRecord> convertMetadata();
|
Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadata();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -589,8 +659,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
*/
|
*/
|
||||||
private <T> void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction, boolean canTriggerTableService) {
|
private <T> void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction, boolean canTriggerTableService) {
|
||||||
if (enabled && metadata != null) {
|
if (enabled && metadata != null) {
|
||||||
List<HoodieRecord> records = convertMetadataFunction.convertMetadata();
|
Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionRecordsMap = convertMetadataFunction.convertMetadata();
|
||||||
commit(engineContext.parallelize(records, 1), MetadataPartitionType.FILES.partitionPath(), instantTime, canTriggerTableService);
|
commit(instantTime, partitionRecordsMap, canTriggerTableService);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -602,7 +672,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean isTableServiceAction) {
|
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 abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void update(HoodieCleanMetadata cleanMetadata, String instantTime) {
|
public void update(HoodieCleanMetadata cleanMetadata, String instantTime) {
|
||||||
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime),
|
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes,
|
||||||
false);
|
cleanMetadata, dataMetaClient, instantTime), false);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -625,8 +696,9 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) {
|
public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) {
|
||||||
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(),
|
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext,
|
||||||
restoreMetadata, instantTime, metadata.getSyncedInstantTime()), false);
|
enabledPartitionTypes, metadataMetaClient.getActiveTimeline(), restoreMetadata, dataMetaClient, instantTime,
|
||||||
|
metadata.getSyncedInstantTime()), false);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -650,9 +722,11 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
List<HoodieRecord> records = HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), rollbackMetadata, instantTime,
|
Map<MetadataPartitionType, HoodieData<HoodieRecord>> records =
|
||||||
metadata.getSyncedInstantTime(), wasSynced);
|
HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes,
|
||||||
commit(engineContext.parallelize(records, 1), MetadataPartitionType.FILES.partitionPath(), instantTime, false);
|
metadataMetaClient.getActiveTimeline(), rollbackMetadata, dataMetaClient, instantTime,
|
||||||
|
metadata.getSyncedInstantTime(), wasSynced);
|
||||||
|
commit(instantTime, records, false);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -665,12 +739,47 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Commit the {@code HoodieRecord}s to Metadata Table as a new delta-commit.
|
* 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 - Action instant time for this commit
|
||||||
* @param instantTime The timestamp to use for the deltacommit.
|
* @param partitionRecordsMap - Map of partition name to its records to commit
|
||||||
* @param canTriggerTableService true if table services can be scheduled and executed. false otherwise.
|
* @param canTriggerTableService true if table services can be scheduled and executed. false otherwise.
|
||||||
*/
|
*/
|
||||||
protected abstract void commit(HoodieData<HoodieRecord> records, String partitionName, String instantTime, boolean canTriggerTableService);
|
protected abstract void commit(
|
||||||
|
String instantTime, Map<MetadataPartitionType, HoodieData<HoodieRecord>> 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<HoodieRecord> prepRecords(Map<MetadataPartitionType,
|
||||||
|
HoodieData<HoodieRecord>> partitionRecordsMap) {
|
||||||
|
// The result set
|
||||||
|
HoodieData<HoodieRecord> allPartitionRecords = engineContext.emptyHoodieData();
|
||||||
|
|
||||||
|
HoodieTableFileSystemView fsView = HoodieTableMetadataUtil.getFileSystemView(metadataMetaClient);
|
||||||
|
for (Map.Entry<MetadataPartitionType, HoodieData<HoodieRecord>> entry : partitionRecordsMap.entrySet()) {
|
||||||
|
final String partitionName = entry.getKey().getPartitionPath();
|
||||||
|
final int fileGroupCount = entry.getKey().getFileGroupCount();
|
||||||
|
HoodieData<HoodieRecord> records = entry.getValue();
|
||||||
|
|
||||||
|
List<FileSlice> 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<HoodieRecord> 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.
|
* Perform a compaction on the Metadata Table.
|
||||||
@@ -735,14 +844,19 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
List<String> partitions = partitionInfoList.stream().map(p ->
|
List<String> partitions = partitionInfoList.stream().map(p ->
|
||||||
p.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : p.getRelativePath()).collect(Collectors.toList());
|
p.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : p.getRelativePath()).collect(Collectors.toList());
|
||||||
final int totalFiles = partitionInfoList.stream().mapToInt(p -> p.getTotalFiles()).sum();
|
final int totalFiles = partitionInfoList.stream().mapToInt(p -> p.getTotalFiles()).sum();
|
||||||
|
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
|
||||||
|
|
||||||
// Record which saves the list of all partitions
|
// Record which saves the list of all partitions
|
||||||
HoodieRecord allPartitionRecord = HoodieMetadataPayload.createPartitionListRecord(partitions);
|
HoodieRecord allPartitionRecord = HoodieMetadataPayload.createPartitionListRecord(partitions);
|
||||||
if (partitions.isEmpty()) {
|
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
|
// in case of bootstrapping 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);
|
final HoodieData<HoodieRecord> allPartitionRecordsRDD = engineContext.parallelize(
|
||||||
|
Collections.singletonList(allPartitionRecord), 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.FILES, allPartitionRecordsRDD);
|
||||||
|
commit(createInstantTime, partitionToRecordsMap, false);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
HoodieData<HoodieRecord> partitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1);
|
HoodieData<HoodieRecord> partitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1);
|
||||||
if (!partitionInfoList.isEmpty()) {
|
if (!partitionInfoList.isEmpty()) {
|
||||||
HoodieData<HoodieRecord> fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> {
|
HoodieData<HoodieRecord> fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> {
|
||||||
@@ -762,7 +876,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
|||||||
|
|
||||||
LOG.info("Committing " + partitions.size() + " partitions and " + totalFiles + " files to metadata");
|
LOG.info("Committing " + partitions.size() + " partitions and " + totalFiles + " files to metadata");
|
||||||
ValidationUtils.checkState(partitionRecords.count() == (partitions.size() + 1));
|
ValidationUtils.checkState(partitionRecords.count() == (partitions.size() + 1));
|
||||||
commit(partitionRecords, MetadataPartitionType.FILES.partitionPath(), createInstantTime, false);
|
partitionToRecordsMap.put(MetadataPartitionType.FILES, partitionRecords);
|
||||||
|
commit(createInstantTime, partitionToRecordsMap, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|||||||
@@ -768,4 +768,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
|||||||
return Option.empty();
|
return Option.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public HoodieTableMetadata getMetadataTable() {
|
||||||
|
return this.metadata;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,11 +22,10 @@ import org.apache.avro.specific.SpecificRecordBase;
|
|||||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
import org.apache.hudi.client.WriteStatus;
|
||||||
import org.apache.hudi.common.data.HoodieData;
|
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.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.metrics.Registry;
|
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.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
|
||||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
@@ -41,7 +40,7 @@ import org.apache.log4j.Logger;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
import java.util.Map;
|
||||||
|
|
||||||
public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
|
public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
|
||||||
|
|
||||||
@@ -101,10 +100,12 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void commit(HoodieData<HoodieRecord> hoodieDataRecords, String partitionName, String instantTime, boolean canTriggerTableService) {
|
protected void commit(String instantTime, Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionRecordsMap,
|
||||||
|
boolean canTriggerTableService) {
|
||||||
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
|
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
|
||||||
List<HoodieRecord> records = (List<HoodieRecord>) hoodieDataRecords.get();
|
ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet.");
|
||||||
List<HoodieRecord> recordList = prepRecords(records, partitionName, 1);
|
HoodieData<HoodieRecord> preppedRecords = prepRecords(partitionRecordsMap);
|
||||||
|
List<HoodieRecord> preppedRecordList = HoodieList.getList(preppedRecords);
|
||||||
|
|
||||||
try (HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(engineContext, metadataWriteConfig)) {
|
try (HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(engineContext, metadataWriteConfig)) {
|
||||||
if (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime)) {
|
if (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime)) {
|
||||||
@@ -119,13 +120,14 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
|||||||
// once rollback is complete, compaction will be retried again, which will eventually hit this code block where the respective commit is
|
// 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.
|
// 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.
|
// 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);
|
HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant);
|
||||||
metadataMetaClient.reloadActiveTimeline();
|
metadataMetaClient.reloadActiveTimeline();
|
||||||
}
|
}
|
||||||
|
|
||||||
List<WriteStatus> statuses = records.size() > 0
|
List<WriteStatus> statuses = preppedRecordList.size() > 0
|
||||||
? writeClient.upsertPreppedRecords(recordList, instantTime)
|
? writeClient.upsertPreppedRecords(preppedRecordList, instantTime)
|
||||||
: Collections.emptyList();
|
: Collections.emptyList();
|
||||||
statuses.forEach(writeStatus -> {
|
statuses.forEach(writeStatus -> {
|
||||||
if (writeStatus.hasErrors()) {
|
if (writeStatus.hasErrors()) {
|
||||||
@@ -147,21 +149,4 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
|||||||
// Update total size of the metadata and count of base/log files
|
// Update total size of the metadata and count of base/log files
|
||||||
metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata));
|
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<HoodieRecord> prepRecords(List<HoodieRecord> records, String partitionName, int numFileGroups) {
|
|
||||||
List<FileSlice> 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());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
@@ -31,7 +31,7 @@ import org.apache.hudi.common.util.Option;
|
|||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieIndexConfig;
|
import org.apache.hudi.config.HoodieIndexConfig;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
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.HoodieFlinkTable;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.testutils.HoodieFlinkClientTestHarness;
|
import org.apache.hudi.testutils.HoodieFlinkClientTestHarness;
|
||||||
@@ -130,7 +130,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness {
|
|||||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||||
|
|
||||||
List<String> partitions = asList("2016/01/21", "2016/04/01", "2015/03/12");
|
List<String> partitions = asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||||
List<Pair<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
List<Pair<String, BloomIndexFileInfo>> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
|
||||||
// Still 0, as no valid commit
|
// Still 0, as no valid commit
|
||||||
assertEquals(0, filesList.size());
|
assertEquals(0, filesList.size());
|
||||||
|
|
||||||
@@ -140,7 +140,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness {
|
|||||||
.withInserts("2015/03/12", "4", record2, record3, record4);
|
.withInserts("2015/03/12", "4", record2, record3, record4);
|
||||||
metaClient.reloadActiveTimeline();
|
metaClient.reloadActiveTimeline();
|
||||||
|
|
||||||
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
|
||||||
assertEquals(4, filesList.size());
|
assertEquals(4, filesList.size());
|
||||||
|
|
||||||
if (rangePruning) {
|
if (rangePruning) {
|
||||||
@@ -242,9 +242,8 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness {
|
|||||||
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||||
HoodieFlinkTable table = HoodieFlinkTable.create(config, context, metaClient);
|
HoodieFlinkTable table = HoodieFlinkTable.create(config, context, metaClient);
|
||||||
HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId));
|
List<String> results = HoodieIndexUtils.filterKeysFromFile(
|
||||||
List<String> results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids,
|
new Path(java.nio.file.Paths.get(basePath, partition, filename).toString()), uuids, hadoopConf);
|
||||||
new Path(java.nio.file.Paths.get(basePath, partition, filename).toString()));
|
|
||||||
assertEquals(results.size(), 2);
|
assertEquals(results.size(), 2);
|
||||||
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
||||||
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
|
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
|
||||||
|
|||||||
@@ -25,7 +25,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieIndexException;
|
import org.apache.hudi.exception.HoodieIndexException;
|
||||||
import org.apache.hudi.io.HoodieKeyLookupHandle;
|
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.hudi.table.HoodieTable;
|
||||||
|
|
||||||
import org.apache.spark.api.java.function.Function2;
|
import org.apache.spark.api.java.function.Function2;
|
||||||
@@ -40,7 +40,7 @@ import scala.Tuple2;
|
|||||||
* Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the actual files.
|
* Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the actual files.
|
||||||
*/
|
*/
|
||||||
public class HoodieBloomIndexCheckFunction
|
public class HoodieBloomIndexCheckFunction
|
||||||
implements Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<KeyLookupResult>>> {
|
implements Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<HoodieKeyLookupResult>>> {
|
||||||
|
|
||||||
private final HoodieTable hoodieTable;
|
private final HoodieTable hoodieTable;
|
||||||
|
|
||||||
@@ -52,12 +52,12 @@ public class HoodieBloomIndexCheckFunction
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<List<KeyLookupResult>> call(Integer partition,
|
public Iterator<List<HoodieKeyLookupResult>> call(Integer partition,
|
||||||
Iterator<Tuple2<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
|
Iterator<Tuple2<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
|
||||||
return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr);
|
return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr);
|
||||||
}
|
}
|
||||||
|
|
||||||
class LazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, HoodieKey>, List<KeyLookupResult>> {
|
class LazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, HoodieKey>, List<HoodieKeyLookupResult>> {
|
||||||
|
|
||||||
private HoodieKeyLookupHandle keyLookupHandle;
|
private HoodieKeyLookupHandle keyLookupHandle;
|
||||||
|
|
||||||
@@ -70,9 +70,9 @@ public class HoodieBloomIndexCheckFunction
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<HoodieKeyLookupHandle.KeyLookupResult> computeNext() {
|
protected List<HoodieKeyLookupResult> computeNext() {
|
||||||
|
|
||||||
List<HoodieKeyLookupHandle.KeyLookupResult> ret = new ArrayList<>();
|
List<HoodieKeyLookupResult> ret = new ArrayList<>();
|
||||||
try {
|
try {
|
||||||
// process one file in each go.
|
// process one file in each go.
|
||||||
while (inputItr.hasNext()) {
|
while (inputItr.hasNext()) {
|
||||||
@@ -88,7 +88,7 @@ public class HoodieBloomIndexCheckFunction
|
|||||||
}
|
}
|
||||||
|
|
||||||
// if continue on current file
|
// if continue on current file
|
||||||
if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) {
|
if (keyLookupHandle.getPartitionPathFileIDPair().equals(partitionPathFilePair)) {
|
||||||
keyLookupHandle.addKey(recordKey);
|
keyLookupHandle.addKey(recordKey);
|
||||||
} else {
|
} else {
|
||||||
// do the actual checking of file & break out
|
// do the actual checking of file & break out
|
||||||
|
|||||||
@@ -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<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<HoodieKeyLookupResult>>> {
|
||||||
|
|
||||||
|
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<List<HoodieKeyLookupResult>> call(Integer integer, Iterator<Tuple2<String, HoodieKey>> tuple2Iterator) throws Exception {
|
||||||
|
return new BloomIndexLazyKeyCheckIterator(tuple2Iterator);
|
||||||
|
}
|
||||||
|
|
||||||
|
private class BloomIndexLazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, HoodieKey>, List<HoodieKeyLookupResult>> {
|
||||||
|
public BloomIndexLazyKeyCheckIterator(Iterator<Tuple2<String, HoodieKey>> tuple2Iterator) {
|
||||||
|
super(tuple2Iterator);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void start() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected List<HoodieKeyLookupResult> computeNext() {
|
||||||
|
// Partition path and file name pair to list of keys
|
||||||
|
final Map<Pair<String, String>, List<HoodieKey>> fileToKeysMap = new HashMap<>();
|
||||||
|
final Map<String, HoodieBaseFile> fileIDBaseFileMap = new HashMap<>();
|
||||||
|
final List<HoodieKeyLookupResult> resultList = new ArrayList<>();
|
||||||
|
|
||||||
|
while (inputItr.hasNext()) {
|
||||||
|
Tuple2<String, HoodieKey> entry = inputItr.next();
|
||||||
|
final String partitionPath = entry._2.getPartitionPath();
|
||||||
|
final String fileId = entry._1;
|
||||||
|
if (!fileIDBaseFileMap.containsKey(fileId)) {
|
||||||
|
Option<HoodieBaseFile> 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<Pair<String, String>> partitionNameFileNameList = new ArrayList<>(fileToKeysMap.keySet());
|
||||||
|
Map<Pair<String, String>, 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<String> candidateRecordKeys = new ArrayList<>();
|
||||||
|
hoodieKeyList.forEach(hoodieKey -> {
|
||||||
|
totalKeys.incrementAndGet();
|
||||||
|
if (fileBloomFilter.mightContain(hoodieKey.getRecordKey())) {
|
||||||
|
candidateRecordKeys.add(hoodieKey.getRecordKey());
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
final HoodieBaseFile dataFile = fileIDBaseFileMap.get(fileId);
|
||||||
|
List<String> 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() {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -29,20 +29,19 @@ import org.apache.hudi.common.util.collection.Pair;
|
|||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.data.HoodieJavaPairRDD;
|
import org.apache.hudi.data.HoodieJavaPairRDD;
|
||||||
import org.apache.hudi.data.HoodieJavaRDD;
|
import org.apache.hudi.data.HoodieJavaRDD;
|
||||||
|
import org.apache.hudi.io.HoodieKeyLookupResult;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.Partitioner;
|
import org.apache.spark.Partitioner;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import scala.Tuple2;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helper for {@link HoodieBloomIndex} containing Spark-specific logic.
|
* Helper for {@link HoodieBloomIndex} containing Spark-specific logic.
|
||||||
*/
|
*/
|
||||||
@@ -70,26 +69,38 @@ public class SparkHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper {
|
|||||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||||
HoodieJavaRDD.getJavaRDD(fileComparisonPairs)
|
HoodieJavaRDD.getJavaRDD(fileComparisonPairs)
|
||||||
.map(pair -> new Tuple2<>(pair.getLeft(), pair.getRight()));
|
.map(pair -> new Tuple2<>(pair.getLeft(), pair.getRight()));
|
||||||
Map<String, Long> 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());
|
int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
||||||
LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${"
|
LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${"
|
||||||
+ config.getBloomIndexParallelism() + "}");
|
+ config.getBloomIndexParallelism() + "}");
|
||||||
|
|
||||||
if (config.useBloomIndexBucketizedChecking()) {
|
JavaRDD<List<HoodieKeyLookupResult>> keyLookupResultRDD;
|
||||||
|
if (config.isMetadataBloomFilterIndexEnabled()) {
|
||||||
|
// Step 1: Sort by file id
|
||||||
|
JavaRDD<Tuple2<String, HoodieKey>> 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<String, Long> comparisonsPerFileGroup = computeComparisonsPerFileGroup(
|
||||||
|
config, recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context);
|
||||||
Partitioner partitioner = new BucketizedBloomCheckPartitioner(joinParallelism, comparisonsPerFileGroup,
|
Partitioner partitioner = new BucketizedBloomCheckPartitioner(joinParallelism, comparisonsPerFileGroup,
|
||||||
config.getBloomIndexKeysPerBucket());
|
config.getBloomIndexKeysPerBucket());
|
||||||
|
|
||||||
fileComparisonsRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
|
keyLookupResultRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
|
||||||
.repartitionAndSortWithinPartitions(partitioner).map(Tuple2::_2);
|
.repartitionAndSortWithinPartitions(partitioner)
|
||||||
|
.map(Tuple2::_2)
|
||||||
|
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true);
|
||||||
} else {
|
} 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)
|
return HoodieJavaPairRDD.of(keyLookupResultRDD.flatMap(List::iterator)
|
||||||
.flatMap(List::iterator).filter(lr -> lr.getMatchingRecordKeys().size() > 0)
|
.filter(lr -> lr.getMatchingRecordKeys().size() > 0)
|
||||||
.flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream()
|
.flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream()
|
||||||
.map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()),
|
.map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()),
|
||||||
new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())))
|
new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())))
|
||||||
|
|||||||
@@ -200,7 +200,7 @@ public class ColumnStatsIndexHelper {
|
|||||||
|
|
||||||
indexRow.add(minMaxValue.getLeft()); // min
|
indexRow.add(minMaxValue.getLeft()); // min
|
||||||
indexRow.add(minMaxValue.getRight()); // max
|
indexRow.add(minMaxValue.getRight()); // max
|
||||||
indexRow.add(colMetadata.getNumNulls());
|
indexRow.add(colMetadata.getNullCount());
|
||||||
});
|
});
|
||||||
|
|
||||||
return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(indexRow));
|
return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(indexRow));
|
||||||
|
|||||||
@@ -25,14 +25,13 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
|||||||
import org.apache.hudi.common.data.HoodieData;
|
import org.apache.hudi.common.data.HoodieData;
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.metrics.Registry;
|
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.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
|
||||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
import org.apache.hudi.data.HoodieJavaRDD;
|
||||||
import org.apache.hudi.exception.HoodieMetadataException;
|
import org.apache.hudi.exception.HoodieMetadataException;
|
||||||
import org.apache.hudi.metrics.DistributedRegistry;
|
import org.apache.hudi.metrics.DistributedRegistry;
|
||||||
|
|
||||||
@@ -43,6 +42,7 @@ import org.apache.spark.api.java.JavaRDD;
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
|
public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
|
||||||
|
|
||||||
@@ -121,11 +121,12 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void commit(HoodieData<HoodieRecord> hoodieDataRecords, String partitionName, String instantTime, boolean canTriggerTableService) {
|
@Override
|
||||||
|
protected void commit(String instantTime, Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionRecordsMap, boolean canTriggerTableService) {
|
||||||
ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet.");
|
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");
|
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
|
||||||
JavaRDD<HoodieRecord> records = (JavaRDD<HoodieRecord>) hoodieDataRecords.get();
|
HoodieData<HoodieRecord> preppedRecords = prepRecords(partitionRecordsMap);
|
||||||
JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName, 1);
|
JavaRDD<HoodieRecord> preppedRecordRDD = HoodieJavaRDD.getJavaRDD(preppedRecords);
|
||||||
|
|
||||||
try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {
|
try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {
|
||||||
if (canTriggerTableService) {
|
if (canTriggerTableService) {
|
||||||
@@ -150,7 +151,7 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
|||||||
HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant);
|
HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant);
|
||||||
metadataMetaClient.reloadActiveTimeline();
|
metadataMetaClient.reloadActiveTimeline();
|
||||||
}
|
}
|
||||||
List<WriteStatus> statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect();
|
List<WriteStatus> statuses = writeClient.upsertPreppedRecords(preppedRecordRDD, instantTime).collect();
|
||||||
statuses.forEach(writeStatus -> {
|
statuses.forEach(writeStatus -> {
|
||||||
if (writeStatus.hasErrors()) {
|
if (writeStatus.hasErrors()) {
|
||||||
throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime);
|
throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime);
|
||||||
@@ -168,20 +169,4 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
|||||||
// Update total size of the metadata and count of base/log files
|
// Update total size of the metadata and count of base/log files
|
||||||
metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata));
|
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<HoodieRecord> prepRecords(JavaRDD<HoodieRecord> recordsRDD, String partitionName, int numFileGroups) {
|
|
||||||
List<FileSlice> 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;
|
|
||||||
});
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -191,6 +191,8 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
// trigger couple of upserts
|
// trigger couple of upserts
|
||||||
doWriteOperation(testTable, "0000005");
|
doWriteOperation(testTable, "0000005");
|
||||||
doWriteOperation(testTable, "0000006");
|
doWriteOperation(testTable, "0000006");
|
||||||
|
doWriteOperation(testTable, "0000007");
|
||||||
|
doCleanAndValidate(testTable, "0000008", Arrays.asList("0000007"));
|
||||||
validateMetadata(testTable, true);
|
validateMetadata(testTable, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -222,7 +224,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
testTable.doWriteOperation("0000003", UPSERT, emptyList(), asList("p1", "p2"), 1, true);
|
testTable.doWriteOperation("0000003", UPSERT, emptyList(), asList("p1", "p2"), 1, true);
|
||||||
syncTableMetadata(writeConfig);
|
syncTableMetadata(writeConfig);
|
||||||
|
|
||||||
List<String> partitions = metadataWriter(writeConfig).metadata().getAllPartitionPaths();
|
List<String> partitions = metadataWriter(writeConfig).getTableMetadata().getAllPartitionPaths();
|
||||||
assertFalse(partitions.contains(nonPartitionDirectory),
|
assertFalse(partitions.contains(nonPartitionDirectory),
|
||||||
"Must not contain the non-partition " + nonPartitionDirectory);
|
"Must not contain the non-partition " + nonPartitionDirectory);
|
||||||
assertTrue(partitions.contains("p1"), "Must contain partition p1");
|
assertTrue(partitions.contains("p1"), "Must contain partition p1");
|
||||||
@@ -345,6 +347,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
if (tableType == MERGE_ON_READ) {
|
if (tableType == MERGE_ON_READ) {
|
||||||
doCompaction(testTable, "0000004");
|
doCompaction(testTable, "0000004");
|
||||||
}
|
}
|
||||||
|
doCleanAndValidate(testTable, "0000005", Arrays.asList("0000001"));
|
||||||
validateMetadata(testTable, emptyList(), true);
|
validateMetadata(testTable, emptyList(), true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -380,6 +383,32 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
assertEquals(tableMetadata.getLatestCompactionTime().get(), "0000003001");
|
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.
|
* Tests that virtual key configs are honored in base files after compaction in metadata table.
|
||||||
*
|
*
|
||||||
@@ -619,7 +648,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
// Compaction should not be triggered yet. Let's verify no base file
|
// Compaction should not be triggered yet. Let's verify no base file
|
||||||
// and few log files available.
|
// and few log files available.
|
||||||
List<FileSlice> fileSlices = table.getSliceView()
|
List<FileSlice> fileSlices = table.getSliceView()
|
||||||
.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
|
.getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList());
|
||||||
if (fileSlices.isEmpty()) {
|
if (fileSlices.isEmpty()) {
|
||||||
throw new IllegalStateException("LogFile slices are not available!");
|
throw new IllegalStateException("LogFile slices are not available!");
|
||||||
}
|
}
|
||||||
@@ -709,7 +738,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
.withBasePath(metadataMetaClient.getBasePath())
|
.withBasePath(metadataMetaClient.getBasePath())
|
||||||
.withLogFilePaths(logFilePaths)
|
.withLogFilePaths(logFilePaths)
|
||||||
.withLatestInstantTime(latestCommitTimestamp)
|
.withLatestInstantTime(latestCommitTimestamp)
|
||||||
.withPartition(MetadataPartitionType.FILES.partitionPath())
|
.withPartition(MetadataPartitionType.FILES.getPartitionPath())
|
||||||
.withReaderSchema(schema)
|
.withReaderSchema(schema)
|
||||||
.withMaxMemorySizeInBytes(100000L)
|
.withMaxMemorySizeInBytes(100000L)
|
||||||
.withBufferSize(4096)
|
.withBufferSize(4096)
|
||||||
@@ -739,7 +768,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table, boolean enableMetaFields) throws IOException {
|
private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table, boolean enableMetaFields) throws IOException {
|
||||||
table.getHoodieView().sync();
|
table.getHoodieView().sync();
|
||||||
List<FileSlice> fileSlices = table.getSliceView()
|
List<FileSlice> fileSlices = table.getSliceView()
|
||||||
.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
|
.getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList());
|
||||||
if (!fileSlices.get(0).getBaseFile().isPresent()) {
|
if (!fileSlices.get(0).getBaseFile().isPresent()) {
|
||||||
throw new IllegalStateException("Base file not available!");
|
throw new IllegalStateException("Base file not available!");
|
||||||
}
|
}
|
||||||
@@ -1058,10 +1087,20 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exception {
|
public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exception {
|
||||||
init(tableType);
|
init(tableType);
|
||||||
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
|
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)
|
// Write 1 (Bulk insert)
|
||||||
String newCommitTime = "0000001";
|
String newCommitTime = "0000001";
|
||||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
|
||||||
@@ -1738,7 +1777,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".count"));
|
assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".count"));
|
||||||
assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".totalDuration"));
|
assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".totalDuration"));
|
||||||
assertTrue(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_STR + ".count") >= 1L);
|
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_BASE_FILES));
|
||||||
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_LOG_FILES));
|
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_LOG_FILES));
|
||||||
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE));
|
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE));
|
||||||
@@ -1931,7 +1970,10 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
// in the .hoodie folder.
|
// in the .hoodie folder.
|
||||||
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
|
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
|
||||||
false, false);
|
false, false);
|
||||||
assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size());
|
assertEquals(metadataWriter.getEnabledPartitionTypes().size(), metadataTablePartitions.size());
|
||||||
|
|
||||||
|
final Map<String, MetadataPartitionType> metadataEnabledPartitionTypes = new HashMap<>();
|
||||||
|
metadataWriter.getEnabledPartitionTypes().forEach(e -> metadataEnabledPartitionTypes.put(e.getPartitionPath(), e));
|
||||||
|
|
||||||
// Metadata table should automatically compact and clean
|
// Metadata table should automatically compact and clean
|
||||||
// versions are +1 as autoclean / compaction happens end of commits
|
// versions are +1 as autoclean / compaction happens end of commits
|
||||||
@@ -1939,10 +1981,13 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metadataMetaClient, metadataMetaClient.getActiveTimeline());
|
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metadataMetaClient, metadataMetaClient.getActiveTimeline());
|
||||||
metadataTablePartitions.forEach(partition -> {
|
metadataTablePartitions.forEach(partition -> {
|
||||||
List<FileSlice> latestSlices = fsView.getLatestFileSlices(partition).collect(Collectors.toList());
|
List<FileSlice> latestSlices = fsView.getLatestFileSlices(partition).collect(Collectors.toList());
|
||||||
assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count() <= 1, "Should have a single latest base file");
|
assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count()
|
||||||
assertTrue(latestSlices.size() <= 1, "Should have a single latest file slice");
|
<= metadataEnabledPartitionTypes.get(partition).getFileGroupCount(), "Should have a single latest base file per file group");
|
||||||
assertTrue(latestSlices.size() <= numFileVersions, "Should limit file slice to "
|
assertTrue(latestSlices.size()
|
||||||
+ numFileVersions + " but was " + 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());
|
LOG.info("Validation time=" + timer.endTimer());
|
||||||
|
|||||||
@@ -247,7 +247,7 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
|
|||||||
// Compaction should not be triggered yet. Let's verify no base file
|
// Compaction should not be triggered yet. Let's verify no base file
|
||||||
// and few log files available.
|
// and few log files available.
|
||||||
List<FileSlice> fileSlices = table.getSliceView()
|
List<FileSlice> fileSlices = table.getSliceView()
|
||||||
.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
|
.getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList());
|
||||||
if (fileSlices.isEmpty()) {
|
if (fileSlices.isEmpty()) {
|
||||||
throw new IllegalStateException("LogFile slices are not available!");
|
throw new IllegalStateException("LogFile slices are not available!");
|
||||||
}
|
}
|
||||||
@@ -322,7 +322,7 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
|
|||||||
.withBasePath(metadataMetaClient.getBasePath())
|
.withBasePath(metadataMetaClient.getBasePath())
|
||||||
.withLogFilePaths(logFilePaths)
|
.withLogFilePaths(logFilePaths)
|
||||||
.withLatestInstantTime(latestCommitTimestamp)
|
.withLatestInstantTime(latestCommitTimestamp)
|
||||||
.withPartition(MetadataPartitionType.FILES.partitionPath())
|
.withPartition(MetadataPartitionType.FILES.getPartitionPath())
|
||||||
.withReaderSchema(schema)
|
.withReaderSchema(schema)
|
||||||
.withMaxMemorySizeInBytes(100000L)
|
.withMaxMemorySizeInBytes(100000L)
|
||||||
.withBufferSize(4096)
|
.withBufferSize(4096)
|
||||||
@@ -351,7 +351,7 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
|
|||||||
private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table) throws IOException {
|
private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table) throws IOException {
|
||||||
table.getHoodieView().sync();
|
table.getHoodieView().sync();
|
||||||
List<FileSlice> fileSlices = table.getSliceView()
|
List<FileSlice> fileSlices = table.getSliceView()
|
||||||
.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
|
.getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList());
|
||||||
if (!fileSlices.get(0).getBaseFile().isPresent()) {
|
if (!fileSlices.get(0).getBaseFile().isPresent()) {
|
||||||
throw new IllegalStateException("Base file not available!");
|
throw new IllegalStateException("Base file not available!");
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -34,6 +34,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
|||||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
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.HoodieCompactionConfig;
|
||||||
import org.apache.hudi.config.HoodieIndexConfig;
|
import org.apache.hudi.config.HoodieIndexConfig;
|
||||||
import org.apache.hudi.config.HoodieStorageConfig;
|
import org.apache.hudi.config.HoodieStorageConfig;
|
||||||
@@ -84,27 +85,22 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
|
|||||||
init(tableType, true);
|
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 {
|
public void init(HoodieTableType tableType, boolean enableMetadataTable) throws IOException {
|
||||||
init(tableType, enableMetadataTable, true, false, false);
|
init(tableType, enableMetadataTable, true, false, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan, boolean enableMetrics, boolean
|
public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan, boolean enableMetrics, boolean
|
||||||
validateMetadataPayloadStateConsistency) throws IOException {
|
validateMetadataPayloadStateConsistency) throws IOException {
|
||||||
this.tableType = tableType;
|
init(tableType, Option.empty(), enableMetadataTable, enableFullScan, enableMetrics,
|
||||||
initPath();
|
validateMetadataPayloadStateConsistency);
|
||||||
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);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void init(HoodieTableType tableType, HoodieWriteConfig writeConfig) throws IOException {
|
public void init(HoodieTableType tableType, Option<HoodieWriteConfig> writeConfig, boolean enableMetadataTable,
|
||||||
|
boolean enableFullScan, boolean enableMetrics, boolean validateMetadataPayloadStateConsistency) throws IOException {
|
||||||
this.tableType = tableType;
|
this.tableType = tableType;
|
||||||
initPath();
|
initPath();
|
||||||
initSparkContexts("TestHoodieMetadata");
|
initSparkContexts("TestHoodieMetadata");
|
||||||
@@ -114,8 +110,12 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
|
|||||||
initMetaClient(tableType);
|
initMetaClient(tableType);
|
||||||
initTestDataGenerator();
|
initTestDataGenerator();
|
||||||
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
||||||
this.writeConfig = writeConfig;
|
this.writeConfig = writeConfig.isPresent()
|
||||||
initWriteConfigAndMetatableWriter(writeConfig, writeConfig.isMetadataTableEnabled());
|
? writeConfig.get() : getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true,
|
||||||
|
enableMetadataTable, enableMetrics, enableFullScan, true,
|
||||||
|
validateMetadataPayloadStateConsistency)
|
||||||
|
.build();
|
||||||
|
initWriteConfigAndMetatableWriter(this.writeConfig, enableMetadataTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) {
|
protected void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) {
|
||||||
|
|||||||
@@ -18,9 +18,13 @@
|
|||||||
|
|
||||||
package org.apache.hudi.index.bloom;
|
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.BloomFilter;
|
||||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
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.HoodieKey;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
@@ -32,14 +36,10 @@ import org.apache.hudi.config.HoodieIndexConfig;
|
|||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.data.HoodieJavaPairRDD;
|
import org.apache.hudi.data.HoodieJavaPairRDD;
|
||||||
import org.apache.hudi.data.HoodieJavaRDD;
|
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.HoodieSparkTable;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
|
||||||
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
|
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.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.junit.jupiter.api.AfterEach;
|
import org.junit.jupiter.api.AfterEach;
|
||||||
@@ -48,6 +48,7 @@ import org.junit.jupiter.api.Test;
|
|||||||
import org.junit.jupiter.params.ParameterizedTest;
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
import org.junit.jupiter.params.provider.Arguments;
|
import org.junit.jupiter.params.provider.Arguments;
|
||||||
import org.junit.jupiter.params.provider.MethodSource;
|
import org.junit.jupiter.params.provider.MethodSource;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
@@ -59,8 +60,6 @@ import java.util.UUID;
|
|||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import scala.Tuple2;
|
|
||||||
|
|
||||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
@@ -69,14 +68,14 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
|
|||||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
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 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}";
|
private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}";
|
||||||
|
|
||||||
public static Stream<Arguments> configParams() {
|
public static Stream<Arguments> configParams() {
|
||||||
Object[][] data =
|
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);
|
return Stream.of(data).map(Arguments::of);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -99,6 +98,10 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
|||||||
.withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning)
|
.withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning)
|
||||||
.bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking)
|
.bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking)
|
||||||
.bloomIndexKeysPerBucket(2).build())
|
.bloomIndexKeysPerBucket(2).build())
|
||||||
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
|
||||||
|
.withMetadataIndexBloomFilter(false)
|
||||||
|
.withMetadataIndexColumnStats(false)
|
||||||
|
.build())
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -134,7 +137,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
|||||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||||
|
|
||||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||||
List<ImmutablePair<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
List<ImmutablePair<String, BloomIndexFileInfo>> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
|
||||||
// Still 0, as no valid commit
|
// Still 0, as no valid commit
|
||||||
assertEquals(0, filesList.size());
|
assertEquals(0, filesList.size());
|
||||||
|
|
||||||
@@ -143,7 +146,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
|||||||
.withInserts("2015/03/12", "3", record1)
|
.withInserts("2015/03/12", "3", record1)
|
||||||
.withInserts("2015/03/12", "4", record2, record3, record4);
|
.withInserts("2015/03/12", "4", record2, record3, record4);
|
||||||
|
|
||||||
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
|
||||||
assertEquals(4, filesList.size());
|
assertEquals(4, filesList.size());
|
||||||
|
|
||||||
if (rangePruning) {
|
if (rangePruning) {
|
||||||
@@ -241,9 +244,9 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
|||||||
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||||
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
|
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||||
HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId));
|
List<String> results = HoodieIndexUtils.filterKeysFromFile(
|
||||||
List<String> results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids,
|
new Path(Paths.get(basePath, partition, filename).toString()), uuids, hadoopConf);
|
||||||
new Path(Paths.get(basePath, partition, filename).toString()));
|
|
||||||
assertEquals(results.size(), 2);
|
assertEquals(results.size(), 2);
|
||||||
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
||||||
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
|
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
|
||||||
|
|||||||
@@ -109,7 +109,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
|||||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
|
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
|
||||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||||
List<Pair<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
List<Pair<String, BloomIndexFileInfo>> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
|
||||||
// Still 0, as no valid commit
|
// Still 0, as no valid commit
|
||||||
assertEquals(0, filesList.size());
|
assertEquals(0, filesList.size());
|
||||||
|
|
||||||
@@ -118,7 +118,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
|||||||
.withInserts("2015/03/12", "3", record1)
|
.withInserts("2015/03/12", "3", record1)
|
||||||
.withInserts("2015/03/12", "4", record2, record3, record4);
|
.withInserts("2015/03/12", "4", record2, record3, record4);
|
||||||
|
|
||||||
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
|
||||||
assertEquals(4, filesList.size());
|
assertEquals(4, filesList.size());
|
||||||
|
|
||||||
Map<String, BloomIndexFileInfo> filesMap = toFileMap(filesList);
|
Map<String, BloomIndexFileInfo> filesMap = toFileMap(filesList);
|
||||||
|
|||||||
@@ -59,7 +59,6 @@ import org.apache.hudi.metadata.FileSystemBackedTableMetadata;
|
|||||||
import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
|
import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
|
||||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||||
import org.apache.hudi.metadata.MetadataPartitionType;
|
|
||||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||||
import org.apache.hudi.table.HoodieSparkTable;
|
import org.apache.hudi.table.HoodieSparkTable;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
@@ -680,7 +679,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
|||||||
// in the .hoodie folder.
|
// in the .hoodie folder.
|
||||||
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
|
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
|
||||||
false, false);
|
false, false);
|
||||||
Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size());
|
Assertions.assertEquals(metadataWriter.getEnabledPartitionTypes().size(), metadataTablePartitions.size());
|
||||||
|
|
||||||
// Metadata table should automatically compact and clean
|
// Metadata table should automatically compact and clean
|
||||||
// versions are +1 as autoClean / compaction happens end of commits
|
// versions are +1 as autoClean / compaction happens end of commits
|
||||||
|
|||||||
@@ -30,27 +30,140 @@
|
|||||||
"doc": "Type of the metadata record",
|
"doc": "Type of the metadata record",
|
||||||
"type": "int"
|
"type": "int"
|
||||||
},
|
},
|
||||||
{ "name": "filesystemMetadata",
|
{
|
||||||
"doc": "Contains information about partitions and files within the dataset",
|
"doc": "Contains information about partitions and files within the dataset",
|
||||||
"type": ["null", {
|
"name": "filesystemMetadata",
|
||||||
"type": "map",
|
"type": [
|
||||||
"values": {
|
"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",
|
"type": "record",
|
||||||
"name": "HoodieMetadataFileInfo",
|
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
"name": "size",
|
"doc": "Bloom filter type code",
|
||||||
"type": "long",
|
"name": "type",
|
||||||
"doc": "Size of the file"
|
"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",
|
"name": "isDeleted",
|
||||||
"type": "boolean",
|
"type": "boolean"
|
||||||
"doc": "True if this file has been deleted"
|
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
}]
|
]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"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"
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
]
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -63,7 +63,7 @@ public class HoodieDynamicBoundedBloomFilter implements BloomFilter {
|
|||||||
* @param serString the serialized string which represents the {@link HoodieDynamicBoundedBloomFilter}
|
* @param serString the serialized string which represents the {@link HoodieDynamicBoundedBloomFilter}
|
||||||
* @param typeCode type code of the bloom filter
|
* @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
|
// ignoring the type code for now, since we have just one version
|
||||||
byte[] bytes = Base64CodecUtil.decode(serString);
|
byte[] bytes = Base64CodecUtil.decode(serString);
|
||||||
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
|
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
|
||||||
|
|||||||
@@ -124,6 +124,47 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
|||||||
.sinceVersion("0.10.0")
|
.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.");
|
.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<Boolean> 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<Integer> 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<Boolean> 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<Integer> 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<Boolean> 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<Boolean> POPULATE_META_FIELDS = ConfigProperty
|
public static final ConfigProperty<Boolean> POPULATE_META_FIELDS = ConfigProperty
|
||||||
.key(METADATA_PREFIX + ".populate.meta.fields")
|
.key(METADATA_PREFIX + ".populate.meta.fields")
|
||||||
.defaultValue(false)
|
.defaultValue(false)
|
||||||
@@ -157,6 +198,26 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
|||||||
return getBoolean(ENABLE);
|
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() {
|
public boolean enableMetrics() {
|
||||||
return getBoolean(METRICS_ENABLE);
|
return getBoolean(METRICS_ENABLE);
|
||||||
}
|
}
|
||||||
@@ -199,6 +260,31 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
|||||||
return this;
|
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) {
|
public Builder enableMetrics(boolean enableMetrics) {
|
||||||
metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics));
|
metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics));
|
||||||
return this;
|
return this;
|
||||||
|
|||||||
@@ -28,14 +28,21 @@ public class HoodieColumnRangeMetadata<T> {
|
|||||||
private final String columnName;
|
private final String columnName;
|
||||||
private final T minValue;
|
private final T minValue;
|
||||||
private final T maxValue;
|
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.filePath = filePath;
|
||||||
this.columnName = columnName;
|
this.columnName = columnName;
|
||||||
this.minValue = minValue;
|
this.minValue = minValue;
|
||||||
this.maxValue = maxValue;
|
this.maxValue = maxValue;
|
||||||
this.numNulls = numNulls;
|
this.nullCount = nullCount;
|
||||||
|
this.valueCount = valueCount;
|
||||||
|
this.totalSize = totalSize;
|
||||||
|
this.totalUncompressedSize = totalUncompressedSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getFilePath() {
|
public String getFilePath() {
|
||||||
@@ -54,8 +61,20 @@ public class HoodieColumnRangeMetadata<T> {
|
|||||||
return this.maxValue;
|
return this.maxValue;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getNumNulls() {
|
public long getNullCount() {
|
||||||
return numNulls;
|
return nullCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getValueCount() {
|
||||||
|
return valueCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getTotalSize() {
|
||||||
|
return totalSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getTotalUncompressedSize() {
|
||||||
|
return totalUncompressedSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -71,12 +90,15 @@ public class HoodieColumnRangeMetadata<T> {
|
|||||||
&& Objects.equals(getColumnName(), that.getColumnName())
|
&& Objects.equals(getColumnName(), that.getColumnName())
|
||||||
&& Objects.equals(getMinValue(), that.getMinValue())
|
&& Objects.equals(getMinValue(), that.getMinValue())
|
||||||
&& Objects.equals(getMaxValue(), that.getMaxValue())
|
&& 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
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
return Objects.hash(getColumnName(), getMinValue(), getMaxValue(), getNumNulls());
|
return Objects.hash(getColumnName(), getMinValue(), getMaxValue(), getNullCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -86,6 +108,10 @@ public class HoodieColumnRangeMetadata<T> {
|
|||||||
+ "columnName='" + columnName + '\''
|
+ "columnName='" + columnName + '\''
|
||||||
+ ", minValue=" + minValue
|
+ ", minValue=" + minValue
|
||||||
+ ", maxValue=" + maxValue
|
+ ", maxValue=" + maxValue
|
||||||
+ ", numNulls=" + numNulls + '}';
|
+ ", nullCount=" + nullCount
|
||||||
|
+ ", valueCount=" + valueCount
|
||||||
|
+ ", totalSize=" + totalSize
|
||||||
|
+ ", totalUncompressedSize=" + totalUncompressedSize
|
||||||
|
+ '}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -175,11 +175,11 @@ public abstract class AbstractHoodieLogRecordReader {
|
|||||||
return this.simpleKeyGenFields.get().getKey();
|
return this.simpleKeyGenFields.get().getKey();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void scan() {
|
public synchronized void scan() {
|
||||||
scan(Option.empty());
|
scan(Option.empty());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void scan(Option<List<String>> keys) {
|
public synchronized void scan(Option<List<String>> keys) {
|
||||||
currentInstantLogBlocks = new ArrayDeque<>();
|
currentInstantLogBlocks = new ArrayDeque<>();
|
||||||
progress = 0.0f;
|
progress = 0.0f;
|
||||||
totalLogFiles = new AtomicLong(0);
|
totalLogFiles = new AtomicLong(0);
|
||||||
|
|||||||
@@ -30,6 +30,8 @@ import org.apache.hudi.common.model.HoodieRecord;
|
|||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.exception.MetadataNotFoundException;
|
import org.apache.hudi.exception.MetadataNotFoundException;
|
||||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
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.AvroParquetReader;
|
||||||
import org.apache.parquet.avro.AvroReadSupport;
|
import org.apache.parquet.avro.AvroReadSupport;
|
||||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||||
@@ -62,6 +64,8 @@ import java.util.stream.Stream;
|
|||||||
*/
|
*/
|
||||||
public class ParquetUtils extends BaseFileUtils {
|
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
|
* 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.
|
* return all the rowkeys.
|
||||||
@@ -300,18 +304,21 @@ public class ParquetUtils extends BaseFileUtils {
|
|||||||
Map<String, List<HoodieColumnRangeMetadata<Comparable>>> columnToStatsListMap = metadata.getBlocks().stream().sequential()
|
Map<String, List<HoodieColumnRangeMetadata<Comparable>>> columnToStatsListMap = metadata.getBlocks().stream().sequential()
|
||||||
.flatMap(blockMetaData -> blockMetaData.getColumns().stream()
|
.flatMap(blockMetaData -> blockMetaData.getColumns().stream()
|
||||||
.filter(f -> cols.contains(f.getPath().toDotString()))
|
.filter(f -> cols.contains(f.getPath().toDotString()))
|
||||||
.map(columnChunkMetaData ->
|
.map(columnChunkMetaData ->
|
||||||
new HoodieColumnRangeMetadata<Comparable>(
|
new HoodieColumnRangeMetadata<Comparable>(
|
||||||
parquetFilePath.getName(),
|
parquetFilePath.getName(),
|
||||||
columnChunkMetaData.getPath().toDotString(),
|
columnChunkMetaData.getPath().toDotString(),
|
||||||
convertToNativeJavaType(
|
convertToNativeJavaType(
|
||||||
columnChunkMetaData.getPrimitiveType(),
|
columnChunkMetaData.getPrimitiveType(),
|
||||||
columnChunkMetaData.getStatistics().genericGetMin()),
|
columnChunkMetaData.getStatistics().genericGetMin()),
|
||||||
convertToNativeJavaType(
|
convertToNativeJavaType(
|
||||||
columnChunkMetaData.getPrimitiveType(),
|
columnChunkMetaData.getPrimitiveType(),
|
||||||
columnChunkMetaData.getStatistics().genericGetMax()),
|
columnChunkMetaData.getStatistics().genericGetMax()),
|
||||||
columnChunkMetaData.getStatistics().getNumNulls())))
|
columnChunkMetaData.getStatistics().getNumNulls(),
|
||||||
.collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName));
|
columnChunkMetaData.getValueCount(),
|
||||||
|
columnChunkMetaData.getTotalSize(),
|
||||||
|
columnChunkMetaData.getTotalUncompressedSize()))
|
||||||
|
).collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName));
|
||||||
|
|
||||||
// Combine those into file-level statistics
|
// Combine those into file-level statistics
|
||||||
// NOTE: Inlining this var makes javac (1.8) upset (due to its inability to infer
|
// NOTE: Inlining this var makes javac (1.8) upset (due to its inability to infer
|
||||||
@@ -355,13 +362,17 @@ public class ParquetUtils extends BaseFileUtils {
|
|||||||
maxValue = one.getMaxValue().compareTo(another.getMaxValue()) < 0 ? another.getMaxValue() : one.getMaxValue();
|
maxValue = one.getMaxValue().compareTo(another.getMaxValue()) < 0 ? another.getMaxValue() : one.getMaxValue();
|
||||||
} else if (one.getMaxValue() == null) {
|
} else if (one.getMaxValue() == null) {
|
||||||
maxValue = another.getMaxValue();
|
maxValue = another.getMaxValue();
|
||||||
} else {
|
} else {
|
||||||
maxValue = one.getMaxValue();
|
maxValue = one.getMaxValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
return new HoodieColumnRangeMetadata<T>(
|
return new HoodieColumnRangeMetadata<T>(
|
||||||
one.getFilePath(),
|
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) {
|
private static Comparable<?> convertToNativeJavaType(PrimitiveType primitiveType, Comparable val) {
|
||||||
@@ -408,7 +419,7 @@ public class ParquetUtils extends BaseFileUtils {
|
|||||||
return BigDecimal.valueOf((Long) val, scale);
|
return BigDecimal.valueOf((Long) val, scale);
|
||||||
} else if (val instanceof Binary) {
|
} else if (val instanceof Binary) {
|
||||||
// NOTE: Unscaled number is stored in BE format (most significant byte is 0th)
|
// 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 {
|
} else {
|
||||||
throw new UnsupportedOperationException(String.format("Unsupported value type (%s)", val.getClass().getName()));
|
throw new UnsupportedOperationException(String.format("Unsupported value type (%s)", val.getClass().getName()));
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -24,14 +24,21 @@ import org.apache.hudi.common.util.Base64CodecUtil;
|
|||||||
/**
|
/**
|
||||||
* A stateful Hoodie object ID representing any table column.
|
* 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 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;
|
private final byte[] hash;
|
||||||
|
|
||||||
public ColumnID(final String message) {
|
public ColumnIndexID(final String column) {
|
||||||
this.hash = HashID.hash(message, ID_COLUMN_HASH_SIZE);
|
this.column = column;
|
||||||
|
this.hash = HashID.hash(column, ID_COLUMN_HASH_SIZE);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return column;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -24,14 +24,21 @@ import org.apache.hudi.common.util.Base64CodecUtil;
|
|||||||
/**
|
/**
|
||||||
* Hoodie object ID representing any file.
|
* 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 Type TYPE = Type.FILE;
|
||||||
private static final HashID.Size ID_FILE_HASH_SIZE = HashID.Size.BITS_128;
|
private static final HashID.Size ID_FILE_HASH_SIZE = HashID.Size.BITS_128;
|
||||||
|
private final String fileName;
|
||||||
private final byte[] hash;
|
private final byte[] hash;
|
||||||
|
|
||||||
public FileID(final String message) {
|
public FileIndexID(final String fileName) {
|
||||||
this.hash = HashID.hash(message, ID_FILE_HASH_SIZE);
|
this.fileName = fileName;
|
||||||
|
this.hash = HashID.hash(fileName, ID_FILE_HASH_SIZE);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return fileName;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -24,9 +24,10 @@ import org.apache.hudi.exception.HoodieNotSupportedException;
|
|||||||
import java.io.Serializable;
|
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;
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
@@ -50,6 +51,13 @@ public abstract class HoodieID implements Serializable {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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.
|
* Get the number of bits representing this ID in memory.
|
||||||
* <p>
|
* <p>
|
||||||
@@ -74,7 +82,7 @@ public abstract class HoodieID implements Serializable {
|
|||||||
public abstract String toString();
|
public abstract String toString();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
* Get the Base64 encoded version of the ID.
|
||||||
*/
|
*/
|
||||||
public String asBase64EncodedString() {
|
public String asBase64EncodedString() {
|
||||||
throw new HoodieNotSupportedException("Unsupported hash for " + getType());
|
throw new HoodieNotSupportedException("Unsupported hash for " + getType());
|
||||||
@@ -24,14 +24,21 @@ import org.apache.hudi.common.util.Base64CodecUtil;
|
|||||||
/**
|
/**
|
||||||
* Hoodie object ID representing any partition.
|
* 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 Type TYPE = Type.PARTITION;
|
||||||
private static final HashID.Size ID_PARTITION_HASH_SIZE = HashID.Size.BITS_64;
|
private static final HashID.Size ID_PARTITION_HASH_SIZE = HashID.Size.BITS_64;
|
||||||
|
private final String partition;
|
||||||
private final byte[] hash;
|
private final byte[] hash;
|
||||||
|
|
||||||
public PartitionID(final String message) {
|
public PartitionIndexID(final String partition) {
|
||||||
this.hash = HashID.hash(message, ID_PARTITION_HASH_SIZE);
|
this.partition = partition;
|
||||||
|
this.hash = HashID.hash(partition, ID_PARTITION_HASH_SIZE);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return partition;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -20,6 +20,8 @@ package org.apache.hudi.io.storage;
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
@@ -35,6 +37,10 @@ public interface HoodieFileReader<R extends IndexedRecord> extends AutoCloseable
|
|||||||
|
|
||||||
public Set<String> filterRowKeys(Set<String> candidateRowKeys);
|
public Set<String> filterRowKeys(Set<String> candidateRowKeys);
|
||||||
|
|
||||||
|
default Map<String, R> getRecordsByKeys(List<String> rowKeys) throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
public Iterator<R> getRecordIterator(Schema readerSchema) throws IOException;
|
public Iterator<R> getRecordIterator(Schema readerSchema) throws IOException;
|
||||||
|
|
||||||
default Iterator<R> getRecordIterator() throws IOException {
|
default Iterator<R> getRecordIterator() throws IOException {
|
||||||
|
|||||||
@@ -22,12 +22,14 @@ import java.io.IOException;
|
|||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.HashSet;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.TreeSet;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
@@ -53,8 +55,11 @@ import org.apache.hudi.common.util.ValidationUtils;
|
|||||||
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
|
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
|
public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader<R> {
|
||||||
|
private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
|
||||||
private Path path;
|
private Path path;
|
||||||
private Configuration conf;
|
private Configuration conf;
|
||||||
private HFile.Reader reader;
|
private HFile.Reader reader;
|
||||||
@@ -133,23 +138,50 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Filter keys by availability.
|
||||||
|
* <p>
|
||||||
|
* 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
|
@Override
|
||||||
public Set<String> filterRowKeys(Set candidateRowKeys) {
|
public Set<String> filterRowKeys(Set<String> candidateRowKeys) {
|
||||||
// Current implementation reads all records and filters them. In certain cases, it many be better to:
|
return candidateRowKeys.stream().filter(k -> {
|
||||||
// 1. Scan a limited subset of keys (min/max range of candidateRowKeys)
|
try {
|
||||||
// 2. Lookup keys individually (if the size of candidateRowKeys is much less than the total keys in file)
|
return isKeyAvailable(k);
|
||||||
try {
|
} catch (IOException e) {
|
||||||
List<Pair<String, R>> allRecords = readAllRecords();
|
LOG.error("Failed to check key availability: " + k);
|
||||||
Set<String> rowKeys = new HashSet<>();
|
return false;
|
||||||
allRecords.forEach(t -> {
|
}
|
||||||
if (candidateRowKeys.contains(t.getFirst())) {
|
}).collect(Collectors.toSet());
|
||||||
rowKeys.add(t.getFirst());
|
}
|
||||||
}
|
|
||||||
});
|
@Override
|
||||||
return rowKeys;
|
public Map<String, R> getRecordsByKeys(List<String> rowKeys) throws IOException {
|
||||||
} catch (IOException e) {
|
return filterRecordsImpl(new TreeSet<>(rowKeys));
|
||||||
throw new HoodieIOException("Failed to read row keys from " + path, e);
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Filter records by sorted keys.
|
||||||
|
* <p>
|
||||||
|
* 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<String, R> filterRecordsImpl(TreeSet<String> sortedCandidateRowKeys) throws IOException {
|
||||||
|
HashMap<String, R> filteredRecords = new HashMap<>();
|
||||||
|
for (String key : sortedCandidateRowKeys) {
|
||||||
|
Option<R> record = getRecordByKey(key);
|
||||||
|
if (record.isPresent()) {
|
||||||
|
filteredRecords.put(key, record.get());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
return filteredRecords;
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
|
public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
|
||||||
@@ -246,6 +278,19 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
|||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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
|
@Override
|
||||||
public Option getRecordByKey(String key, Schema readerSchema) throws IOException {
|
public Option getRecordByKey(String key, Schema readerSchema) throws IOException {
|
||||||
byte[] value = null;
|
byte[] value = null;
|
||||||
|
|||||||
@@ -19,6 +19,8 @@
|
|||||||
|
|
||||||
package org.apache.hudi.metadata;
|
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.HoodieMetadataConfig;
|
||||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
@@ -30,7 +32,11 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.util.HoodieTimer;
|
import org.apache.hudi.common.util.HoodieTimer;
|
||||||
import org.apache.hudi.common.util.Option;
|
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.collection.Pair;
|
||||||
|
import org.apache.hudi.common.util.hash.ColumnIndexID;
|
||||||
|
import org.apache.hudi.common.util.hash.FileIndexID;
|
||||||
|
import org.apache.hudi.common.util.hash.PartitionIndexID;
|
||||||
import org.apache.hudi.exception.HoodieMetadataException;
|
import org.apache.hudi.exception.HoodieMetadataException;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
@@ -39,12 +45,15 @@ import org.apache.log4j.LogManager;
|
|||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.TreeSet;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||||
@@ -63,7 +72,9 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
// Directory used for Spillable Map when merging records
|
// Directory used for Spillable Map when merging records
|
||||||
protected final String spillableMapDirectory;
|
protected final String spillableMapDirectory;
|
||||||
|
|
||||||
protected boolean enabled;
|
protected boolean isMetadataTableEnabled;
|
||||||
|
protected boolean isBloomFilterIndexEnabled = false;
|
||||||
|
protected boolean isColumnStatsIndexEnabled = false;
|
||||||
|
|
||||||
protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
|
protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
|
||||||
String dataBasePath, String spillableMapDirectory) {
|
String dataBasePath, String spillableMapDirectory) {
|
||||||
@@ -74,7 +85,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
this.spillableMapDirectory = spillableMapDirectory;
|
this.spillableMapDirectory = spillableMapDirectory;
|
||||||
this.metadataConfig = metadataConfig;
|
this.metadataConfig = metadataConfig;
|
||||||
|
|
||||||
this.enabled = metadataConfig.enabled();
|
this.isMetadataTableEnabled = metadataConfig.enabled();
|
||||||
if (metadataConfig.enableMetrics()) {
|
if (metadataConfig.enableMetrics()) {
|
||||||
this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata")));
|
this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata")));
|
||||||
} else {
|
} else {
|
||||||
@@ -84,16 +95,15 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the list of partitions in the dataset.
|
* Return the list of partitions in the dataset.
|
||||||
*
|
* <p>
|
||||||
* If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of
|
* 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}.
|
* partitions is retrieved directly from the underlying {@code FileSystem}.
|
||||||
*
|
* <p>
|
||||||
* On any errors retrieving the listing from the metadata, defaults to using the file system listings.
|
* On any errors retrieving the listing from the metadata, defaults to using the file system listings.
|
||||||
*
|
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<String> getAllPartitionPaths() throws IOException {
|
public List<String> getAllPartitionPaths() throws IOException {
|
||||||
if (enabled) {
|
if (isMetadataTableEnabled) {
|
||||||
try {
|
try {
|
||||||
return fetchAllPartitionPaths();
|
return fetchAllPartitionPaths();
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
@@ -106,10 +116,10 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the list of files in a partition.
|
* Return the list of files in a partition.
|
||||||
*
|
* <p>
|
||||||
* If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of
|
* 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}.
|
* partitions is retrieved directly from the underlying {@code FileSystem}.
|
||||||
*
|
* <p>
|
||||||
* On any errors retrieving the listing from the metadata, defaults to using the file system listings.
|
* 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
|
* @param partitionPath The absolute path of the partition to list
|
||||||
@@ -117,7 +127,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
@Override
|
@Override
|
||||||
public FileStatus[] getAllFilesInPartition(Path partitionPath)
|
public FileStatus[] getAllFilesInPartition(Path partitionPath)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (enabled) {
|
if (isMetadataTableEnabled) {
|
||||||
try {
|
try {
|
||||||
return fetchAllFilesInPartition(partitionPath);
|
return fetchAllFilesInPartition(partitionPath);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
@@ -132,7 +142,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
@Override
|
@Override
|
||||||
public Map<String, FileStatus[]> getAllFilesInPartitions(List<String> partitions)
|
public Map<String, FileStatus[]> getAllFilesInPartitions(List<String> partitions)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (enabled) {
|
if (isMetadataTableEnabled) {
|
||||||
try {
|
try {
|
||||||
List<Path> partitionPaths = partitions.stream().map(entry -> new Path(entry)).collect(Collectors.toList());
|
List<Path> partitionPaths = partitions.stream().map(entry -> new Path(entry)).collect(Collectors.toList());
|
||||||
Map<String, FileStatus[]> partitionsFilesMap = fetchAllFilesInPartitionPaths(partitionPaths);
|
Map<String, FileStatus[]> partitionsFilesMap = fetchAllFilesInPartitionPaths(partitionPaths);
|
||||||
@@ -146,12 +156,124 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
.getAllFilesInPartitions(partitions);
|
.getAllFilesInPartitions(partitions);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Option<ByteBuffer> getBloomFilter(final String partitionName, final String fileName)
|
||||||
|
throws HoodieMetadataException {
|
||||||
|
if (!isBloomFilterIndexEnabled) {
|
||||||
|
LOG.error("Metadata bloom filter index is disabled!");
|
||||||
|
return Option.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
final Pair<String, String> partitionFileName = Pair.of(partitionName, fileName);
|
||||||
|
Map<Pair<String, String>, 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<Pair<String, String>, ByteBuffer> getBloomFilters(final List<Pair<String, String>> 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<String> partitionIDFileIDSortedStrings = new TreeSet<>();
|
||||||
|
Map<String, Pair<String, String>> 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<String> partitionIDFileIDStrings = new ArrayList<>(partitionIDFileIDSortedStrings);
|
||||||
|
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> hoodieRecordList =
|
||||||
|
getRecordsByKeys(partitionIDFileIDStrings, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath());
|
||||||
|
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_BLOOM_FILTERS_METADATA_STR,
|
||||||
|
(timer.endTimer() / partitionIDFileIDStrings.size())));
|
||||||
|
|
||||||
|
Map<Pair<String, String>, ByteBuffer> partitionFileToBloomFilterMap = new HashMap<>();
|
||||||
|
for (final Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>> entry : hoodieRecordList) {
|
||||||
|
if (entry.getRight().isPresent()) {
|
||||||
|
final Option<HoodieMetadataBloomFilter> 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<Pair<String, String>, HoodieMetadataColumnStats> getColumnStats(final List<Pair<String, String>> partitionNameFileNameList, final String columnName)
|
||||||
|
throws HoodieMetadataException {
|
||||||
|
if (!isColumnStatsIndexEnabled) {
|
||||||
|
LOG.error("Metadata column stats index is disabled!");
|
||||||
|
return Collections.emptyMap();
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, Pair<String, String>> columnStatKeyToFileNameMap = new HashMap<>();
|
||||||
|
TreeSet<String> sortedKeys = new TreeSet<>();
|
||||||
|
final ColumnIndexID columnIndexID = new ColumnIndexID(columnName);
|
||||||
|
for (Pair<String, String> partitionNameFileNamePair : partitionNameFileNameList) {
|
||||||
|
final String columnStatsIndexKey = HoodieMetadataPayload.getColumnStatsIndexKey(
|
||||||
|
new PartitionIndexID(partitionNameFileNamePair.getLeft()),
|
||||||
|
new FileIndexID(partitionNameFileNamePair.getRight()),
|
||||||
|
columnIndexID);
|
||||||
|
sortedKeys.add(columnStatsIndexKey);
|
||||||
|
columnStatKeyToFileNameMap.put(columnStatsIndexKey, partitionNameFileNamePair);
|
||||||
|
}
|
||||||
|
|
||||||
|
List<String> columnStatKeys = new ArrayList<>(sortedKeys);
|
||||||
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
|
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> hoodieRecordList =
|
||||||
|
getRecordsByKeys(columnStatKeys, MetadataPartitionType.COLUMN_STATS.getPartitionPath());
|
||||||
|
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_COLUMN_STATS_METADATA_STR, timer.endTimer()));
|
||||||
|
|
||||||
|
Map<Pair<String, String>, HoodieMetadataColumnStats> fileToColumnStatMap = new HashMap<>();
|
||||||
|
for (final Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>> entry : hoodieRecordList) {
|
||||||
|
if (entry.getRight().isPresent()) {
|
||||||
|
final Option<HoodieMetadataColumnStats> columnStatMetadata =
|
||||||
|
entry.getRight().get().getData().getColumnStatMetadata();
|
||||||
|
if (columnStatMetadata.isPresent()) {
|
||||||
|
if (!columnStatMetadata.get().getIsDeleted()) {
|
||||||
|
ValidationUtils.checkState(columnStatKeyToFileNameMap.containsKey(entry.getLeft()));
|
||||||
|
final Pair<String, String> 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.
|
* Returns a list of all partitions.
|
||||||
*/
|
*/
|
||||||
protected List<String> fetchAllPartitionPaths() throws IOException {
|
protected List<String> fetchAllPartitionPaths() throws IOException {
|
||||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord = getRecordByKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath());
|
Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord = getRecordByKey(RECORDKEY_PARTITION_LIST,
|
||||||
|
MetadataPartitionType.FILES.getPartitionPath());
|
||||||
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer()));
|
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer()));
|
||||||
|
|
||||||
List<String> partitions = Collections.emptyList();
|
List<String> partitions = Collections.emptyList();
|
||||||
@@ -181,7 +303,8 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
}
|
}
|
||||||
|
|
||||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord = getRecordByKey(partitionName, MetadataPartitionType.FILES.partitionPath());
|
Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord = getRecordByKey(partitionName,
|
||||||
|
MetadataPartitionType.FILES.getPartitionPath());
|
||||||
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer()));
|
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer()));
|
||||||
|
|
||||||
FileStatus[] statuses = {};
|
FileStatus[] statuses = {};
|
||||||
@@ -215,7 +338,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
|||||||
|
|
||||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> partitionsFileStatus =
|
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> 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()));
|
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer()));
|
||||||
Map<String, FileStatus[]> result = new HashMap<>();
|
Map<String, FileStatus[]> result = new HashMap<>();
|
||||||
|
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.metadata;
|
package org.apache.hudi.metadata;
|
||||||
|
|
||||||
|
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
@@ -29,8 +30,10 @@ import org.apache.hudi.common.util.collection.Pair;
|
|||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hudi.exception.HoodieMetadataException;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@@ -139,4 +142,21 @@ public class FileSystemBackedTableMetadata implements HoodieTableMetadata {
|
|||||||
public void reset() {
|
public void reset() {
|
||||||
// no-op
|
// no-op
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Option<ByteBuffer> getBloomFilter(final String partitionName, final String fileName)
|
||||||
|
throws HoodieMetadataException {
|
||||||
|
throw new HoodieMetadataException("Unsupported operation: getBloomFilter for " + fileName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Pair<String, String>, ByteBuffer> getBloomFilters(final List<Pair<String, String>> partitionNameFileNameList)
|
||||||
|
throws HoodieMetadataException {
|
||||||
|
throw new HoodieMetadataException("Unsupported operation: getBloomFilters!");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Pair<String, String>, HoodieMetadataColumnStats> getColumnStats(final List<Pair<String, String>> partitionNameFileNameList, final String columnName)
|
||||||
|
throws HoodieMetadataException {
|
||||||
|
throw new HoodieMetadataException("Unsupported operation: getColumnsStats!");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,6 +18,9 @@
|
|||||||
|
|
||||||
package org.apache.hudi.metadata;
|
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.HoodieAvroUtils;
|
||||||
import org.apache.hudi.avro.model.HoodieMetadataRecord;
|
import org.apache.hudi.avro.model.HoodieMetadataRecord;
|
||||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||||
@@ -48,10 +51,6 @@ import org.apache.hudi.exception.HoodieMetadataException;
|
|||||||
import org.apache.hudi.exception.TableNotFoundException;
|
import org.apache.hudi.exception.TableNotFoundException;
|
||||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
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.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
@@ -64,6 +63,7 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -80,8 +80,9 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
|||||||
// should we reuse the open file handles, across calls
|
// should we reuse the open file handles, across calls
|
||||||
private final boolean reuse;
|
private final boolean reuse;
|
||||||
|
|
||||||
// Readers for latest file slice corresponding to file groups in the metadata partition of interest
|
// Readers for the latest file slice corresponding to file groups in the metadata partition
|
||||||
private Map<String, Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader>> partitionReaders = new ConcurrentHashMap<>();
|
private Map<Pair<String, String>, Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader>> partitionReaders =
|
||||||
|
new ConcurrentHashMap<>();
|
||||||
|
|
||||||
public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
|
public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
|
||||||
String datasetBasePath, String spillableMapDirectory) {
|
String datasetBasePath, String spillableMapDirectory) {
|
||||||
@@ -97,7 +98,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
|||||||
|
|
||||||
private void initIfNeeded() {
|
private void initIfNeeded() {
|
||||||
this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(dataBasePath);
|
this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(dataBasePath);
|
||||||
if (!enabled) {
|
if (!isMetadataTableEnabled) {
|
||||||
if (!HoodieTableMetadata.isMetadataTable(metadataBasePath)) {
|
if (!HoodieTableMetadata.isMetadataTable(metadataBasePath)) {
|
||||||
LOG.info("Metadata table is disabled.");
|
LOG.info("Metadata table is disabled.");
|
||||||
}
|
}
|
||||||
@@ -105,14 +106,16 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
|||||||
try {
|
try {
|
||||||
this.metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build();
|
this.metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build();
|
||||||
this.metadataTableConfig = metadataMetaClient.getTableConfig();
|
this.metadataTableConfig = metadataMetaClient.getTableConfig();
|
||||||
|
this.isBloomFilterIndexEnabled = metadataConfig.isBloomFilterIndexEnabled();
|
||||||
|
this.isColumnStatsIndexEnabled = metadataConfig.isColumnStatsIndexEnabled();
|
||||||
} catch (TableNotFoundException e) {
|
} catch (TableNotFoundException e) {
|
||||||
LOG.warn("Metadata table was not found at path " + metadataBasePath);
|
LOG.warn("Metadata table was not found at path " + metadataBasePath);
|
||||||
this.enabled = false;
|
this.isMetadataTableEnabled = false;
|
||||||
this.metadataMetaClient = null;
|
this.metadataMetaClient = null;
|
||||||
this.metadataTableConfig = null;
|
this.metadataTableConfig = null;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e);
|
LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e);
|
||||||
this.enabled = false;
|
this.isMetadataTableEnabled = false;
|
||||||
this.metadataMetaClient = null;
|
this.metadataMetaClient = null;
|
||||||
this.metadataTableConfig = null;
|
this.metadataTableConfig = null;
|
||||||
}
|
}
|
||||||
@@ -125,30 +128,43 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
|||||||
return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue();
|
return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordsByKeys(List<String> keys, String partitionName) {
|
@Override
|
||||||
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers = openReadersIfNeeded(keys.get(0), partitionName);
|
protected List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordsByKeys(List<String> keys,
|
||||||
try {
|
String partitionName) {
|
||||||
List<Long> timings = new ArrayList<>();
|
Map<Pair<String, FileSlice>, List<String>> partitionFileSliceToKeysMap = getPartitionFileSliceToKeysMapping(partitionName, keys);
|
||||||
HoodieFileReader baseFileReader = readers.getKey();
|
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> result = new ArrayList<>();
|
||||||
HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
|
AtomicInteger fileSlicesKeysCount = new AtomicInteger();
|
||||||
|
partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, fileSliceKeys) -> {
|
||||||
|
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers = openReadersIfNeeded(partitionName,
|
||||||
|
partitionFileSlicePair.getRight());
|
||||||
|
try {
|
||||||
|
List<Long> timings = new ArrayList<>();
|
||||||
|
HoodieFileReader baseFileReader = readers.getKey();
|
||||||
|
HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
|
||||||
|
|
||||||
if (baseFileReader == null && logRecordScanner == null) {
|
if (baseFileReader == null && logRecordScanner == null) {
|
||||||
return Collections.emptyList();
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// local map to assist in merging with base file records
|
// local map to assist in merging with base file records
|
||||||
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords = readLogRecords(logRecordScanner, keys, timings);
|
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords = readLogRecords(logRecordScanner,
|
||||||
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> result = readFromBaseAndMergeWithLogRecords(
|
fileSliceKeys, timings);
|
||||||
baseFileReader, keys, logRecords, timings, partitionName);
|
result.addAll(readFromBaseAndMergeWithLogRecords(baseFileReader, fileSliceKeys, logRecords,
|
||||||
LOG.info(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", keys.size(), timings));
|
timings, partitionName));
|
||||||
return result;
|
LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms",
|
||||||
} catch (IOException ioe) {
|
fileSliceKeys.size(), timings));
|
||||||
throw new HoodieIOException("Error merging records from metadata table for " + keys.size() + " key : ", ioe);
|
fileSlicesKeysCount.addAndGet(fileSliceKeys.size());
|
||||||
} finally {
|
} catch (IOException ioe) {
|
||||||
if (!reuse) {
|
throw new HoodieIOException("Error merging records from metadata table for " + keys.size() + " key : ", ioe);
|
||||||
close(partitionName);
|
} finally {
|
||||||
|
if (!reuse) {
|
||||||
|
close(Pair.of(partitionFileSlicePair.getLeft(), partitionFileSlicePair.getRight().getFileId()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
});
|
||||||
|
|
||||||
|
ValidationUtils.checkState(keys.size() == fileSlicesKeysCount.get());
|
||||||
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> readLogRecords(HoodieMetadataMergedLogRecordReader logRecordScanner,
|
private Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> readLogRecords(HoodieMetadataMergedLogRecordReader logRecordScanner,
|
||||||
@@ -190,11 +206,11 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
|||||||
// Retrieve record from base file
|
// Retrieve record from base file
|
||||||
if (baseFileReader != null) {
|
if (baseFileReader != null) {
|
||||||
HoodieTimer readTimer = new HoodieTimer();
|
HoodieTimer readTimer = new HoodieTimer();
|
||||||
|
Map<String, GenericRecord> baseFileRecords = baseFileReader.getRecordsByKeys(keys);
|
||||||
for (String key : keys) {
|
for (String key : keys) {
|
||||||
readTimer.startTimer();
|
readTimer.startTimer();
|
||||||
Option<GenericRecord> baseRecord = baseFileReader.getRecordByKey(key);
|
if (baseFileRecords.containsKey(key)) {
|
||||||
if (baseRecord.isPresent()) {
|
hoodieRecord = getRecord(Option.of(baseFileRecords.get(key)), partitionName);
|
||||||
hoodieRecord = getRecord(baseRecord, partitionName);
|
|
||||||
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer()));
|
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer()));
|
||||||
// merge base file record w/ log record if present
|
// merge base file record w/ log record if present
|
||||||
if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) {
|
if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) {
|
||||||
@@ -233,38 +249,52 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* 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<HoodieFileReader, HoodieMetadataMergedLogRecordReader> openReadersIfNeeded(String key, String partitionName) {
|
private Map<Pair<String, FileSlice>, List<String>> getPartitionFileSliceToKeysMapping(final String partitionName, final List<String> keys) {
|
||||||
return partitionReaders.computeIfAbsent(partitionName, k -> {
|
// Metadata is in sync till the latest completed instant on the dataset
|
||||||
try {
|
List<FileSlice> latestFileSlices =
|
||||||
final long baseFileOpenMs;
|
HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName);
|
||||||
final long logScannerOpenMs;
|
|
||||||
HoodieFileReader baseFileReader = null;
|
|
||||||
HoodieMetadataMergedLogRecordReader logRecordScanner = null;
|
|
||||||
|
|
||||||
// Metadata is in sync till the latest completed instant on the dataset
|
Map<Pair<String, FileSlice>, List<String>> partitionFileSliceToKeysMap = new HashMap<>();
|
||||||
|
for (String key : keys) {
|
||||||
|
final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key,
|
||||||
|
latestFileSlices.size()));
|
||||||
|
final Pair<String, FileSlice> partitionNameFileSlicePair = Pair.of(partitionName, slice);
|
||||||
|
partitionFileSliceToKeysMap.computeIfAbsent(partitionNameFileSlicePair, k -> new ArrayList<>()).add(key);
|
||||||
|
}
|
||||||
|
return partitionFileSliceToKeysMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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<HoodieFileReader, HoodieMetadataMergedLogRecordReader> openReadersIfNeeded(String partitionName, FileSlice slice) {
|
||||||
|
return partitionReaders.computeIfAbsent(Pair.of(partitionName, slice.getFileId()), k -> {
|
||||||
|
try {
|
||||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
List<FileSlice> 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
|
// Open base file reader
|
||||||
Pair<HoodieFileReader, Long> baseFileReaderOpenTimePair = getBaseFileReader(slice, timer);
|
Pair<HoodieFileReader, Long> baseFileReaderOpenTimePair = getBaseFileReader(slice, timer);
|
||||||
baseFileReader = baseFileReaderOpenTimePair.getKey();
|
HoodieFileReader baseFileReader = baseFileReaderOpenTimePair.getKey();
|
||||||
baseFileOpenMs = baseFileReaderOpenTimePair.getValue();
|
final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue();
|
||||||
|
|
||||||
// Open the log record scanner using the log files from the latest file slice
|
// Open the log record scanner using the log files from the latest file slice
|
||||||
Pair<HoodieMetadataMergedLogRecordReader, Long> logRecordScannerOpenTimePair = getLogRecordScanner(slice,
|
Pair<HoodieMetadataMergedLogRecordReader, Long> logRecordScannerOpenTimePair = getLogRecordScanner(slice, partitionName);
|
||||||
partitionName);
|
HoodieMetadataMergedLogRecordReader logRecordScanner = logRecordScannerOpenTimePair.getKey();
|
||||||
logRecordScanner = logRecordScannerOpenTimePair.getKey();
|
final long logScannerOpenMs = logRecordScannerOpenTimePair.getValue();
|
||||||
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);
|
return Pair.of(baseFileReader, logRecordScanner);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieIOException("Error opening readers for metadata table partition " + partitionName, e);
|
throw new HoodieIOException("Error opening readers for metadata table partition " + partitionName, e);
|
||||||
@@ -382,14 +412,20 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
for (String partitionName : partitionReaders.keySet()) {
|
for (Pair<String, String> partitionFileSlicePair : partitionReaders.keySet()) {
|
||||||
close(partitionName);
|
close(partitionFileSlicePair);
|
||||||
}
|
}
|
||||||
partitionReaders.clear();
|
partitionReaders.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void close(String partitionName) {
|
/**
|
||||||
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> 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<String, String> partitionFileSlicePair) {
|
||||||
|
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
|
||||||
|
partitionReaders.remove(partitionFileSlicePair);
|
||||||
if (readers != null) {
|
if (readers != null) {
|
||||||
try {
|
try {
|
||||||
if (readers.getKey() != null) {
|
if (readers.getKey() != null) {
|
||||||
@@ -405,7 +441,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public boolean enabled() {
|
public boolean enabled() {
|
||||||
return enabled;
|
return isMetadataTableEnabled;
|
||||||
}
|
}
|
||||||
|
|
||||||
public SerializableConfiguration getHadoopConf() {
|
public SerializableConfiguration getHadoopConf() {
|
||||||
|
|||||||
@@ -116,7 +116,7 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
|
|||||||
* @param key Key of the record to retrieve
|
* @param key Key of the record to retrieve
|
||||||
* @return {@code HoodieRecord} if key was found else {@code Option.empty()}
|
* @return {@code HoodieRecord} if key was found else {@code Option.empty()}
|
||||||
*/
|
*/
|
||||||
public List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordByKey(String key) {
|
public synchronized List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordByKey(String key) {
|
||||||
return Collections.singletonList(Pair.of(key, Option.ofNullable((HoodieRecord) records.get(key))));
|
return Collections.singletonList(Pair.of(key, Option.ofNullable((HoodieRecord) records.get(key))));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -41,6 +41,8 @@ public class HoodieMetadataMetrics implements Serializable {
|
|||||||
// Metric names
|
// Metric names
|
||||||
public static final String LOOKUP_PARTITIONS_STR = "lookup_partitions";
|
public static final String LOOKUP_PARTITIONS_STR = "lookup_partitions";
|
||||||
public static final String LOOKUP_FILES_STR = "lookup_files";
|
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 SCAN_STR = "scan";
|
||||||
public static final String BASEFILE_READ_STR = "basefile_read";
|
public static final String BASEFILE_READ_STR = "basefile_read";
|
||||||
public static final String INITIALIZE_STR = "initialize";
|
public static final String INITIALIZE_STR = "initialize";
|
||||||
@@ -77,7 +79,7 @@ public class HoodieMetadataMetrics implements Serializable {
|
|||||||
Map<String, String> stats = new HashMap<>();
|
Map<String, String> stats = new HashMap<>();
|
||||||
|
|
||||||
// Total size of the metadata and count of base/log files
|
// Total size of the metadata and count of base/log files
|
||||||
for (String metadataPartition : MetadataPartitionType.all()) {
|
for (String metadataPartition : MetadataPartitionType.allPaths()) {
|
||||||
List<FileSlice> latestSlices = fsView.getLatestFileSlices(metadataPartition).collect(Collectors.toList());
|
List<FileSlice> latestSlices = fsView.getLatestFileSlices(metadataPartition).collect(Collectors.toList());
|
||||||
|
|
||||||
// Total size of the metadata and count of base/log files
|
// Total size of the metadata and count of base/log files
|
||||||
|
|||||||
@@ -18,13 +18,22 @@
|
|||||||
|
|
||||||
package org.apache.hudi.metadata;
|
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.HoodieMetadataFileInfo;
|
||||||
import org.apache.hudi.avro.model.HoodieMetadataRecord;
|
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.HoodieKey;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
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.hudi.exception.HoodieMetadataException;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
@@ -36,7 +45,9 @@ import org.apache.hadoop.fs.Path;
|
|||||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@@ -46,36 +57,67 @@ import java.util.stream.Stream;
|
|||||||
import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST;
|
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.
|
* MetadataTable records are persisted with the schema defined in HoodieMetadata.avsc.
|
||||||
*
|
* This class represents the payload for the MetadataTable.
|
||||||
* The type of the entry is determined by the "type" saved within the record. The following types of entries are saved:
|
* <p>
|
||||||
*
|
* This single metadata payload is shared by all the partitions under the metadata table.
|
||||||
* 1. List of partitions: There is a single such record
|
* The partition specific records are determined by the field "type" saved within the record.
|
||||||
* key="__all_partitions__"
|
* The following types are supported:
|
||||||
*
|
* <p>
|
||||||
* 2. List of files in a Partition: There is one such record for each partition
|
* METADATA_TYPE_PARTITION_LIST (1):
|
||||||
* key=Partition name
|
* -- List of all partitions. There is a single such record
|
||||||
*
|
* -- key = @{@link HoodieTableMetadata.RECORDKEY_PARTITION_LIST}
|
||||||
* During compaction on the table, the deletions are merged with additions and hence pruned.
|
* <p>
|
||||||
*
|
* METADATA_TYPE_FILE_LIST (2):
|
||||||
* Metadata Table records are saved with the schema defined in HoodieMetadata.avsc. This class encapsulates the
|
* -- List of all files in a partition. There is one such record for each partition
|
||||||
* HoodieMetadataRecord for ease of operations.
|
* -- key = partition name
|
||||||
|
* <p>
|
||||||
|
* METADATA_TYPE_COLUMN_STATS (3):
|
||||||
|
* -- This is an index for column stats in the table
|
||||||
|
* <p>
|
||||||
|
* METADATA_TYPE_BLOOM_FILTER (4):
|
||||||
|
* -- This is an index for base file bloom filters. This is a map of FileID to its BloomFilter byte[].
|
||||||
|
* <p>
|
||||||
|
* During compaction on the table, the deletions are merged with additions and hence records are pruned.
|
||||||
*/
|
*/
|
||||||
public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadataPayload> {
|
public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadataPayload> {
|
||||||
|
|
||||||
|
// 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
|
// HoodieMetadata schema field ids
|
||||||
public static final String KEY_FIELD_NAME = HoodieHFileReader.KEY_FIELD_NAME;
|
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_TYPE = "type";
|
||||||
public static final String SCHEMA_FIELD_NAME_METADATA = "filesystemMetadata";
|
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
|
// HoodieMetadata bloom filter payload field ids
|
||||||
// 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 String FIELD_IS_DELETED = "isDeleted";
|
||||||
private static final int PARTITION_LIST = 1;
|
private static final String BLOOM_FILTER_FIELD_TYPE = "type";
|
||||||
private static final int FILE_LIST = 2;
|
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 String key = null;
|
||||||
private int type = 0;
|
private int type = 0;
|
||||||
private Map<String, HoodieMetadataFileInfo> filesystemMetadata = null;
|
private Map<String, HoodieMetadataFileInfo> filesystemMetadata = null;
|
||||||
|
private HoodieMetadataBloomFilter bloomFilterMetadata = null;
|
||||||
|
private HoodieMetadataColumnStats columnStatMetadata = null;
|
||||||
|
|
||||||
public HoodieMetadataPayload(GenericRecord record, Comparable<?> orderingVal) {
|
public HoodieMetadataPayload(GenericRecord record, Comparable<?> orderingVal) {
|
||||||
this(Option.of(record));
|
this(Option.of(record));
|
||||||
@@ -94,13 +136,60 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
|||||||
filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted")));
|
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<String, HoodieMetadataFileInfo> filesystemMetadata) {
|
private HoodieMetadataPayload(String key, int type, Map<String, HoodieMetadataFileInfo> 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<String, HoodieMetadataFileInfo> filesystemMetadata,
|
||||||
|
HoodieMetadataBloomFilter metadataBloomFilter,
|
||||||
|
HoodieMetadataColumnStats columnStats) {
|
||||||
this.key = key;
|
this.key = key;
|
||||||
this.type = type;
|
this.type = type;
|
||||||
this.filesystemMetadata = filesystemMetadata;
|
this.filesystemMetadata = filesystemMetadata;
|
||||||
|
this.bloomFilterMetadata = metadataBloomFilter;
|
||||||
|
this.columnStatMetadata = columnStats;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -110,55 +199,97 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
|||||||
*/
|
*/
|
||||||
public static HoodieRecord<HoodieMetadataPayload> createPartitionListRecord(List<String> partitions) {
|
public static HoodieRecord<HoodieMetadataPayload> createPartitionListRecord(List<String> partitions) {
|
||||||
Map<String, HoodieMetadataFileInfo> fileInfo = new HashMap<>();
|
Map<String, HoodieMetadataFileInfo> 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());
|
HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.getPartitionPath());
|
||||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), PARTITION_LIST, fileInfo);
|
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_PARTITION_LIST,
|
||||||
|
fileInfo);
|
||||||
return new HoodieRecord<>(key, payload);
|
return new HoodieRecord<>(key, payload);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create and return a {@code HoodieMetadataPayload} to save list of files within a partition.
|
* Create and return a {@code HoodieMetadataPayload} to save list of files within a partition.
|
||||||
*
|
*
|
||||||
* @param partition The name of the 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 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
|
* @param filesDeleted List of files which have been deleted from this partition
|
||||||
*/
|
*/
|
||||||
public static HoodieRecord<HoodieMetadataPayload> createPartitionFilesRecord(String partition,
|
public static HoodieRecord<HoodieMetadataPayload> createPartitionFilesRecord(String partition,
|
||||||
Option<Map<String, Long>> filesAdded, Option<List<String>> filesDeleted) {
|
Option<Map<String, Long>> filesAdded,
|
||||||
|
Option<List<String>> filesDeleted) {
|
||||||
Map<String, HoodieMetadataFileInfo> fileInfo = new HashMap<>();
|
Map<String, HoodieMetadataFileInfo> fileInfo = new HashMap<>();
|
||||||
filesAdded.ifPresent(
|
filesAdded.ifPresent(
|
||||||
m -> m.forEach((filename, size) -> fileInfo.put(filename, new HoodieMetadataFileInfo(size, false))));
|
m -> m.forEach((filename, size) -> fileInfo.put(filename, new HoodieMetadataFileInfo(size, false))));
|
||||||
filesDeleted.ifPresent(
|
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());
|
HoodieKey key = new HoodieKey(partition, MetadataPartitionType.FILES.getPartitionPath());
|
||||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), FILE_LIST, fileInfo);
|
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_FILE_LIST, fileInfo);
|
||||||
return new HoodieRecord<>(key, payload);
|
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<HoodieMetadataPayload> 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
|
@Override
|
||||||
public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) {
|
public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) {
|
||||||
ValidationUtils.checkArgument(previousRecord.type == type,
|
ValidationUtils.checkArgument(previousRecord.type == type,
|
||||||
"Cannot combine " + previousRecord.type + " with " + type);
|
"Cannot combine " + previousRecord.type + " with " + type);
|
||||||
|
|
||||||
Map<String, HoodieMetadataFileInfo> combinedFileInfo = null;
|
|
||||||
|
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case PARTITION_LIST:
|
case METADATA_TYPE_PARTITION_LIST:
|
||||||
case FILE_LIST:
|
case METADATA_TYPE_FILE_LIST:
|
||||||
combinedFileInfo = combineFilesystemMetadata(previousRecord);
|
Map<String, HoodieMetadataFileInfo> combinedFileInfo = combineFilesystemMetadata(previousRecord);
|
||||||
break;
|
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:
|
default:
|
||||||
throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type);
|
throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return new HoodieMetadataPayload(key, type, combinedFileInfo);
|
private HoodieMetadataBloomFilter combineBloomFilterMetadata(HoodieMetadataPayload previousRecord) {
|
||||||
|
return this.bloomFilterMetadata;
|
||||||
|
}
|
||||||
|
|
||||||
|
private HoodieMetadataColumnStats combineColumnStatsMetadatat(HoodieMetadataPayload previousRecord) {
|
||||||
|
return this.columnStatMetadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRecord, Schema schema) throws IOException {
|
public Option<IndexedRecord> 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);
|
HoodieRecordPayload combinedPayload = preCombine(anotherPayload);
|
||||||
return combinedPayload.getInsertValue(schema);
|
return combinedPayload.getInsertValue(schema);
|
||||||
}
|
}
|
||||||
@@ -169,7 +300,8 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
|||||||
return Option.empty();
|
return Option.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata);
|
HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata, bloomFilterMetadata,
|
||||||
|
columnStatMetadata);
|
||||||
return Option.of(record);
|
return Option.of(record);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -187,6 +319,28 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
|||||||
return filterFileInfoEntries(true).map(Map.Entry::getKey).sorted().collect(Collectors.toList());
|
return filterFileInfoEntries(true).map(Map.Entry::getKey).sorted().collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the bloom filter metadata from this payload.
|
||||||
|
*/
|
||||||
|
public Option<HoodieMetadataBloomFilter> getBloomFilterMetadata() {
|
||||||
|
if (bloomFilterMetadata == null) {
|
||||||
|
return Option.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
return Option.of(bloomFilterMetadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the bloom filter metadata from this payload.
|
||||||
|
*/
|
||||||
|
public Option<HoodieMetadataColumnStats> getColumnStatMetadata() {
|
||||||
|
if (columnStatMetadata == null) {
|
||||||
|
return Option.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
return Option.of(columnStatMetadata);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the files added as part of this record.
|
* Returns the files added as part of this record.
|
||||||
*/
|
*/
|
||||||
@@ -235,6 +389,70 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
|||||||
return combinedFileInfo;
|
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<Comparable> 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<HoodieRecord> createColumnStatsRecords(
|
||||||
|
String partitionName, Collection<HoodieColumnRangeMetadata<Comparable>> 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
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {");
|
final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {");
|
||||||
@@ -242,6 +460,20 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
|||||||
sb.append(SCHEMA_FIELD_NAME_TYPE + "=").append(type).append(", ");
|
sb.append(SCHEMA_FIELD_NAME_TYPE + "=").append(type).append(", ");
|
||||||
sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", ");
|
sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", ");
|
||||||
sb.append("deletions=").append(Arrays.toString(getDeletions().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('}');
|
sb.append('}');
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.metadata;
|
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.HoodieMetadataConfig;
|
||||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
@@ -25,9 +26,12 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
|||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.Path;
|
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.IOException;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
@@ -104,6 +108,38 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
|
|||||||
*/
|
*/
|
||||||
Map<String, FileStatus[]> getAllFilesInPartitions(List<String> partitionPaths) throws IOException;
|
Map<String, FileStatus[]> getAllFilesInPartitions(List<String> 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<ByteBuffer> 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<Pair<String, String>, ByteBuffer> getBloomFilters(final List<Pair<String, String>> 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<Pair<String, String>, HoodieMetadataColumnStats> getColumnStats(final List<Pair<String, String>> partitionNameFileNameList, final String columnName)
|
||||||
|
throws HoodieMetadataException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the instant time to which the metadata is synced w.r.t data timeline.
|
* Get the instant time to which the metadata is synced w.r.t data timeline.
|
||||||
*/
|
*/
|
||||||
|
|||||||
@@ -18,29 +18,44 @@
|
|||||||
|
|
||||||
package org.apache.hudi.metadata;
|
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.HoodieCleanMetadata;
|
||||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
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.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.FileSlice;
|
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.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.HoodieRecord;
|
||||||
|
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
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.HoodieActiveTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||||
import org.apache.hudi.common.util.Option;
|
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.ValidationUtils;
|
||||||
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieMetadataException;
|
import org.apache.hudi.exception.HoodieMetadataException;
|
||||||
|
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
@@ -62,12 +77,17 @@ public class HoodieTableMetadataUtil {
|
|||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class);
|
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.
|
* process should be running.
|
||||||
*
|
*
|
||||||
* @param basePath base path of the dataset
|
* @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) {
|
public static void deleteMetadataTable(String basePath, HoodieEngineContext context) {
|
||||||
final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
||||||
@@ -79,14 +99,53 @@ public class HoodieTableMetadataUtil {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
|
||||||
|
HoodieEngineContext context, List<MetadataPartitionType> enabledPartitionTypes,
|
||||||
|
HoodieCommitMetadata commitMetadata, HoodieTableMetaClient dataMetaClient,
|
||||||
|
boolean isMetaIndexColumnStatsForAllColumns, String instantTime) {
|
||||||
|
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
|
||||||
|
final HoodieData<HoodieRecord> filesPartitionRecordsRDD = context.parallelize(
|
||||||
|
convertMetadataToFilesPartitionRecords(commitMetadata, instantTime), 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD);
|
||||||
|
|
||||||
|
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) {
|
||||||
|
final List<HoodieRecord> metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(commitMetadata,
|
||||||
|
dataMetaClient, instantTime);
|
||||||
|
if (!metadataBloomFilterRecords.isEmpty()) {
|
||||||
|
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = context.parallelize(metadataBloomFilterRecords, 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) {
|
||||||
|
final List<HoodieRecord> metadataColumnStats = convertMetadataToColumnStatsRecords(commitMetadata, context,
|
||||||
|
dataMetaClient, isMetaIndexColumnStatsForAllColumns, instantTime);
|
||||||
|
if (!metadataColumnStats.isEmpty()) {
|
||||||
|
final HoodieData<HoodieRecord> 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.
|
* Finds all new files/partitions created as part of commit and creates metadata table records for them.
|
||||||
*
|
*
|
||||||
* @param commitMetadata
|
* @param commitMetadata - Commit action metadata
|
||||||
* @param instantTime
|
* @param instantTime - Commit action instant time
|
||||||
* @return a list of metadata table records
|
* @return List of metadata table records
|
||||||
*/
|
*/
|
||||||
public static List<HoodieRecord> convertMetadataToRecords(HoodieCommitMetadata commitMetadata, String instantTime) {
|
public static List<HoodieRecord> convertMetadataToFilesPartitionRecords(HoodieCommitMetadata commitMetadata,
|
||||||
|
String instantTime) {
|
||||||
List<HoodieRecord> records = new LinkedList<>();
|
List<HoodieRecord> records = new LinkedList<>();
|
||||||
List<String> allPartitions = new LinkedList<>();
|
List<String> allPartitions = new LinkedList<>();
|
||||||
commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> {
|
commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> {
|
||||||
@@ -124,6 +183,102 @@ public class HoodieTableMetadataUtil {
|
|||||||
return records;
|
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<HoodieRecord> convertMetadataToBloomFilterRecords(HoodieCommitMetadata commitMetadata,
|
||||||
|
HoodieTableMetaClient dataMetaClient,
|
||||||
|
String instantTime) {
|
||||||
|
List<HoodieRecord> records = new LinkedList<>();
|
||||||
|
commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> {
|
||||||
|
final String partition = partitionStatName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionStatName;
|
||||||
|
Map<String, Long> 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<IndexedRecord> 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<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
|
||||||
|
HoodieEngineContext engineContext, List<MetadataPartitionType> enabledPartitionTypes,
|
||||||
|
HoodieCleanMetadata cleanMetadata, HoodieTableMetaClient dataMetaClient, String instantTime) {
|
||||||
|
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
|
||||||
|
final HoodieData<HoodieRecord> filesPartitionRecordsRDD = engineContext.parallelize(
|
||||||
|
convertMetadataToFilesPartitionRecords(cleanMetadata, instantTime), 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD);
|
||||||
|
|
||||||
|
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) {
|
||||||
|
final List<HoodieRecord> metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(cleanMetadata,
|
||||||
|
engineContext, instantTime);
|
||||||
|
if (!metadataBloomFilterRecords.isEmpty()) {
|
||||||
|
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) {
|
||||||
|
final List<HoodieRecord> metadataColumnStats = convertMetadataToColumnStatsRecords(cleanMetadata, engineContext,
|
||||||
|
dataMetaClient);
|
||||||
|
if (!metadataColumnStats.isEmpty()) {
|
||||||
|
final HoodieData<HoodieRecord> 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.
|
* Finds all files that were deleted as part of a clean and creates metadata table records for them.
|
||||||
*
|
*
|
||||||
@@ -131,7 +286,8 @@ public class HoodieTableMetadataUtil {
|
|||||||
* @param instantTime
|
* @param instantTime
|
||||||
* @return a list of metadata table records
|
* @return a list of metadata table records
|
||||||
*/
|
*/
|
||||||
public static List<HoodieRecord> convertMetadataToRecords(HoodieCleanMetadata cleanMetadata, String instantTime) {
|
public static List<HoodieRecord> convertMetadataToFilesPartitionRecords(HoodieCleanMetadata cleanMetadata,
|
||||||
|
String instantTime) {
|
||||||
List<HoodieRecord> records = new LinkedList<>();
|
List<HoodieRecord> records = new LinkedList<>();
|
||||||
int[] fileDeleteCount = {0};
|
int[] fileDeleteCount = {0};
|
||||||
cleanMetadata.getPartitionMetadata().forEach((partitionName, partitionMetadata) -> {
|
cleanMetadata.getPartitionMetadata().forEach((partitionName, partitionMetadata) -> {
|
||||||
@@ -150,48 +306,187 @@ public class HoodieTableMetadataUtil {
|
|||||||
return records;
|
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<HoodieRecord> convertMetadataToBloomFilterRecords(HoodieCleanMetadata cleanMetadata,
|
||||||
|
HoodieEngineContext engineContext,
|
||||||
|
String instantTime) {
|
||||||
|
List<Pair<String, String>> deleteFileList = new ArrayList<>();
|
||||||
|
cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> {
|
||||||
|
// Files deleted from a partition
|
||||||
|
List<String> 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<HoodieRecord> convertMetadataToColumnStatsRecords(HoodieCleanMetadata cleanMetadata,
|
||||||
|
HoodieEngineContext engineContext,
|
||||||
|
HoodieTableMetaClient datasetMetaClient) {
|
||||||
|
List<Pair<String, String>> deleteFileList = new ArrayList<>();
|
||||||
|
cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> {
|
||||||
|
// Files deleted from a partition
|
||||||
|
List<String> deletedFiles = partitionMetadata.getDeletePathPatterns();
|
||||||
|
deletedFiles.forEach(entry -> deleteFileList.add(Pair.of(partition, entry)));
|
||||||
|
});
|
||||||
|
|
||||||
|
List<String> 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<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
|
||||||
|
HoodieEngineContext engineContext, List<MetadataPartitionType> enabledPartitionTypes,
|
||||||
|
HoodieActiveTimeline metadataTableTimeline, HoodieRestoreMetadata restoreMetadata,
|
||||||
|
HoodieTableMetaClient dataMetaClient, String instantTime, Option<String> lastSyncTs) {
|
||||||
|
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
|
||||||
|
final Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
|
||||||
|
final Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
|
||||||
|
|
||||||
|
processRestoreMetadata(metadataTableTimeline, restoreMetadata,
|
||||||
|
partitionToAppendedFiles, partitionToDeletedFiles, lastSyncTs);
|
||||||
|
|
||||||
|
final HoodieData<HoodieRecord> filesPartitionRecordsRDD = engineContext.parallelize(
|
||||||
|
convertFilesToFilesPartitionRecords(partitionToDeletedFiles,
|
||||||
|
partitionToAppendedFiles, instantTime, "Restore"), 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD);
|
||||||
|
|
||||||
|
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) {
|
||||||
|
final List<HoodieRecord> metadataBloomFilterRecords = convertFilesToBloomFilterRecords(
|
||||||
|
engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime);
|
||||||
|
if (!metadataBloomFilterRecords.isEmpty()) {
|
||||||
|
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) {
|
||||||
|
final List<HoodieRecord> metadataColumnStats = convertFilesToColumnStatsRecords(
|
||||||
|
engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime);
|
||||||
|
if (!metadataColumnStats.isEmpty()) {
|
||||||
|
final HoodieData<HoodieRecord> 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
|
* Aggregates all files deleted and appended to from all rollbacks associated with a restore operation then
|
||||||
* creates metadata table records for them.
|
* creates metadata table records for them.
|
||||||
*
|
*
|
||||||
* @param restoreMetadata
|
* @param restoreMetadata - Restore action metadata
|
||||||
* @param instantTime
|
|
||||||
* @return a list of metadata table records
|
* @return a list of metadata table records
|
||||||
*/
|
*/
|
||||||
public static List<HoodieRecord> convertMetadataToRecords(HoodieActiveTimeline metadataTableTimeline,
|
private static void processRestoreMetadata(HoodieActiveTimeline metadataTableTimeline,
|
||||||
HoodieRestoreMetadata restoreMetadata, String instantTime, Option<String> lastSyncTs) {
|
HoodieRestoreMetadata restoreMetadata,
|
||||||
Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
|
Map<String, Map<String, Long>> partitionToAppendedFiles,
|
||||||
Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
|
Map<String, List<String>> partitionToDeletedFiles,
|
||||||
|
Option<String> lastSyncTs) {
|
||||||
restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> {
|
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<HoodieRecord> convertMetadataToRecords(HoodieActiveTimeline metadataTableTimeline,
|
/**
|
||||||
HoodieRollbackMetadata rollbackMetadata, String instantTime,
|
* Convert rollback action metadata to metadata table records.
|
||||||
Option<String> lastSyncTs, boolean wasSynced) {
|
*/
|
||||||
|
public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
|
||||||
|
HoodieEngineContext engineContext, List<MetadataPartitionType> enabledPartitionTypes,
|
||||||
|
HoodieActiveTimeline metadataTableTimeline, HoodieRollbackMetadata rollbackMetadata,
|
||||||
|
HoodieTableMetaClient dataMetaClient, String instantTime, Option<String> lastSyncTs, boolean wasSynced) {
|
||||||
|
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
|
||||||
|
|
||||||
Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
|
|
||||||
Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
|
Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
|
||||||
processRollbackMetadata(metadataTableTimeline, rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs);
|
Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
|
||||||
|
List<HoodieRecord> filesPartitionRecords = convertMetadataToRollbackRecords(metadataTableTimeline, rollbackMetadata,
|
||||||
|
partitionToDeletedFiles, partitionToAppendedFiles, instantTime, lastSyncTs, wasSynced);
|
||||||
|
final HoodieData<HoodieRecord> rollbackRecordsRDD = engineContext.parallelize(filesPartitionRecords, 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.FILES, rollbackRecordsRDD);
|
||||||
|
|
||||||
|
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) {
|
||||||
|
final List<HoodieRecord> metadataBloomFilterRecords = convertFilesToBloomFilterRecords(
|
||||||
|
engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime);
|
||||||
|
if (!metadataBloomFilterRecords.isEmpty()) {
|
||||||
|
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) {
|
||||||
|
final List<HoodieRecord> metadataColumnStats = convertFilesToColumnStatsRecords(
|
||||||
|
engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime);
|
||||||
|
if (!metadataColumnStats.isEmpty()) {
|
||||||
|
final HoodieData<HoodieRecord> metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1);
|
||||||
|
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return partitionToRecordsMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert rollback action metadata to files partition records.
|
||||||
|
*/
|
||||||
|
private static List<HoodieRecord> convertMetadataToRollbackRecords(HoodieActiveTimeline metadataTableTimeline,
|
||||||
|
HoodieRollbackMetadata rollbackMetadata,
|
||||||
|
Map<String, List<String>> partitionToDeletedFiles,
|
||||||
|
Map<String, Map<String, Long>> partitionToAppendedFiles,
|
||||||
|
String instantTime,
|
||||||
|
Option<String> lastSyncTs, boolean wasSynced) {
|
||||||
|
processRollbackMetadata(metadataTableTimeline, rollbackMetadata, partitionToDeletedFiles,
|
||||||
|
partitionToAppendedFiles, lastSyncTs);
|
||||||
if (!wasSynced) {
|
if (!wasSynced) {
|
||||||
// Since the instant-being-rolled-back was never committed to the metadata table, the files added there
|
// 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.
|
// need not be deleted. For MOR Table, the rollback appends logBlocks so we need to keep the appended files.
|
||||||
partitionToDeletedFiles.clear();
|
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}.
|
* Extracts information about the deleted and append files from the {@code HoodieRollbackMetadata}.
|
||||||
*
|
* <p>
|
||||||
* During a rollback files may be deleted (COW, MOR) or rollback blocks be appended (MOR only) to files. This
|
* 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.
|
* function will extract this change file for each partition.
|
||||||
* @param metadataTableTimeline Current timeline of the Metdata Table
|
*
|
||||||
* @param rollbackMetadata {@code HoodieRollbackMetadata}
|
* @param metadataTableTimeline Current timeline of the Metadata Table
|
||||||
* @param partitionToDeletedFiles The {@code Map} to fill with files deleted per partition.
|
* @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.
|
* @param partitionToAppendedFiles The {@code Map} to fill with files appended per partition and their sizes.
|
||||||
*/
|
*/
|
||||||
private static void processRollbackMetadata(HoodieActiveTimeline metadataTableTimeline,
|
private static void processRollbackMetadata(HoodieActiveTimeline metadataTableTimeline,
|
||||||
@@ -268,9 +563,12 @@ public class HoodieTableMetadataUtil {
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private static List<HoodieRecord> convertFilesToRecords(Map<String, List<String>> partitionToDeletedFiles,
|
/**
|
||||||
Map<String, Map<String, Long>> partitionToAppendedFiles, String instantTime,
|
* Convert rollback action metadata to files partition records.
|
||||||
String operation) {
|
*/
|
||||||
|
private static List<HoodieRecord> convertFilesToFilesPartitionRecords(Map<String, List<String>> partitionToDeletedFiles,
|
||||||
|
Map<String, Map<String, Long>> partitionToAppendedFiles,
|
||||||
|
String instantTime, String operation) {
|
||||||
List<HoodieRecord> records = new LinkedList<>();
|
List<HoodieRecord> records = new LinkedList<>();
|
||||||
int[] fileChangeCount = {0, 0}; // deletes, appends
|
int[] fileChangeCount = {0, 0}; // deletes, appends
|
||||||
|
|
||||||
@@ -309,9 +607,88 @@ public class HoodieTableMetadataUtil {
|
|||||||
return records;
|
return records;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert rollback action metadata to bloom filter index records.
|
||||||
|
*/
|
||||||
|
private static List<HoodieRecord> convertFilesToBloomFilterRecords(HoodieEngineContext engineContext,
|
||||||
|
HoodieTableMetaClient dataMetaClient,
|
||||||
|
Map<String, List<String>> partitionToDeletedFiles,
|
||||||
|
Map<String, Map<String, Long>> partitionToAppendedFiles,
|
||||||
|
String instantTime) {
|
||||||
|
List<HoodieRecord> 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<IndexedRecord> 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<HoodieRecord> convertFilesToColumnStatsRecords(HoodieEngineContext engineContext,
|
||||||
|
HoodieTableMetaClient datasetMetaClient,
|
||||||
|
Map<String, List<String>> partitionToDeletedFiles,
|
||||||
|
Map<String, Map<String, Long>> partitionToAppendedFiles,
|
||||||
|
String instantTime) {
|
||||||
|
List<HoodieRecord> records = new LinkedList<>();
|
||||||
|
List<String> 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.
|
* Map a record key to a file group in partition of interest.
|
||||||
*
|
* <p>
|
||||||
* Note: For hashing, the algorithm is same as String.hashCode() but is being defined here as hashCode()
|
* 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.
|
* implementation is not guaranteed by the JVM to be consistent across JVM versions and implementations.
|
||||||
*
|
*
|
||||||
@@ -339,7 +716,7 @@ public class HoodieTableMetadataUtil {
|
|||||||
*/
|
*/
|
||||||
public static List<FileSlice> getPartitionLatestMergedFileSlices(HoodieTableMetaClient metaClient, String partition) {
|
public static List<FileSlice> getPartitionLatestMergedFileSlices(HoodieTableMetaClient metaClient, String partition) {
|
||||||
LOG.info("Loading latest merged file slices for metadata table partition " + 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,12 +724,33 @@ public class HoodieTableMetadataUtil {
|
|||||||
* returned is sorted in the correct order of file group name.
|
* returned is sorted in the correct order of file group name.
|
||||||
*
|
*
|
||||||
* @param metaClient - Instance of {@link HoodieTableMetaClient}.
|
* @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.
|
* @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.
|
* @return List of latest file slices for all file groups in a given partition.
|
||||||
*/
|
*/
|
||||||
public static List<FileSlice> getPartitionLatestFileSlices(HoodieTableMetaClient metaClient, String partition) {
|
public static List<FileSlice> getPartitionLatestFileSlices(HoodieTableMetaClient metaClient,
|
||||||
|
Option<HoodieTableFileSystemView> fsView, String partition) {
|
||||||
LOG.info("Loading latest file slices for metadata table partition " + partition);
|
LOG.info("Loading latest file slices for metadata table partition " + partition);
|
||||||
return getPartitionFileSlices(metaClient, partition, false);
|
return getPartitionFileSlices(metaClient, fsView, partition, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get metadata table file system view.
|
||||||
|
*
|
||||||
|
* @param metaClient - Metadata table meta client
|
||||||
|
* @return Filesystem view for the metadata table
|
||||||
|
*/
|
||||||
|
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.
|
||||||
|
HoodieTimeline timeline = metaClient.getActiveTimeline();
|
||||||
|
if (timeline.empty()) {
|
||||||
|
final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||||
|
HoodieActiveTimeline.createNewInstantTime());
|
||||||
|
timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails);
|
||||||
|
}
|
||||||
|
return new HoodieTableFileSystemView(metaClient, timeline);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -366,27 +764,161 @@ public class HoodieTableMetadataUtil {
|
|||||||
* slices without any merging, and this is needed for the writers.
|
* 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.
|
* @return List of latest file slices for all file groups in a given partition.
|
||||||
*/
|
*/
|
||||||
private static List<FileSlice> getPartitionFileSlices(HoodieTableMetaClient metaClient, String partition,
|
private static List<FileSlice> getPartitionFileSlices(HoodieTableMetaClient metaClient,
|
||||||
|
Option<HoodieTableFileSystemView> fileSystemView,
|
||||||
|
String partition,
|
||||||
boolean mergeFileSlices) {
|
boolean mergeFileSlices) {
|
||||||
// If there are no commits on the metadata table then the table's
|
HoodieTableFileSystemView fsView = fileSystemView.orElse(getFileSystemView(metaClient));
|
||||||
// default FileSystemView will not return any file slices even
|
|
||||||
// though we may have initialized them.
|
|
||||||
HoodieTimeline timeline = metaClient.getActiveTimeline();
|
|
||||||
if (timeline.empty()) {
|
|
||||||
final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION,
|
|
||||||
HoodieActiveTimeline.createNewInstantTime());
|
|
||||||
timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails);
|
|
||||||
}
|
|
||||||
|
|
||||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, timeline);
|
|
||||||
Stream<FileSlice> fileSliceStream;
|
Stream<FileSlice> fileSliceStream;
|
||||||
if (mergeFileSlices) {
|
if (mergeFileSlices) {
|
||||||
fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn(
|
fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn(
|
||||||
partition, timeline.filterCompletedInstants().lastInstant().get().getTimestamp());
|
partition, metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get().getTimestamp());
|
||||||
} else {
|
} else {
|
||||||
fileSliceStream = fsView.getLatestFileSlices(partition);
|
fileSliceStream = fsView.getLatestFileSlices(partition);
|
||||||
}
|
}
|
||||||
return fileSliceStream.sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())).collect(Collectors.toList());
|
return fileSliceStream.sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())).collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static List<HoodieRecord> convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata,
|
||||||
|
HoodieEngineContext engineContext,
|
||||||
|
HoodieTableMetaClient dataMetaClient,
|
||||||
|
boolean isMetaIndexColumnStatsForAllColumns,
|
||||||
|
String instantTime) {
|
||||||
|
|
||||||
|
try {
|
||||||
|
List<HoodieWriteStat> 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<HoodieRecord> createColumnStatsFromWriteStats(HoodieEngineContext engineContext,
|
||||||
|
HoodieTableMetaClient datasetMetaClient,
|
||||||
|
List<HoodieWriteStat> allWriteStats,
|
||||||
|
boolean isMetaIndexColumnStatsForAllColumns) throws Exception {
|
||||||
|
if (allWriteStats.isEmpty()) {
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
List<HoodieWriteStat> 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<String> 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<String> getLatestColumns(HoodieTableMetaClient datasetMetaClient) {
|
||||||
|
return getLatestColumns(datasetMetaClient, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Stream<HoodieRecord> translateWriteStatToColumnStats(HoodieWriteStat writeStat,
|
||||||
|
HoodieTableMetaClient datasetMetaClient,
|
||||||
|
List<String> latestColumns) {
|
||||||
|
return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, latestColumns, false);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Stream<HoodieRecord> getColumnStats(final String partitionPath, final String filePathWithPartition,
|
||||||
|
HoodieTableMetaClient datasetMetaClient,
|
||||||
|
List<String> 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<HoodieColumnRangeMetadata<Comparable>> 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<Comparable>(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<HoodieTableMetaClient> metaClient,
|
||||||
|
final Option<HoodieTableFileSystemView> fsView,
|
||||||
|
final HoodieMetadataConfig metadataConfig, boolean isBootstrapCompleted) {
|
||||||
|
if (isBootstrapCompleted) {
|
||||||
|
final List<FileSlice> 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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,19 +22,23 @@ import java.util.Arrays;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
public enum MetadataPartitionType {
|
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;
|
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;
|
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.partitionPath = partitionPath;
|
||||||
this.fileIdPrefix = fileIdPrefix;
|
this.fileIdPrefix = fileIdPrefix;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String partitionPath() {
|
public String getPartitionPath() {
|
||||||
return partitionPath;
|
return partitionPath;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -42,7 +46,28 @@ public enum MetadataPartitionType {
|
|||||||
return fileIdPrefix;
|
return fileIdPrefix;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<String> all() {
|
void setFileGroupCount(final int fileGroupCount) {
|
||||||
return Arrays.asList(MetadataPartitionType.FILES.partitionPath());
|
this.fileGroupCount = fileGroupCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getFileGroupCount() {
|
||||||
|
return this.fileGroupCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static List<String> 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()
|
||||||
|
+ "}";
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
Reference in New Issue
Block a user