[HUDI-3825] Fixing Column Stats Index updating sequence (#5267)
This commit is contained in:
@@ -38,6 +38,8 @@ public class HoodieConfig implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieConfig.class);
|
||||
|
||||
protected static final String CONFIG_VALUES_DELIMITER = ",";
|
||||
|
||||
public static HoodieConfig create(FSDataInputStream inputStream) throws IOException {
|
||||
HoodieConfig config = new HoodieConfig();
|
||||
config.props.load(inputStream);
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.common.config;
|
||||
|
||||
import org.apache.hudi.common.engine.EngineType;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
@@ -26,6 +27,7 @@ import javax.annotation.concurrent.Immutable;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
@@ -173,15 +175,6 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
+ "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 column ranges of user data files for all columns under "
|
||||
+ "metadata table key lookups. When enabled, metadata table will have a partition to "
|
||||
+ "store the column ranges and will be used for pruning files during the index lookups. "
|
||||
+ "Only applies if " + ENABLE_METADATA_INDEX_COLUMN_STATS.key() + " is enabled.");
|
||||
|
||||
public static final ConfigProperty<Integer> COLUMN_STATS_INDEX_PARALLELISM = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".index.column.stats.parallelism")
|
||||
.defaultValue(10)
|
||||
@@ -249,16 +242,12 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
return getBooleanOrDefault(ENABLE_METADATA_INDEX_COLUMN_STATS);
|
||||
}
|
||||
|
||||
public boolean isMetadataColumnStatsIndexForAllColumnsEnabled() {
|
||||
return getBooleanOrDefault(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS);
|
||||
public List<String> getColumnsEnabledForColumnStatsIndex() {
|
||||
return StringUtils.split(getString(COLUMN_STATS_INDEX_FOR_COLUMNS), CONFIG_VALUES_DELIMITER);
|
||||
}
|
||||
|
||||
public String getColumnsEnabledForColumnStatsIndex() {
|
||||
return getString(COLUMN_STATS_INDEX_FOR_COLUMNS);
|
||||
}
|
||||
|
||||
public String getColumnsEnabledForBloomFilterIndex() {
|
||||
return getString(BLOOM_FILTER_INDEX_FOR_COLUMNS);
|
||||
public List<String> getColumnsEnabledForBloomFilterIndex() {
|
||||
return StringUtils.split(getString(BLOOM_FILTER_INDEX_FOR_COLUMNS), CONFIG_VALUES_DELIMITER);
|
||||
}
|
||||
|
||||
public int getBloomFilterIndexFileGroupCount() {
|
||||
@@ -353,11 +342,6 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMetadataIndexForAllColumns(boolean enable) {
|
||||
metadataConfig.setValue(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS, String.valueOf(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withColumnStatsIndexForColumns(String columns) {
|
||||
metadataConfig.setValue(COLUMN_STATS_INDEX_FOR_COLUMNS, columns);
|
||||
return this;
|
||||
|
||||
@@ -618,12 +618,18 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
return getLong(TABLE_CHECKSUM);
|
||||
}
|
||||
|
||||
public String getMetadataPartitionsInflight() {
|
||||
return getStringOrDefault(TABLE_METADATA_PARTITIONS_INFLIGHT, StringUtils.EMPTY_STRING);
|
||||
public List<String> getMetadataPartitionsInflight() {
|
||||
return StringUtils.split(
|
||||
getStringOrDefault(TABLE_METADATA_PARTITIONS_INFLIGHT, StringUtils.EMPTY_STRING),
|
||||
CONFIG_VALUES_DELIMITER
|
||||
);
|
||||
}
|
||||
|
||||
public String getMetadataPartitions() {
|
||||
return getStringOrDefault(TABLE_METADATA_PARTITIONS, StringUtils.EMPTY_STRING);
|
||||
public List<String> getMetadataPartitions() {
|
||||
return StringUtils.split(
|
||||
getStringOrDefault(TABLE_METADATA_PARTITIONS, StringUtils.EMPTY_STRING),
|
||||
CONFIG_VALUES_DELIMITER
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -19,12 +19,8 @@
|
||||
package org.apache.hudi.common.util;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -34,10 +30,6 @@ import java.util.stream.Stream;
|
||||
public class StringUtils {
|
||||
|
||||
public static final String EMPTY_STRING = "";
|
||||
private static final Function<String, Set<String>> STRING_TO_SET = (str) -> Stream.of(str.split(","))
|
||||
.map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet());
|
||||
private static final Function<String, List<String>> STRING_TO_LIST = (str) -> Stream.of(str.split(","))
|
||||
.map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
|
||||
|
||||
/**
|
||||
* <p>
|
||||
@@ -114,22 +106,13 @@ public class StringUtils {
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the input string, delimited by comma, to a set of strings.
|
||||
*
|
||||
* @param input
|
||||
* @return
|
||||
* Splits input string, delimited {@code delimiter} into a list of non-empty strings
|
||||
* (skipping any empty string produced during splitting)
|
||||
*/
|
||||
public static Set<String> toSet(@Nullable String input) {
|
||||
return isNullOrEmpty(input) ? new HashSet<>() : STRING_TO_SET.apply(input);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the input string, delimited by comma, to a list of strings.
|
||||
*
|
||||
* @param input
|
||||
* @return
|
||||
*/
|
||||
public static List<String> toList(@Nullable String input) {
|
||||
return isNullOrEmpty(input) ? new ArrayList<>() : STRING_TO_LIST.apply(input);
|
||||
public static List<String> split(@Nullable String input, String delimiter) {
|
||||
if (isNullOrEmpty(input)) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return Stream.of(input.split(delimiter)).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -42,7 +42,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.util.LazyRef;
|
||||
import org.apache.hudi.util.Lazy;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -79,7 +79,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
|
||||
private final Path path;
|
||||
|
||||
private final LazyRef<Schema> schema;
|
||||
private final Lazy<Schema> schema;
|
||||
|
||||
// NOTE: Reader is ONLY THREAD-SAFE for {@code Scanner} operating in Positional Read ("pread")
|
||||
// mode (ie created w/ "pread = true")
|
||||
@@ -110,8 +110,8 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
// For shared scanner, which is primarily used for point-lookups, we're caching blocks
|
||||
// by default, to minimize amount of traffic to the underlying storage
|
||||
this.sharedScanner = getHFileScanner(reader, true);
|
||||
this.schema = schemaOpt.map(LazyRef::eager)
|
||||
.orElseGet(() -> LazyRef.lazy(() -> fetchSchema(reader)));
|
||||
this.schema = schemaOpt.map(Lazy::eagerly)
|
||||
.orElseGet(() -> Lazy.lazily(() -> fetchSchema(reader)));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -49,7 +49,6 @@ import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
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.HoodieIOException;
|
||||
@@ -64,6 +63,7 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.util.Lazy;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -74,11 +74,11 @@ import java.math.BigDecimal;
|
||||
import java.math.RoundingMode;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -94,6 +94,7 @@ import static org.apache.hudi.avro.HoodieAvroUtils.convertValueForSpecificDataTy
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldSchemaFromWriteSchema;
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.resolveNullableSchema;
|
||||
import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
|
||||
import static org.apache.hudi.common.util.ValidationUtils.checkState;
|
||||
import static org.apache.hudi.metadata.HoodieMetadataPayload.unwrapStatisticValueWrapper;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadata.EMPTY_PARTITION_NAME;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME;
|
||||
@@ -445,21 +446,24 @@ public class HoodieTableMetadataUtil {
|
||||
/**
|
||||
* Convert the clean action to metadata records.
|
||||
*/
|
||||
public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
|
||||
HoodieEngineContext engineContext, HoodieCleanMetadata cleanMetadata,
|
||||
MetadataRecordsGenerationParams recordsGenerationParams, String instantTime) {
|
||||
public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(HoodieEngineContext engineContext,
|
||||
HoodieCleanMetadata cleanMetadata,
|
||||
MetadataRecordsGenerationParams recordsGenerationParams,
|
||||
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 (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) {
|
||||
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = convertMetadataToBloomFilterRecords(cleanMetadata, engineContext, instantTime, recordsGenerationParams);
|
||||
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD =
|
||||
convertMetadataToBloomFilterRecords(cleanMetadata, engineContext, instantTime, recordsGenerationParams);
|
||||
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
|
||||
}
|
||||
|
||||
if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) {
|
||||
final HoodieData<HoodieRecord> metadataColumnStatsRDD = convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, recordsGenerationParams);
|
||||
final HoodieData<HoodieRecord> metadataColumnStatsRDD =
|
||||
convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, recordsGenerationParams);
|
||||
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
|
||||
}
|
||||
|
||||
@@ -554,8 +558,9 @@ public class HoodieTableMetadataUtil {
|
||||
|
||||
HoodieTableMetaClient dataTableMetaClient = recordsGenerationParams.getDataMetaClient();
|
||||
|
||||
List<String> columnsToIndex = getColumnsToIndex(recordsGenerationParams,
|
||||
dataTableMetaClient.getTableConfig(), tryResolveSchemaForTable(dataTableMetaClient));
|
||||
List<String> columnsToIndex =
|
||||
getColumnsToIndex(recordsGenerationParams,
|
||||
Lazy.lazily(() -> tryResolveSchemaForTable(dataTableMetaClient)));
|
||||
|
||||
if (columnsToIndex.isEmpty()) {
|
||||
// In case there are no columns to index, bail
|
||||
@@ -597,7 +602,8 @@ public class HoodieTableMetadataUtil {
|
||||
}
|
||||
|
||||
if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) {
|
||||
final HoodieData<HoodieRecord> metadataColumnStatsRDD = convertFilesToColumnStatsRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams);
|
||||
final HoodieData<HoodieRecord> metadataColumnStatsRDD =
|
||||
convertFilesToColumnStatsRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams);
|
||||
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
|
||||
}
|
||||
return partitionToRecordsMap;
|
||||
@@ -803,7 +809,7 @@ public class HoodieTableMetadataUtil {
|
||||
fileChangeCount[1] += appendedFileMap.size();
|
||||
|
||||
// Validate that no appended file has been deleted
|
||||
ValidationUtils.checkState(
|
||||
checkState(
|
||||
!appendedFileMap.keySet().removeAll(partitionToDeletedFiles.getOrDefault(partition, Collections.emptyList())),
|
||||
"Rollback file cannot both be appended and deleted");
|
||||
|
||||
@@ -904,39 +910,47 @@ public class HoodieTableMetadataUtil {
|
||||
HoodieData<HoodieRecord> allRecordsRDD = engineContext.emptyHoodieData();
|
||||
HoodieTableMetaClient dataTableMetaClient = recordsGenerationParams.getDataMetaClient();
|
||||
|
||||
final List<String> columnsToIndex = getColumnsToIndex(recordsGenerationParams,
|
||||
dataTableMetaClient.getTableConfig(), tryResolveSchemaForTable(dataTableMetaClient));
|
||||
final List<String> columnsToIndex =
|
||||
getColumnsToIndex(recordsGenerationParams,
|
||||
Lazy.lazily(() -> tryResolveSchemaForTable(dataTableMetaClient)));
|
||||
|
||||
if (columnsToIndex.isEmpty()) {
|
||||
// In case there are no columns to index, bail
|
||||
return engineContext.emptyHoodieData();
|
||||
}
|
||||
|
||||
final List<Pair<String, List<String>>> partitionToDeletedFilesList = partitionToDeletedFiles.entrySet()
|
||||
.stream().map(e -> Pair.of(e.getKey(), e.getValue())).collect(Collectors.toList());
|
||||
int parallelism = Math.max(Math.min(partitionToDeletedFilesList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1);
|
||||
final HoodieData<Pair<String, List<String>>> partitionToDeletedFilesRDD = engineContext.parallelize(partitionToDeletedFilesList, parallelism);
|
||||
final List<Pair<String, List<String>>> partitionToDeletedFilesList = partitionToDeletedFiles.entrySet().stream()
|
||||
.map(e -> Pair.of(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
int deletedFilesTargetParallelism = Math.max(Math.min(partitionToDeletedFilesList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1);
|
||||
final HoodieData<Pair<String, List<String>>> partitionToDeletedFilesRDD =
|
||||
engineContext.parallelize(partitionToDeletedFilesList, deletedFilesTargetParallelism);
|
||||
|
||||
HoodieData<HoodieRecord> deletedFilesRecordsRDD = partitionToDeletedFilesRDD.flatMap(partitionToDeletedFilesPair -> {
|
||||
final String partitionName = partitionToDeletedFilesPair.getLeft();
|
||||
final String partition = getPartitionIdentifier(partitionName);
|
||||
final String partitionPath = partitionToDeletedFilesPair.getLeft();
|
||||
final String partitionId = getPartitionIdentifier(partitionPath);
|
||||
final List<String> deletedFileList = partitionToDeletedFilesPair.getRight();
|
||||
|
||||
return deletedFileList.stream().flatMap(deletedFile -> {
|
||||
final String filePathWithPartition = partitionName + "/" + deletedFile;
|
||||
return getColumnStatsRecords(partition, filePathWithPartition, dataTableMetaClient, columnsToIndex, true);
|
||||
final String filePathWithPartition = partitionPath + "/" + deletedFile;
|
||||
return getColumnStatsRecords(partitionId, filePathWithPartition, dataTableMetaClient, columnsToIndex, true);
|
||||
}).iterator();
|
||||
});
|
||||
|
||||
allRecordsRDD = allRecordsRDD.union(deletedFilesRecordsRDD);
|
||||
|
||||
final List<Pair<String, Map<String, Long>>> partitionToAppendedFilesList = partitionToAppendedFiles.entrySet()
|
||||
.stream().map(entry -> Pair.of(entry.getKey(), entry.getValue())).collect(Collectors.toList());
|
||||
parallelism = Math.max(Math.min(partitionToAppendedFilesList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1);
|
||||
final HoodieData<Pair<String, Map<String, Long>>> partitionToAppendedFilesRDD = engineContext.parallelize(partitionToAppendedFilesList, parallelism);
|
||||
final List<Pair<String, Map<String, Long>>> partitionToAppendedFilesList = partitionToAppendedFiles.entrySet().stream()
|
||||
.map(entry -> Pair.of(entry.getKey(), entry.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
int appendedFilesTargetParallelism = Math.max(Math.min(partitionToAppendedFilesList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1);
|
||||
final HoodieData<Pair<String, Map<String, Long>>> partitionToAppendedFilesRDD =
|
||||
engineContext.parallelize(partitionToAppendedFilesList, appendedFilesTargetParallelism);
|
||||
|
||||
HoodieData<HoodieRecord> appendedFilesRecordsRDD = partitionToAppendedFilesRDD.flatMap(partitionToAppendedFilesPair -> {
|
||||
final String partitionName = partitionToAppendedFilesPair.getLeft();
|
||||
final String partition = getPartitionIdentifier(partitionName);
|
||||
final String partitionPath = partitionToAppendedFilesPair.getLeft();
|
||||
final String partitionId = getPartitionIdentifier(partitionPath);
|
||||
final Map<String, Long> appendedFileMap = partitionToAppendedFilesPair.getRight();
|
||||
|
||||
return appendedFileMap.entrySet().stream().flatMap(appendedFileNameLengthEntry -> {
|
||||
@@ -944,11 +958,11 @@ public class HoodieTableMetadataUtil {
|
||||
|| !appendedFileNameLengthEntry.getKey().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
return Stream.empty();
|
||||
}
|
||||
final String filePathWithPartition = partitionName + "/" + appendedFileNameLengthEntry.getKey();
|
||||
return getColumnStatsRecords(partition, filePathWithPartition, dataTableMetaClient, columnsToIndex, false);
|
||||
final String filePathWithPartition = partitionPath + "/" + appendedFileNameLengthEntry.getKey();
|
||||
return getColumnStatsRecords(partitionId, filePathWithPartition, dataTableMetaClient, columnsToIndex, false);
|
||||
}).iterator();
|
||||
|
||||
});
|
||||
|
||||
allRecordsRDD = allRecordsRDD.union(appendedFilesRecordsRDD);
|
||||
|
||||
return allRecordsRDD;
|
||||
@@ -1091,7 +1105,7 @@ public class HoodieTableMetadataUtil {
|
||||
tableConfig.populateMetaFields() ? addMetadataFields(schema) : schema);
|
||||
|
||||
List<String> columnsToIndex = getColumnsToIndex(recordsGenerationParams,
|
||||
tableConfig, tableSchema);
|
||||
Lazy.eagerly(tableSchema));
|
||||
|
||||
if (columnsToIndex.isEmpty()) {
|
||||
// In case there are no columns to index, bail
|
||||
@@ -1108,19 +1122,24 @@ public class HoodieTableMetadataUtil {
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest columns for the table for column stats indexing.
|
||||
* Get the list of columns for the table for column stats indexing
|
||||
*/
|
||||
private static List<String> getColumnsToIndex(MetadataRecordsGenerationParams recordsGenParams,
|
||||
HoodieTableConfig tableConfig,
|
||||
Option<Schema> writerSchemaOpt) {
|
||||
if (recordsGenParams.isAllColumnStatsIndexEnabled() && writerSchemaOpt.isPresent()) {
|
||||
return writerSchemaOpt.get().getFields().stream()
|
||||
.map(Schema.Field::name).collect(Collectors.toList());
|
||||
Lazy<Option<Schema>> lazyWriterSchemaOpt) {
|
||||
checkState(recordsGenParams.isColumnStatsIndexEnabled());
|
||||
|
||||
List<String> targetColumns = recordsGenParams.getTargetColumnsForColumnStatsIndex();
|
||||
if (!targetColumns.isEmpty()) {
|
||||
return targetColumns;
|
||||
}
|
||||
|
||||
// In case no writer schema could be obtained we fall back to only index primary key
|
||||
// columns
|
||||
return Arrays.asList(tableConfig.getRecordKeyFields().get());
|
||||
Option<Schema> writerSchemaOpt = lazyWriterSchemaOpt.get();
|
||||
return writerSchemaOpt
|
||||
.map(writerSchema ->
|
||||
writerSchema.getFields().stream()
|
||||
.map(Schema.Field::name)
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(Collections.emptyList());
|
||||
}
|
||||
|
||||
private static Stream<HoodieRecord> translateWriteStatToColumnStats(HoodieWriteStat writeStat,
|
||||
@@ -1331,11 +1350,11 @@ public class HoodieTableMetadataUtil {
|
||||
}
|
||||
|
||||
public static Set<String> getInflightMetadataPartitions(HoodieTableConfig tableConfig) {
|
||||
return StringUtils.toSet(tableConfig.getMetadataPartitionsInflight());
|
||||
return new HashSet<>(tableConfig.getMetadataPartitionsInflight());
|
||||
}
|
||||
|
||||
public static Set<String> getCompletedMetadataPartitions(HoodieTableConfig tableConfig) {
|
||||
return StringUtils.toSet(tableConfig.getMetadataPartitions());
|
||||
return new HashSet<>(tableConfig.getMetadataPartitions());
|
||||
}
|
||||
|
||||
public static Set<String> getInflightAndCompletedMetadataPartitions(HoodieTableConfig tableConfig) {
|
||||
|
||||
@@ -26,28 +26,33 @@ import java.util.List;
|
||||
|
||||
/**
|
||||
* Encapsulates all parameters required to generate metadata index for enabled index types.
|
||||
*
|
||||
* @deprecated this component currently duplicates configuration coming from the {@code HoodieWriteConfig}
|
||||
* which is problematic; instead we should break this component down and use source of truth
|
||||
* for each respective data-point directly ({@code HoodieWriteConfig}, {@code HoodieTableMetaClient}, etc)
|
||||
*/
|
||||
@Deprecated
|
||||
public class MetadataRecordsGenerationParams implements Serializable {
|
||||
|
||||
private final HoodieTableMetaClient dataMetaClient;
|
||||
private final List<MetadataPartitionType> enabledPartitionTypes;
|
||||
private final String bloomFilterType;
|
||||
private final int bloomIndexParallelism;
|
||||
private final boolean isAllColumnStatsIndexEnabled;
|
||||
private final boolean isColumnStatsIndexEnabled;
|
||||
private final int columnStatsIndexParallelism;
|
||||
private final List<String> columnsToIndex;
|
||||
private final List<String> bloomSecondaryKeys;
|
||||
private final List<String> targetColumnsForColumnStatsIndex;
|
||||
private final List<String> targetColumnsForBloomFilterIndex;
|
||||
|
||||
MetadataRecordsGenerationParams(HoodieTableMetaClient dataMetaClient, List<MetadataPartitionType> enabledPartitionTypes, String bloomFilterType, int bloomIndexParallelism,
|
||||
boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism, List<String> columnsToIndex, List<String> bloomSecondaryKeys) {
|
||||
boolean isColumnStatsIndexEnabled, int columnStatsIndexParallelism, List<String> targetColumnsForColumnStatsIndex, List<String> targetColumnsForBloomFilterIndex) {
|
||||
this.dataMetaClient = dataMetaClient;
|
||||
this.enabledPartitionTypes = enabledPartitionTypes;
|
||||
this.bloomFilterType = bloomFilterType;
|
||||
this.bloomIndexParallelism = bloomIndexParallelism;
|
||||
this.isAllColumnStatsIndexEnabled = isAllColumnStatsIndexEnabled;
|
||||
this.isColumnStatsIndexEnabled = isColumnStatsIndexEnabled;
|
||||
this.columnStatsIndexParallelism = columnStatsIndexParallelism;
|
||||
this.columnsToIndex = columnsToIndex;
|
||||
this.bloomSecondaryKeys = bloomSecondaryKeys;
|
||||
this.targetColumnsForColumnStatsIndex = targetColumnsForColumnStatsIndex;
|
||||
this.targetColumnsForBloomFilterIndex = targetColumnsForBloomFilterIndex;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getDataMetaClient() {
|
||||
@@ -62,8 +67,8 @@ public class MetadataRecordsGenerationParams implements Serializable {
|
||||
return bloomFilterType;
|
||||
}
|
||||
|
||||
public boolean isAllColumnStatsIndexEnabled() {
|
||||
return isAllColumnStatsIndexEnabled;
|
||||
public boolean isColumnStatsIndexEnabled() {
|
||||
return isColumnStatsIndexEnabled;
|
||||
}
|
||||
|
||||
public int getBloomIndexParallelism() {
|
||||
@@ -74,11 +79,11 @@ public class MetadataRecordsGenerationParams implements Serializable {
|
||||
return columnStatsIndexParallelism;
|
||||
}
|
||||
|
||||
public List<String> getColumnsToIndex() {
|
||||
return columnsToIndex;
|
||||
public List<String> getTargetColumnsForColumnStatsIndex() {
|
||||
return targetColumnsForColumnStatsIndex;
|
||||
}
|
||||
|
||||
public List<String> getBloomSecondaryKeys() {
|
||||
return bloomSecondaryKeys;
|
||||
public List<String> getSecondaryKeysForBloomFilterIndex() {
|
||||
return targetColumnsForBloomFilterIndex;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,21 +20,25 @@ package org.apache.hudi.util;
|
||||
|
||||
import java.util.function.Supplier;
|
||||
|
||||
// TODO java-doc
|
||||
public class LazyRef<T> {
|
||||
/**
|
||||
* Utility implementing lazy semantics in Java
|
||||
*
|
||||
* @param <T> type of the object being held by {@link Lazy}
|
||||
*/
|
||||
public class Lazy<T> {
|
||||
|
||||
private volatile boolean initialized;
|
||||
|
||||
private Supplier<T> initializer;
|
||||
private T ref;
|
||||
|
||||
private LazyRef(Supplier<T> initializer) {
|
||||
private Lazy(Supplier<T> initializer) {
|
||||
this.initializer = initializer;
|
||||
this.ref = null;
|
||||
this.initialized = false;
|
||||
}
|
||||
|
||||
private LazyRef(T ref) {
|
||||
private Lazy(T ref) {
|
||||
this.initializer = null;
|
||||
this.ref = ref;
|
||||
this.initialized = true;
|
||||
@@ -54,11 +58,20 @@ public class LazyRef<T> {
|
||||
return ref;
|
||||
}
|
||||
|
||||
public static <T> LazyRef<T> lazy(Supplier<T> initializer) {
|
||||
return new LazyRef<>(initializer);
|
||||
/**
|
||||
* Executes provided {@code initializer} lazily, while providing for "exactly once" semantic,
|
||||
* to instantiate value of type {@link T} being subsequently held by the returned instance of
|
||||
* {@link Lazy}
|
||||
*/
|
||||
public static <T> Lazy<T> lazily(Supplier<T> initializer) {
|
||||
return new Lazy<>(initializer);
|
||||
}
|
||||
|
||||
public static <T> LazyRef<T> eager(T ref) {
|
||||
return new LazyRef<>(ref);
|
||||
/**
|
||||
* Instantiates {@link Lazy} in an "eagerly" fashion setting it w/ the provided value of
|
||||
* type {@link T} directly, bypassing lazy initialization sequence
|
||||
*/
|
||||
public static <T> Lazy<T> eagerly(T ref) {
|
||||
return new Lazy<>(ref);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user