1
0

[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:
Manoj Govindassamy
2022-02-03 04:42:48 -08:00
committed by GitHub
parent d681824982
commit 5927bdd1c0
49 changed files with 2304 additions and 522 deletions

View File

@@ -25,7 +25,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.io.HoodieKeyLookupHandle;
import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult;
import org.apache.hudi.io.HoodieKeyLookupResult;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.function.Function2;
@@ -40,7 +40,7 @@ import scala.Tuple2;
* Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the actual files.
*/
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;
@@ -52,12 +52,12 @@ public class HoodieBloomIndexCheckFunction
}
@Override
public Iterator<List<KeyLookupResult>> call(Integer partition,
Iterator<Tuple2<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
public Iterator<List<HoodieKeyLookupResult>> call(Integer partition,
Iterator<Tuple2<String, HoodieKey>> 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;
@@ -70,9 +70,9 @@ public class HoodieBloomIndexCheckFunction
}
@Override
protected List<HoodieKeyLookupHandle.KeyLookupResult> computeNext() {
protected List<HoodieKeyLookupResult> computeNext() {
List<HoodieKeyLookupHandle.KeyLookupResult> ret = new ArrayList<>();
List<HoodieKeyLookupResult> ret = new ArrayList<>();
try {
// process one file in each go.
while (inputItr.hasNext()) {
@@ -88,7 +88,7 @@ public class HoodieBloomIndexCheckFunction
}
// if continue on current file
if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) {
if (keyLookupHandle.getPartitionPathFileIDPair().equals(partitionPathFilePair)) {
keyLookupHandle.addKey(recordKey);
} else {
// do the actual checking of file & break out

View File

@@ -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() {
}
}
}

View File

@@ -29,20 +29,19 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.data.HoodieJavaPairRDD;
import org.apache.hudi.data.HoodieJavaRDD;
import org.apache.hudi.io.HoodieKeyLookupResult;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaRDD;
import scala.Tuple2;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import scala.Tuple2;
/**
* Helper for {@link HoodieBloomIndex} containing Spark-specific logic.
*/
@@ -70,26 +69,38 @@ public class SparkHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper {
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
HoodieJavaRDD.getJavaRDD(fileComparisonPairs)
.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());
LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${"
+ 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,
config.getBloomIndexKeysPerBucket());
fileComparisonsRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
.repartitionAndSortWithinPartitions(partitioner).map(Tuple2::_2);
keyLookupResultRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
.repartitionAndSortWithinPartitions(partitioner)
.map(Tuple2::_2)
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true);
} else {
fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism);
keyLookupResultRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism)
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true);
}
return HoodieJavaPairRDD.of(fileComparisonsRDD.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true)
.flatMap(List::iterator).filter(lr -> lr.getMatchingRecordKeys().size() > 0)
return HoodieJavaPairRDD.of(keyLookupResultRDD.flatMap(List::iterator)
.filter(lr -> lr.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream()
.map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()),
new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())))

View File

@@ -200,7 +200,7 @@ public class ColumnStatsIndexHelper {
indexRow.add(minMaxValue.getLeft()); // min
indexRow.add(minMaxValue.getRight()); // max
indexRow.add(colMetadata.getNumNulls());
indexRow.add(colMetadata.getNullCount());
});
return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(indexRow));

View File

@@ -25,14 +25,13 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.data.HoodieJavaRDD;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.metrics.DistributedRegistry;
@@ -43,6 +42,7 @@ import org.apache.spark.api.java.JavaRDD;
import java.io.IOException;
import java.util.List;
import java.util.Map;
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(enabled, "Metadata table cannot be committed to as it is not enabled");
JavaRDD<HoodieRecord> records = (JavaRDD<HoodieRecord>) hoodieDataRecords.get();
JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName, 1);
HoodieData<HoodieRecord> preppedRecords = prepRecords(partitionRecordsMap);
JavaRDD<HoodieRecord> preppedRecordRDD = HoodieJavaRDD.getJavaRDD(preppedRecords);
try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {
if (canTriggerTableService) {
@@ -150,7 +151,7 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant);
metadataMetaClient.reloadActiveTimeline();
}
List<WriteStatus> statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect();
List<WriteStatus> statuses = writeClient.upsertPreppedRecords(preppedRecordRDD, instantTime).collect();
statuses.forEach(writeStatus -> {
if (writeStatus.hasErrors()) {
throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime);
@@ -168,20 +169,4 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
// Update total size of the metadata and count of base/log files
metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata));
}
/**
* Tag each record with the location in the given partition.
*
* The record is tagged with respective file slice's location based on its record key.
*/
private JavaRDD<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;
});
}
}

View File

