[HUDI-4250][HUDI-4202] Optimize performance of Column Stats Index reading in Data Skipping (#5746)
We provide an alternative way of fetching Column Stats Index within the reading process to avoid the penalty of a more heavy-weight execution scheduled through a Spark engine.
This commit is contained in:
@@ -38,6 +38,7 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.hadoop.CachingPath;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -62,7 +63,7 @@ import java.util.stream.Collectors;
|
||||
* <li>Query instant/range</li>
|
||||
* </ul>
|
||||
*/
|
||||
public abstract class BaseHoodieTableFileIndex {
|
||||
public abstract class BaseHoodieTableFileIndex {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BaseHoodieTableFileIndex.class);
|
||||
|
||||
@@ -166,6 +167,11 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
.collect(Collectors.toMap(e -> e.getKey().path, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
public int getFileSlicesCount() {
|
||||
return cachedAllInputFileSlices.values().stream()
|
||||
.mapToInt(List::size).sum();
|
||||
}
|
||||
|
||||
protected List<PartitionPath> getAllQueryPartitionPaths() {
|
||||
List<String> queryRelativePartitionPaths = queryPaths.stream()
|
||||
.map(path -> FSUtils.getRelativePartitionPath(new Path(basePath), path))
|
||||
@@ -349,10 +355,10 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
|
||||
Path fullPartitionPath(String basePath) {
|
||||
if (!path.isEmpty()) {
|
||||
return new Path(basePath, path);
|
||||
return new CachingPath(basePath, path);
|
||||
}
|
||||
|
||||
return new Path(basePath);
|
||||
return new CachingPath(basePath);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -187,6 +187,26 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("Comma-separated list of columns for which column stats index will be built. If not set, all columns will be indexed");
|
||||
|
||||
public static final String COLUMN_STATS_INDEX_PROCESSING_MODE_IN_MEMORY = "in-memory";
|
||||
public static final String COLUMN_STATS_INDEX_PROCESSING_MODE_ENGINE = "engine";
|
||||
|
||||
public static final ConfigProperty<String> COLUMN_STATS_INDEX_PROCESSING_MODE_OVERRIDE = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".index.column.stats.processing.mode.override")
|
||||
.noDefaultValue()
|
||||
.withValidValues(COLUMN_STATS_INDEX_PROCESSING_MODE_IN_MEMORY, COLUMN_STATS_INDEX_PROCESSING_MODE_ENGINE)
|
||||
.sinceVersion("0.12.0")
|
||||
.withDocumentation("By default Column Stats Index is automatically determining whether it should be read and processed either"
|
||||
+ "'in-memory' (w/in executing process) or using Spark (on a cluster), based on some factors like the size of the Index "
|
||||
+ "and how many columns are read. This config allows to override this behavior.");
|
||||
|
||||
public static final ConfigProperty<Integer> COLUMN_STATS_INDEX_IN_MEMORY_PROJECTION_THRESHOLD = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".index.column.stats.inMemory.projection.threshold")
|
||||
.defaultValue(100000)
|
||||
.sinceVersion("0.12.0")
|
||||
.withDocumentation("When reading Column Stats Index, if the size of the expected resulting projection is below the in-memory"
|
||||
+ " threshold (counted by the # of rows), it will be attempted to be loaded \"in-memory\" (ie not using the execution engine"
|
||||
+ " like Spark, Flink, etc). If the value is above the threshold execution engine will be used to compose the projection.");
|
||||
|
||||
public static final ConfigProperty<String> BLOOM_FILTER_INDEX_FOR_COLUMNS = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".index.bloom.filter.column.list")
|
||||
.noDefaultValue()
|
||||
@@ -246,6 +266,14 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
return StringUtils.split(getString(COLUMN_STATS_INDEX_FOR_COLUMNS), CONFIG_VALUES_DELIMITER);
|
||||
}
|
||||
|
||||
public String getColumnStatsIndexProcessingModeOverride() {
|
||||
return getString(COLUMN_STATS_INDEX_PROCESSING_MODE_OVERRIDE);
|
||||
}
|
||||
|
||||
public Integer getColumnStatsIndexInMemoryProjectionThreshold() {
|
||||
return getIntOrDefault(COLUMN_STATS_INDEX_IN_MEMORY_PROJECTION_THRESHOLD);
|
||||
}
|
||||
|
||||
public List<String> getColumnsEnabledForBloomFilterIndex() {
|
||||
return StringUtils.split(getString(BLOOM_FILTER_INDEX_FOR_COLUMNS), CONFIG_VALUES_DELIMITER);
|
||||
}
|
||||
|
||||
@@ -168,7 +168,7 @@ public class FileSystemBackedTableMetadata implements HoodieTableMetadata {
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes, String partitionName) {
|
||||
public HoodieData<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes, String partitionName, boolean shouldLoadInMemory) {
|
||||
throw new HoodieMetadataException("Unsupported operation: getRecordsByKeyPrefixes!");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -29,6 +29,7 @@ import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.data.HoodieListData;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.function.SerializableFunction;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
@@ -143,10 +144,11 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes,
|
||||
String partitionName) {
|
||||
String partitionName,
|
||||
boolean shouldLoadInMemory) {
|
||||
// Sort the columns so that keys are looked up in order
|
||||
List<String> sortedkeyPrefixes = new ArrayList<>(keyPrefixes);
|
||||
Collections.sort(sortedkeyPrefixes);
|
||||
List<String> sortedKeyPrefixes = new ArrayList<>(keyPrefixes);
|
||||
Collections.sort(sortedKeyPrefixes);
|
||||
|
||||
// NOTE: Since we partition records to a particular file-group by full key, we will have
|
||||
// to scan all file-groups for all key-prefixes as each of these might contain some
|
||||
@@ -154,44 +156,44 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
List<FileSlice> partitionFileSlices =
|
||||
HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName);
|
||||
|
||||
return engineContext.parallelize(partitionFileSlices)
|
||||
.flatMap(
|
||||
(SerializableFunction<FileSlice, Iterator<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>>>) fileSlice -> {
|
||||
// NOTE: Since this will be executed by executors, we can't access previously cached
|
||||
// readers, and therefore have to always open new ones
|
||||
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
|
||||
openReaders(partitionName, fileSlice);
|
||||
try {
|
||||
List<Long> timings = new ArrayList<>();
|
||||
return (shouldLoadInMemory ? HoodieListData.lazy(partitionFileSlices) : engineContext.parallelize(partitionFileSlices))
|
||||
.flatMap((SerializableFunction<FileSlice, Iterator<HoodieRecord<HoodieMetadataPayload>>>) fileSlice -> {
|
||||
// NOTE: Since this will be executed by executors, we can't access previously cached
|
||||
// readers, and therefore have to always open new ones
|
||||
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
|
||||
openReaders(partitionName, fileSlice);
|
||||
|
||||
HoodieFileReader baseFileReader = readers.getKey();
|
||||
HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
|
||||
try {
|
||||
List<Long> timings = new ArrayList<>();
|
||||
|
||||
if (baseFileReader == null && logRecordScanner == null) {
|
||||
// TODO: what do we do if both does not exist? should we throw an exception and let caller do the fallback ?
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
HoodieFileReader baseFileReader = readers.getKey();
|
||||
HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
|
||||
|
||||
boolean fullKeys = false;
|
||||
|
||||
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords =
|
||||
readLogRecords(logRecordScanner, sortedkeyPrefixes, fullKeys, timings);
|
||||
|
||||
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> mergedRecords =
|
||||
readFromBaseAndMergeWithLogRecords(baseFileReader, sortedkeyPrefixes, fullKeys, logRecords, timings, partitionName);
|
||||
|
||||
LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms",
|
||||
sortedkeyPrefixes.size(), timings));
|
||||
|
||||
return mergedRecords.iterator();
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error merging records from metadata table for " + sortedkeyPrefixes.size() + " key : ", ioe);
|
||||
} finally {
|
||||
closeReader(readers);
|
||||
}
|
||||
if (baseFileReader == null && logRecordScanner == null) {
|
||||
// TODO: what do we do if both does not exist? should we throw an exception and let caller do the fallback ?
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
)
|
||||
.map(keyRecordPair -> keyRecordPair.getValue().orElse(null))
|
||||
|
||||
boolean fullKeys = false;
|
||||
|
||||
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords =
|
||||
readLogRecords(logRecordScanner, sortedKeyPrefixes, fullKeys, timings);
|
||||
|
||||
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> mergedRecords =
|
||||
readFromBaseAndMergeWithLogRecords(baseFileReader, sortedKeyPrefixes, fullKeys, logRecords, timings, partitionName);
|
||||
|
||||
LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms",
|
||||
sortedKeyPrefixes.size(), timings));
|
||||
|
||||
return mergedRecords.stream()
|
||||
.map(keyRecordPair -> keyRecordPair.getValue().orElse(null))
|
||||
.iterator();
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error merging records from metadata table for " + sortedKeyPrefixes.size() + " key : ", ioe);
|
||||
} finally {
|
||||
closeReader(readers);
|
||||
}
|
||||
})
|
||||
.filter(Objects::nonNull);
|
||||
}
|
||||
|
||||
|
||||
@@ -170,7 +170,8 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
|
||||
* @return {@link HoodieData} of {@link HoodieRecord}s with records matching the passed in key prefixes.
|
||||
*/
|
||||
HoodieData<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes,
|
||||
String partitionName);
|
||||
String partitionName,
|
||||
boolean shouldLoadInMemory);
|
||||
|
||||
/**
|
||||
* Get the instant time to which the metadata is synced w.r.t data timeline.
|
||||
|
||||
Reference in New Issue
Block a user