[HUDI-3664] Fixing Column Stats Index composition (#5181)
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
@@ -40,7 +40,6 @@ import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.io.HoodieRangeInfoHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -55,6 +54,7 @@ import static java.util.stream.Collectors.mapping;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import static org.apache.hudi.common.util.CollectionUtils.isNullOrEmpty;
|
||||
import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions;
|
||||
import static org.apache.hudi.metadata.HoodieMetadataPayload.unwrapStatisticValueWrapper;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions;
|
||||
import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS;
|
||||
|
||||
@@ -206,7 +206,7 @@ public class HoodieBloomIndex extends HoodieIndex<Object, Object> {
|
||||
* @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) {
|
||||
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");
|
||||
|
||||
@@ -221,15 +221,16 @@ public class HoodieBloomIndex extends HoodieIndex<Object, Object> {
|
||||
return Stream.empty();
|
||||
}
|
||||
try {
|
||||
Map<Pair<String, String>, HoodieMetadataColumnStats> fileToColumnStatsMap = hoodieTable
|
||||
.getMetadataTable().getColumnStats(partitionFileNameList, keyField);
|
||||
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()
|
||||
// NOTE: Here we assume that the type of the primary key field is string
|
||||
(String) unwrapStatisticValueWrapper(entry.getValue().getMinValue()),
|
||||
(String) unwrapStatisticValueWrapper(entry.getValue().getMaxValue())
|
||||
)));
|
||||
}
|
||||
return result.stream();
|
||||
|
||||
@@ -18,6 +18,10 @@
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
@@ -56,11 +60,6 @@ import org.apache.hudi.exception.HoodieAppendException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -76,8 +75,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.accumulateColumnRanges;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.aggregateColumnStats;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.collectColumnRangeMetadata;
|
||||
|
||||
/**
|
||||
* IO Operation to append data onto an existing file.
|
||||
@@ -349,26 +347,21 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
|
||||
if (config.isMetadataColumnStatsIndexEnabled()) {
|
||||
final List<Schema.Field> fieldsToIndex;
|
||||
if (!StringUtils.isNullOrEmpty(config.getColumnsEnabledForColumnStatsIndex())) {
|
||||
if (StringUtils.isNullOrEmpty(config.getColumnsEnabledForColumnStatsIndex())) {
|
||||
// If column stats index is enabled but columns not configured then we assume that all columns should be indexed
|
||||
fieldsToIndex = writeSchemaWithMetaFields.getFields();
|
||||
} else {
|
||||
Set<String> columnsToIndex = Stream.of(config.getColumnsEnabledForColumnStatsIndex().split(","))
|
||||
.map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet());
|
||||
|
||||
fieldsToIndex = writeSchemaWithMetaFields.getFields().stream()
|
||||
.filter(field -> columnsToIndex.contains(field.name())).collect(Collectors.toList());
|
||||
} else {
|
||||
// if column stats index is enabled but columns not configured then we assume that all columns should be indexed
|
||||
fieldsToIndex = writeSchemaWithMetaFields.getFields();
|
||||
}
|
||||
|
||||
Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangeMap = stat.getRecordsStats().isPresent()
|
||||
? stat.getRecordsStats().get().getStats() : new HashMap<>();
|
||||
final String filePath = stat.getPath();
|
||||
// initialize map of column name to map of stats name to stats value
|
||||
Map<String, Map<String, Object>> columnToStats = new HashMap<>();
|
||||
fieldsToIndex.forEach(field -> columnToStats.putIfAbsent(field.name(), new HashMap<>()));
|
||||
// collect stats for columns at once per record and keep iterating through every record to eventually find col stats for all fields.
|
||||
recordList.forEach(record -> aggregateColumnStats(record, fieldsToIndex, columnToStats, config.isConsistentLogicalTimestampEnabled()));
|
||||
fieldsToIndex.forEach(field -> accumulateColumnRanges(field, filePath, columnRangeMap, columnToStats));
|
||||
stat.setRecordsStats(new HoodieDeltaWriteStat.RecordsStats<>(columnRangeMap));
|
||||
Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangesMetadataMap =
|
||||
collectColumnRangeMetadata(recordList, fieldsToIndex, stat.getPath());
|
||||
|
||||
stat.setRecordsStats(columnRangesMetadataMap);
|
||||
}
|
||||
|
||||
resetWriteCounts();
|
||||
|
||||
Reference in New Issue
Block a user