@@ -191,6 +191,8 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
// trigger couple of upserts
doWriteOperation(testTable, "0000005");
doWriteOperation(testTable, "0000006");
doWriteOperation(testTable, "0000007");
doCleanAndValidate(testTable, "0000008", Arrays.asList("0000007"));
validateMetadata(testTable, true);
}
@@ -222,7 +224,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
testTable.doWriteOperation("0000003", UPSERT, emptyList(), asList("p1", "p2"), 1, true);
syncTableMetadata(writeConfig);
List<String> partitions = metadataWriter(writeConfig).metadata().getAllPartitionPaths();
List<String> partitions = metadataWriter(writeConfig).getTableMetadata().getAllPartitionPaths();
assertFalse(partitions.contains(nonPartitionDirectory),
"Must not contain the non-partition " + nonPartitionDirectory);
assertTrue(partitions.contains("p1"), "Must contain partition p1");
@@ -345,6 +347,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
if (tableType == MERGE_ON_READ) {
doCompaction(testTable, "0000004");
}
doCleanAndValidate(testTable, "0000005", Arrays.asList("0000001"));
validateMetadata(testTable, emptyList(), true);
}
@@ -380,6 +383,32 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
assertEquals(tableMetadata.getLatestCompactionTime().get(), "0000003001");
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testTableOperationsWithMetadataIndex(HoodieTableType tableType) throws Exception {
initPath();
HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false)
.withIndexConfig(HoodieIndexConfig.newBuilder()
.bloomIndexBucketizedChecking(false)
.build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.enable(true)
.withMetadataIndexBloomFilter(true)
.withMetadataIndexBloomFilterFileGroups(4)
.withMetadataIndexColumnStats(true)
.withMetadataIndexBloomFilterFileGroups(2)
.withMetadataIndexForAllColumns(true)
.build())
.build();
init(tableType, writeConfig);
testTableOperationsForMetaIndexImpl(writeConfig);
}
private void testTableOperationsForMetaIndexImpl(final HoodieWriteConfig writeConfig) throws Exception {
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
testTableOperationsImpl(engineContext, writeConfig);
}
/**
* Tests that virtual key configs are honored in base files after compaction in metadata table.
*
@@ -619,7 +648,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
// Compaction should not be triggered yet. Let's verify no base file
// and few log files available.
List<FileSlice> fileSlices = table.getSliceView()
.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
.getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList());
if (fileSlices.isEmpty()) {
throw new IllegalStateException("LogFile slices are not available!");
}
@@ -709,7 +738,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
.withBasePath(metadataMetaClient.getBasePath())
.withLogFilePaths(logFilePaths)
.withLatestInstantTime(latestCommitTimestamp)
.withPartition(MetadataPartitionType.FILES.partitionPath())
.withPartition(MetadataPartitionType.FILES.getPartitionPath())
.withReaderSchema(schema)
.withMaxMemorySizeInBytes(100000L)
.withBufferSize(4096)
@@ -739,7 +768,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table, boolean enableMetaFields) throws IOException {
table.getHoodieView().sync();
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()) {
throw new IllegalStateException("Base file not available!");
}
@@ -1058,10 +1087,20 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exception {
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false)
.withRollbackUsingMarkers(false).build();
testTableOperationsImpl(engineContext, writeConfig);
}
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).build())) {
/**
* Test all major table operations with the given table, config and context.
*
* @param engineContext - Engine context
* @param writeConfig - Write config
* @throws IOException
*/
private void testTableOperationsImpl(HoodieSparkEngineContext engineContext, HoodieWriteConfig writeConfig) throws IOException {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) {
// Write 1 (Bulk insert)
String newCommitTime = "0000001";
List<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 + ".totalDuration"));
assertTrue(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_STR + ".count") >= 1L);
final String prefix = MetadataPartitionType.FILES.partitionPath() + ".";
final String prefix = MetadataPartitionType.FILES.getPartitionPath() + ".";
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_BASE_FILES));
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_LOG_FILES));
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE));
@@ -1931,7 +1970,10 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
// in the .hoodie folder.
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
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
// 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());
metadataTablePartitions.forEach(partition -> {
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.size() <= 1, "Should have a single latest file slice");
assertTrue(latestSlices.size() <= numFileVersions, "Should limit file slice to "
+ numFileVersions + " but was " + latestSlices.size());
assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count()
<= metadataEnabledPartitionTypes.get(partition).getFileGroupCount(), "Should have a single latest base file per file group");
assertTrue(latestSlices.size()
<= metadataEnabledPartitionTypes.get(partition).getFileGroupCount(), "Should have a single latest file slice per file group");
assertTrue(latestSlices.size()
<= (numFileVersions * metadataEnabledPartitionTypes.get(partition).getFileGroupCount()), "Should limit file slice to "
+ numFileVersions + " per file group, but was " + latestSlices.size());
});
LOG.info("Validation time=" + timer.endTimer());

View File

@@ -247,7 +247,7 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
// Compaction should not be triggered yet. Let's verify no base file
// and few log files available.
List<FileSlice> fileSlices = table.getSliceView()
.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
.getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList());
if (fileSlices.isEmpty()) {
throw new IllegalStateException("LogFile slices are not available!");
}
@@ -322,7 +322,7 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
.withBasePath(metadataMetaClient.getBasePath())
.withLogFilePaths(logFilePaths)
.withLatestInstantTime(latestCommitTimestamp)
.withPartition(MetadataPartitionType.FILES.partitionPath())
.withPartition(MetadataPartitionType.FILES.getPartitionPath())
.withReaderSchema(schema)
.withMaxMemorySizeInBytes(100000L)
.withBufferSize(4096)
@@ -351,7 +351,7 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table) throws IOException {
table.getHoodieView().sync();
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()) {
throw new IllegalStateException("Base file not available!");
}

View File

@@ -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.testutils.HoodieMetadataTestTable;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
@@ -84,27 +85,22 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
init(tableType, true);
}
public void init(HoodieTableType tableType, HoodieWriteConfig writeConfig) throws IOException {
init(tableType, Option.of(writeConfig), true, false, false, false);
}
public void init(HoodieTableType tableType, boolean enableMetadataTable) throws IOException {
init(tableType, enableMetadataTable, true, false, false);
}
public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan, boolean enableMetrics, boolean
validateMetadataPayloadStateConsistency) throws IOException {
this.tableType = tableType;
initPath();
initSparkContexts("TestHoodieMetadata");
initFileSystem();
fs.mkdirs(new Path(basePath));
initTimelineService();
initMetaClient(tableType);
initTestDataGenerator();
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, enableMetrics,
enableFullScan, true, validateMetadataPayloadStateConsistency).build();
initWriteConfigAndMetatableWriter(writeConfig, enableMetadataTable);
validateMetadataPayloadStateConsistency) throws IOException {
init(tableType, Option.empty(), enableMetadataTable, enableFullScan, enableMetrics,
validateMetadataPayloadStateConsistency);
}
public void init(HoodieTableType tableType, HoodieWriteConfig writeConfig) throws IOException {
public void init(HoodieTableType tableType, Option<HoodieWriteConfig> writeConfig, boolean enableMetadataTable,
boolean enableFullScan, boolean enableMetrics, boolean validateMetadataPayloadStateConsistency) throws IOException {
this.tableType = tableType;
initPath();
initSparkContexts("TestHoodieMetadata");
@@ -114,8 +110,12 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
initMetaClient(tableType);
initTestDataGenerator();
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
this.writeConfig = writeConfig;
initWriteConfigAndMetatableWriter(writeConfig, writeConfig.isMetadataTableEnabled());
this.writeConfig = writeConfig.isPresent()
? writeConfig.get() : getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true,
enableMetadataTable, enableMetrics, enableFullScan, true,
validateMetadataPayloadStateConsistency)
.build();
initWriteConfigAndMetatableWriter(this.writeConfig, enableMetadataTable);
}
protected void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) {

View File

@@ -18,9 +18,13 @@
package org.apache.hudi.index.bloom;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.functional.TestHoodieMetadataBase;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory;
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -32,14 +36,10 @@ import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.data.HoodieJavaPairRDD;
import org.apache.hudi.data.HoodieJavaRDD;
import org.apache.hudi.io.HoodieKeyLookupHandle;
import org.apache.hudi.index.HoodieIndexUtils;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterEach;
@@ -48,6 +48,7 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import scala.Tuple2;
import java.nio.file.Paths;
import java.util.Arrays;
@@ -59,8 +60,6 @@ import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import scala.Tuple2;
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -69,14 +68,14 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieBloomIndex extends HoodieClientTestHarness {
public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBloomIndex.class, "/exampleSchema.avsc", true);
private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}";
public static Stream<Arguments> configParams() {
Object[][] data =
new Object[][] {{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}};
new Object[][]{{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}};
return Stream.of(data).map(Arguments::of);
}
@@ -99,6 +98,10 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
.withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning)
.bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking)
.bloomIndexKeysPerBucket(2).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.withMetadataIndexBloomFilter(false)
.withMetadataIndexColumnStats(false)
.build())
.build();
}
@@ -134,7 +137,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
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
assertEquals(0, filesList.size());
@@ -143,7 +146,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
.withInserts("2015/03/12", "3", record1)
.withInserts("2015/03/12", "4", record2, record3, record4);
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
assertEquals(4, filesList.size());
if (rangePruning) {
@@ -241,9 +244,9 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId));
List<String> results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids,
new Path(Paths.get(basePath, partition, filename).toString()));
List<String> results = HoodieIndexUtils.filterKeysFromFile(
new Path(Paths.get(basePath, partition, filename).toString()), uuids, hadoopConf);
assertEquals(results.size(), 2);
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));

View File

@@ -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
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
// 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
assertEquals(0, filesList.size());
@@ -118,7 +118,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
.withInserts("2015/03/12", "3", record1)
.withInserts("2015/03/12", "4", record2, record3, record4);
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
assertEquals(4, filesList.size());
Map<String, BloomIndexFileInfo> filesMap = toFileMap(filesList);

View File

@@ -59,7 +59,6 @@ import org.apache.hudi.metadata.FileSystemBackedTableMetadata;
import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.MetadataPartitionType;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
@@ -680,7 +679,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
// in the .hoodie folder.
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
false, false);
Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size());
Assertions.assertEquals(metadataWriter.getEnabledPartitionTypes().size(), metadataTablePartitions.size());
// Metadata table should automatically compact and clean
// versions are +1 as autoClean / compaction happens end of commits