1
0

[HUDI-3356][HUDI-3203] HoodieData for metadata index records; BloomFilter construction from index based on the type param (#4848)

Rework of #4761 
This diff introduces following changes:

- Write stats are converted to metadata index records during the commit. Making them use the HoodieData type so that the record generation scales up with needs. 
- Metadata index init support for bloom filter and column stats partitions.
- When building the BloomFilter from the index records, using the type param stored in the payload instead of hardcoded type.
- Delta writes can change column ranges and the column stats index need to be properly updated with new ranges to be consistent with the table dataset. This fix add column stats index update support for the delta writes.

Co-authored-by: Manoj Govindassamy <manoj.govindassamy@gmail.com>
This commit is contained in:
Sagar Sumit
2022-03-08 21:09:04 +05:30
committed by GitHub
parent ed26c5265c
commit 575bc63468
24 changed files with 1051 additions and 533 deletions

View File

@@ -1492,6 +1492,10 @@ public class HoodieWriteConfig extends HoodieConfig {
return isMetadataTableEnabled() && getMetadataConfig().isMetadataColumnStatsIndexForAllColumnsEnabled(); return isMetadataTableEnabled() && getMetadataConfig().isMetadataColumnStatsIndexForAllColumnsEnabled();
} }
public int getColumnStatsIndexParallelism() {
return metadataConfig.getColumnStatsIndexParallelism();
}
public int getBloomIndexKeysPerBucket() { public int getBloomIndexKeysPerBucket() {
return getInt(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET); return getInt(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET);
} }

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.common.engine.TaskContextSupplier;
import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.BaseFile;
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.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieDeltaWriteStat;
import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieLogFile;
@@ -71,6 +72,9 @@ import java.util.Properties;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.accumulateColumnRanges;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.aggregateColumnStats;
/** /**
* IO Operation to append data onto an existing file. * IO Operation to append data onto an existing file.
*/ */
@@ -320,7 +324,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
statuses.add(this.writeStatus); statuses.add(this.writeStatus);
} }
private void processAppendResult(AppendResult result) { private void processAppendResult(AppendResult result, List<IndexedRecord> recordList) {
HoodieDeltaWriteStat stat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); HoodieDeltaWriteStat stat = (HoodieDeltaWriteStat) this.writeStatus.getStat();
if (stat.getPath() == null) { if (stat.getPath() == null) {
@@ -339,6 +343,19 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
updateWriteStatus(stat, result); updateWriteStatus(stat, result);
} }
if (config.isMetadataIndexColumnStatsForAllColumnsEnabled()) {
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<>();
writeSchemaWithMetaFields.getFields().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, writeSchemaWithMetaFields, columnToStats, config.isConsistentLogicalTimestampEnabled()));
writeSchemaWithMetaFields.getFields().forEach(field -> accumulateColumnRanges(field, filePath, columnRangeMap, columnToStats));
stat.setRecordsStats(new HoodieDeltaWriteStat.RecordsStats<>(columnRangeMap));
}
resetWriteCounts(); resetWriteCounts();
assert stat.getRuntimeStats() != null; assert stat.getRuntimeStats() != null;
LOG.info(String.format("AppendHandle for partitionPath %s filePath %s, took %d ms.", partitionPath, LOG.info(String.format("AppendHandle for partitionPath %s filePath %s, took %d ms.", partitionPath,
@@ -376,7 +393,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
if (blocks.size() > 0) { if (blocks.size() > 0) {
AppendResult appendResult = writer.appendBlocks(blocks); AppendResult appendResult = writer.appendBlocks(blocks);
processAppendResult(appendResult); processAppendResult(appendResult, recordList);
recordList.clear(); recordList.clear();
keysToDelete.clear(); keysToDelete.clear();
} }
@@ -419,7 +436,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
// update final size, once for all log files // update final size, once for all log files
// TODO we can actually deduce file size purely from AppendResult (based on offset and size // TODO we can actually deduce file size purely from AppendResult (based on offset and size
// of the appended block) // of the appended block)
for (WriteStatus status: statuses) { for (WriteStatus status : statuses) {
long logFileSize = FSUtils.getFileSize(fs, new Path(config.getBasePath(), status.getStat().getPath())); long logFileSize = FSUtils.getFileSize(fs, new Path(config.getBasePath(), status.getStat().getPath()));
status.getStat().setFileSizeInBytes(logFileSize); status.getStat().setFileSizeInBytes(logFileSize);
} }

View File

@@ -19,14 +19,9 @@
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.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;
@@ -39,8 +34,6 @@ 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.List; import java.util.List;
@@ -53,27 +46,13 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload, I, K, O> exten
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> partitionPathFileIDPair) { Pair<String, String> partitionPathFileIDPair) {
this(config, hoodieTable, partitionPathFileIDPair, Option.empty(), false);
}
public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
Pair<String, String> partitionPathFileIDPair, Option<String> fileName,
boolean useMetadataTableIndex) {
super(config, hoodieTable, partitionPathFileIDPair); super(config, hoodieTable, partitionPathFileIDPair);
this.candidateRecordKeys = new ArrayList<>(); this.candidateRecordKeys = new ArrayList<>();
this.totalKeysChecked = 0; 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(); this.bloomFilter = getBloomFilter();
} }
@@ -81,25 +60,16 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload, I, K, O> exten
BloomFilter bloomFilter = null; BloomFilter bloomFilter = null;
HoodieTimer timer = new HoodieTimer().startTimer(); HoodieTimer timer = new HoodieTimer().startTimer();
try { try {
if (this.useMetadataTableIndex) { if (config.isMetadataBloomFilterIndexEnabled()) {
ValidationUtils.checkArgument(this.fileName.isPresent(), bloomFilter = hoodieTable.getMetadataTable().getBloomFilter(partitionPathFileIDPair.getLeft(), partitionPathFileIDPair.getRight())
"File name not available to fetch bloom filter from the metadata table index."); .orElseThrow(() -> new HoodieIndexException("BloomFilter missing for " + partitionPathFileIDPair.getRight()));
Option<ByteBuffer> bloomFilterByteBuffer =
hoodieTable.getMetadataTable().getBloomFilter(partitionPathFileIDPair.getLeft(), fileName.get());
if (!bloomFilterByteBuffer.isPresent()) {
throw new HoodieIndexException("BloomFilter missing for " + partitionPathFileIDPair.getRight());
}
bloomFilter =
new HoodieDynamicBoundedBloomFilter(StandardCharsets.UTF_8.decode(bloomFilterByteBuffer.get()).toString(),
BloomFilterTypeCode.DYNAMIC_V0);
} else { } else {
try (HoodieFileReader reader = createNewFileReader()) { try (HoodieFileReader reader = createNewFileReader()) {
bloomFilter = reader.readBloomFilter(); bloomFilter = reader.readBloomFilter();
} }
} }
} catch (IOException e) { } catch (IOException e) {
throw new HoodieIndexException(String.format("Error reading bloom filter from %s/%s - %s", throw new HoodieIndexException(String.format("Error reading bloom filter from %s", getPartitionPathFileIDPair()), e);
getPartitionPathFileIDPair().getLeft(), this.fileName, e));
} }
LOG.info(String.format("Read bloom filter from %s in %d ms", partitionPathFileIDPair, timer.endTimer())); LOG.info(String.format("Read bloom filter from %s in %d ms", partitionPathFileIDPair, timer.endTimer()));
return bloomFilter; return bloomFilter;

View File

@@ -55,6 +55,7 @@ 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;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig;
@@ -81,11 +82,11 @@ import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX;
import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME;
import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP;
/** /**
@@ -121,7 +122,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
* @param hadoopConf - Hadoop configuration to use for the metadata writer * @param hadoopConf - Hadoop configuration to use for the metadata writer
* @param writeConfig - Writer config * @param writeConfig - Writer config
* @param engineContext - Engine context * @param engineContext - Engine context
* @param actionMetadata - Optional action metadata to help decide bootstrap operations * @param actionMetadata - Optional action metadata to help decide initialize operations
* @param <T> - Action metadata types extending Avro generated SpecificRecordBase * @param <T> - Action metadata types extending Avro generated SpecificRecordBase
* @param inflightInstantTimestamp - Timestamp of any instant in progress * @param inflightInstantTimestamp - Timestamp of any instant in progress
*/ */
@@ -203,7 +204,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
* @param metadataConfig - Table config * @param metadataConfig - Table config
* @param metaClient - Meta client for the metadata table * @param metaClient - Meta client for the metadata table
* @param fsView - Metadata table filesystem view to use * @param fsView - Metadata table filesystem view to use
* @param isBootstrapCompleted - Is metadata table bootstrap completed * @param isBootstrapCompleted - Is metadata table initializing completed
*/ */
private void enablePartition(final MetadataPartitionType partitionType, final HoodieMetadataConfig metadataConfig, private void enablePartition(final MetadataPartitionType partitionType, final HoodieMetadataConfig metadataConfig,
final Option<HoodieTableMetaClient> metaClient, Option<HoodieTableFileSystemView> fsView, boolean isBootstrapCompleted) { final Option<HoodieTableMetaClient> metaClient, Option<HoodieTableFileSystemView> fsView, boolean isBootstrapCompleted) {
@@ -319,13 +320,13 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
/** /**
* Initialize the metadata table if it does not exist. * Initialize the metadata table if it does not exist.
* * <p>
* If the metadata table does not exist, then file and partition listing is used to bootstrap the table. * If the metadata table does not exist, then file and partition listing is used to initialize the table.
* *
* @param engineContext * @param engineContext
* @param actionMetadata Action metadata types extending Avro generated SpecificRecordBase * @param actionMetadata Action metadata types extending Avro generated SpecificRecordBase
* @param inflightInstantTimestamp Timestamp of an instant in progress on the dataset. This instant is ignored * @param inflightInstantTimestamp Timestamp of an instant in progress on the dataset. This instant is ignored
* while deciding to bootstrap the metadata table. * while deciding to initialize the metadata table.
*/ */
protected abstract <T extends SpecificRecordBase> void initialize(HoodieEngineContext engineContext, protected abstract <T extends SpecificRecordBase> void initialize(HoodieEngineContext engineContext,
Option<T> actionMetadata, Option<T> actionMetadata,
@@ -345,27 +346,25 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
} }
/** /**
* Bootstrap the metadata table if needed. * Initialize the metadata table if needed.
* *
* @param engineContext - Engine context * @param dataMetaClient - meta client for the data table
* @param dataMetaClient - Meta client for the data table * @param actionMetadata - optional action metadata
* @param actionMetadata - Optional action metadata * @param inflightInstantTimestamp - timestamp of an instant in progress on the dataset
* @param <T> - Action metadata types extending Avro generated SpecificRecordBase * @param <T> - action metadata types extending Avro generated SpecificRecordBase
* @param inflightInstantTimestamp - Timestamp of an instant in progress on the dataset. This instant is ignored
* @throws IOException * @throws IOException
*/ */
protected <T extends SpecificRecordBase> void bootstrapIfNeeded(HoodieEngineContext engineContext, protected <T extends SpecificRecordBase> void initializeIfNeeded(HoodieTableMetaClient dataMetaClient,
HoodieTableMetaClient dataMetaClient, Option<T> actionMetadata,
Option<T> actionMetadata, Option<String> inflightInstantTimestamp) throws IOException {
Option<String> inflightInstantTimestamp) throws IOException {
HoodieTimer timer = new HoodieTimer().startTimer(); HoodieTimer timer = new HoodieTimer().startTimer();
boolean exists = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), boolean exists = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(),
HoodieTableMetaClient.METAFOLDER_NAME)); HoodieTableMetaClient.METAFOLDER_NAME));
boolean rebootstrap = false; boolean reInitialize = false;
// If the un-synced instants have been archived, then // If the un-synced instants have been archived, then
// the metadata table will need to be bootstrapped again. // the metadata table will need to be initialized again.
if (exists) { if (exists) {
HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()) HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get())
.setBasePath(metadataWriteConfig.getBasePath()).build(); .setBasePath(metadataWriteConfig.getBasePath()).build();
@@ -378,39 +377,39 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
final Option<HoodieInstant> latestMetadataInstant = final Option<HoodieInstant> latestMetadataInstant =
metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant();
rebootstrap = isBootstrapNeeded(latestMetadataInstant, actionMetadata); reInitialize = isBootstrapNeeded(latestMetadataInstant, actionMetadata);
} }
if (rebootstrap) { if (reInitialize) {
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.REBOOTSTRAP_STR, 1)); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.REBOOTSTRAP_STR, 1));
LOG.info("Deleting Metadata Table directory so that it can be re-bootstrapped"); LOG.info("Deleting Metadata Table directory so that it can be re-initialized");
dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath()), true); dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath()), true);
exists = false; exists = false;
} }
if (!exists) { if (!exists) {
// Initialize for the first time by listing partitions and files directly from the file system // Initialize for the first time by listing partitions and files directly from the file system
if (bootstrapFromFilesystem(engineContext, dataMetaClient, inflightInstantTimestamp)) { if (initializeFromFilesystem(dataMetaClient, inflightInstantTimestamp)) {
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer()));
} }
} }
} }
/** /**
* Whether bootstrap operation needed for this metadata table. * Whether initialize operation needed for this metadata table.
* <p> * <p>
* Rollback of the first commit would look like un-synced instants in the metadata table. * Rollback of the first commit would look like un-synced instants in the metadata table.
* Action metadata is needed to verify the instant time and avoid erroneous bootstrapping. * Action metadata is needed to verify the instant time and avoid erroneous initializing.
* <p> * <p>
* TODO: Revisit this logic and validate that filtering for all * TODO: Revisit this logic and validate that filtering for all
* commits timeline is the right thing to do * commits timeline is the right thing to do
* *
* @return True if the bootstrap is not needed, False otherwise * @return True if the initialize is not needed, False otherwise
*/ */
private <T extends SpecificRecordBase> boolean isBootstrapNeeded(Option<HoodieInstant> latestMetadataInstant, private <T extends SpecificRecordBase> boolean isBootstrapNeeded(Option<HoodieInstant> latestMetadataInstant,
Option<T> actionMetadata) { Option<T> actionMetadata) {
if (!latestMetadataInstant.isPresent()) { if (!latestMetadataInstant.isPresent()) {
LOG.warn("Metadata Table will need to be re-bootstrapped as no instants were found"); LOG.warn("Metadata Table will need to be re-initialized as no instants were found");
return true; return true;
} }
@@ -423,7 +422,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
if (dataMetaClient.getActiveTimeline().getAllCommitsTimeline().isBeforeTimelineStarts( if (dataMetaClient.getActiveTimeline().getAllCommitsTimeline().isBeforeTimelineStarts(
latestMetadataInstant.get().getTimestamp()) latestMetadataInstant.get().getTimestamp())
&& !isCommitRevertedByInFlightAction(actionMetadata, latestMetadataInstantTimestamp)) { && !isCommitRevertedByInFlightAction(actionMetadata, latestMetadataInstantTimestamp)) {
LOG.error("Metadata Table will need to be re-bootstrapped as un-synced instants have been archived." LOG.error("Metadata Table will need to be re-initialized as un-synced instants have been archived."
+ " latestMetadataInstant=" + latestMetadataInstant.get().getTimestamp() + " latestMetadataInstant=" + latestMetadataInstant.get().getTimestamp()
+ ", latestDataInstant=" + dataMetaClient.getActiveTimeline().firstInstant().get().getTimestamp()); + ", latestDataInstant=" + dataMetaClient.getActiveTimeline().firstInstant().get().getTimestamp());
return true; return true;
@@ -455,9 +454,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
case HoodieTimeline.ROLLBACK_ACTION: case HoodieTimeline.ROLLBACK_ACTION:
List<HoodieInstantInfo> rollbackedInstants = List<HoodieInstantInfo> rollbackedInstants =
((HoodieRollbackMetadata) actionMetadata.get()).getInstantsRollback(); ((HoodieRollbackMetadata) actionMetadata.get()).getInstantsRollback();
affectedInstantTimestamps = rollbackedInstants.stream().map(instant -> { affectedInstantTimestamps = rollbackedInstants.stream().map(HoodieInstantInfo::getCommitTime).collect(Collectors.toList());
return instant.getCommitTime().toString();
}).collect(Collectors.toList());
if (affectedInstantTimestamps.contains(latestMetadataInstantTimestamp)) { if (affectedInstantTimestamps.contains(latestMetadataInstantTimestamp)) {
return true; return true;
@@ -466,9 +463,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
case HoodieTimeline.RESTORE_ACTION: case HoodieTimeline.RESTORE_ACTION:
List<HoodieInstantInfo> restoredInstants = List<HoodieInstantInfo> restoredInstants =
((HoodieRestoreMetadata) actionMetadata.get()).getRestoreInstantInfo(); ((HoodieRestoreMetadata) actionMetadata.get()).getRestoreInstantInfo();
affectedInstantTimestamps = restoredInstants.stream().map(instant -> { affectedInstantTimestamps = restoredInstants.stream().map(HoodieInstantInfo::getCommitTime).collect(Collectors.toList());
return instant.getCommitTime().toString();
}).collect(Collectors.toList());
if (affectedInstantTimestamps.contains(latestMetadataInstantTimestamp)) { if (affectedInstantTimestamps.contains(latestMetadataInstantTimestamp)) {
return true; return true;
@@ -484,14 +479,14 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
/** /**
* Initialize the Metadata Table by listing files and partitions from the file system. * Initialize the Metadata Table by listing files and partitions from the file system.
* *
* @param dataMetaClient {@code HoodieTableMetaClient} for the dataset. * @param dataMetaClient - {@code HoodieTableMetaClient} for the dataset.
* @param inflightInstantTimestamp * @param inflightInstantTimestamp - Current action instant responsible for this initialization
*/ */
private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTableMetaClient dataMetaClient, private boolean initializeFromFilesystem(HoodieTableMetaClient dataMetaClient,
Option<String> inflightInstantTimestamp) throws IOException { Option<String> inflightInstantTimestamp) throws IOException {
ValidationUtils.checkState(enabled, "Metadata table cannot be initialized as it is not enabled"); ValidationUtils.checkState(enabled, "Metadata table cannot be initialized as it is not enabled");
// We can only bootstrap if there are no pending operations on the dataset // We can only initialize if there are no pending operations on the dataset
List<HoodieInstant> pendingDataInstant = dataMetaClient.getActiveTimeline() List<HoodieInstant> pendingDataInstant = dataMetaClient.getActiveTimeline()
.getInstants().filter(i -> !i.isCompleted()) .getInstants().filter(i -> !i.isCompleted())
.filter(i -> !inflightInstantTimestamp.isPresent() || !i.getTimestamp().equals(inflightInstantTimestamp.get())) .filter(i -> !inflightInstantTimestamp.isPresent() || !i.getTimestamp().equals(inflightInstantTimestamp.get()))
@@ -499,7 +494,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
if (!pendingDataInstant.isEmpty()) { if (!pendingDataInstant.isEmpty()) {
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BOOTSTRAP_ERR_STR, 1)); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BOOTSTRAP_ERR_STR, 1));
LOG.warn("Cannot bootstrap metadata table as operation(s) are in progress on the dataset: " LOG.warn("Cannot initialize metadata table as operation(s) are in progress on the dataset: "
+ Arrays.toString(pendingDataInstant.toArray())); + Arrays.toString(pendingDataInstant.toArray()));
return false; return false;
} }
@@ -514,15 +509,10 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
initTableMetadata(); initTableMetadata();
initializeEnabledFileGroups(dataMetaClient, createInstantTime); initializeEnabledFileGroups(dataMetaClient, createInstantTime);
// List all partitions in the basePath of the containing dataset // During cold startup, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out
LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); // of these large number of files and calling the existing update(HoodieCommitMetadata) function does not scale
engineContext.setJobStatus(this.getClass().getSimpleName(), "Bootstrap: initializing metadata table by listing files and partitions"); // well. Hence, we have a special commit just for the initialization scenario.
List<DirectoryInfo> dirInfoList = listAllPartitions(dataMetaClient); initialCommit(createInstantTime);
// During bootstrap, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out of these
// large number of files and calling the existing update(HoodieCommitMetadata) function does not scale well.
// Hence, we have a special commit just for the bootstrap scenario.
bootstrapCommit(dirInfoList, createInstantTime);
return true; return true;
} }
@@ -651,6 +641,14 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
} }
} }
private MetadataRecordsGenerationParams getRecordsGenerationParams() {
return new MetadataRecordsGenerationParams(
dataMetaClient, enabledPartitionTypes, dataWriteConfig.getBloomFilterType(),
dataWriteConfig.getBloomIndexParallelism(),
dataWriteConfig.isMetadataIndexColumnStatsForAllColumnsEnabled(),
dataWriteConfig.getColumnStatsIndexParallelism());
}
/** /**
* Interface to assist in converting commit metadata to List of HoodieRecords to be written to metadata table. * Interface to assist in converting commit metadata to List of HoodieRecords to be written to metadata table.
* 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.
@@ -681,8 +679,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(engineContext, enabledPartitionTypes, processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(
commitMetadata, dataMetaClient, dataWriteConfig.isMetadataIndexColumnStatsForAllColumnsEnabled(), instantTime), !isTableServiceAction); engineContext, commitMetadata, instantTime, getRecordsGenerationParams()), !isTableServiceAction);
} }
/** /**
@@ -693,8 +691,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(engineContext, enabledPartitionTypes, processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext,
cleanMetadata, dataMetaClient, instantTime), false); cleanMetadata, getRecordsGenerationParams(), instantTime), false);
} }
/** /**
@@ -706,7 +704,7 @@ 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(engineContext, processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext,
enabledPartitionTypes, metadataMetaClient.getActiveTimeline(), restoreMetadata, dataMetaClient, instantTime, metadataMetaClient.getActiveTimeline(), restoreMetadata, getRecordsGenerationParams(), instantTime,
metadata.getSyncedInstantTime()), false); metadata.getSyncedInstantTime()), false);
} }
@@ -732,8 +730,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
} }
Map<MetadataPartitionType, HoodieData<HoodieRecord>> records = Map<MetadataPartitionType, HoodieData<HoodieRecord>> records =
HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes, HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, metadataMetaClient.getActiveTimeline(),
metadataMetaClient.getActiveTimeline(), rollbackMetadata, dataMetaClient, instantTime, rollbackMetadata, getRecordsGenerationParams(), instantTime,
metadata.getSyncedInstantTime(), wasSynced); metadata.getSyncedInstantTime(), wasSynced);
commit(instantTime, records, false); commit(instantTime, records, false);
} }
@@ -845,20 +843,29 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
} }
/** /**
* This is invoked to bootstrap metadata table for a dataset. Bootstrap Commit has special handling mechanism due to its scale compared to * This is invoked to initialize metadata table for a dataset. Bootstrap Commit has special handling mechanism due to its scale compared to
* other regular commits. * other regular commits.
*
*/ */
protected void bootstrapCommit(List<DirectoryInfo> partitionInfoList, String createInstantTime) { private void initialCommit(String createInstantTime) {
List<String> partitions = partitionInfoList.stream().map(p -> // List all partitions in the basePath of the containing dataset
p.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : p.getRelativePath()).collect(Collectors.toList()); LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath());
final int totalFiles = partitionInfoList.stream().mapToInt(p -> p.getTotalFiles()).sum(); engineContext.setJobStatus(this.getClass().getSimpleName(), "Initializing metadata table by listing files and partitions");
List<DirectoryInfo> partitionInfoList = listAllPartitions(dataMetaClient);
List<String> partitions = new ArrayList<>();
AtomicLong totalFiles = new AtomicLong(0);
Map<String, Map<String, Long>> partitionToFilesMap = partitionInfoList.stream().map(p -> {
final String partitionName = HoodieTableMetadataUtil.getPartition(p.getRelativePath());
partitions.add(partitionName);
totalFiles.addAndGet(p.getTotalFiles());
return Pair.of(partitionName, p.getFileNameToSizeMap());
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>(); 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 bootstrapping of a fresh table, there won't be any partitions, but we need to make a boostrap commit // in case of initializing of a fresh table, there won't be any partitions, but we need to make a boostrap commit
final HoodieData<HoodieRecord> allPartitionRecordsRDD = engineContext.parallelize( final HoodieData<HoodieRecord> allPartitionRecordsRDD = engineContext.parallelize(
Collections.singletonList(allPartitionRecord), 1); Collections.singletonList(allPartitionRecord), 1);
partitionToRecordsMap.put(MetadataPartitionType.FILES, allPartitionRecordsRDD); partitionToRecordsMap.put(MetadataPartitionType.FILES, allPartitionRecordsRDD);
@@ -866,7 +873,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
return; return;
} }
HoodieData<HoodieRecord> partitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); HoodieData<HoodieRecord> filesPartitionRecords = 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 -> {
Map<String, Long> fileNameToSizeMap = partitionInfo.getFileNameToSizeMap(); Map<String, Long> fileNameToSizeMap = partitionInfo.getFileNameToSizeMap();
@@ -878,29 +885,41 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
// Record which saves files within a partition // Record which saves files within a partition
return HoodieMetadataPayload.createPartitionFilesRecord( return HoodieMetadataPayload.createPartitionFilesRecord(
partitionInfo.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : partitionInfo.getRelativePath(), Option.of(validFileNameToSizeMap), Option.empty()); HoodieTableMetadataUtil.getPartition(partitionInfo.getRelativePath()), Option.of(validFileNameToSizeMap), Option.empty());
}); });
partitionRecords = partitionRecords.union(fileListRecords); filesPartitionRecords = filesPartitionRecords.union(fileListRecords);
}
ValidationUtils.checkState(filesPartitionRecords.count() == (partitions.size() + 1));
partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecords);
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) {
final HoodieData<HoodieRecord> recordsRDD = HoodieTableMetadataUtil.convertFilesToBloomFilterRecords(
engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams(), createInstantTime);
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, recordsRDD);
}
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) {
final HoodieData<HoodieRecord> recordsRDD = HoodieTableMetadataUtil.convertFilesToColumnStatsRecords(
engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams());
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, recordsRDD);
} }
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));
partitionToRecordsMap.put(MetadataPartitionType.FILES, partitionRecords);
commit(createInstantTime, partitionToRecordsMap, false); commit(createInstantTime, partitionToRecordsMap, false);
} }
/** /**
* A class which represents a directory and the files and directories inside it. * A class which represents a directory and the files and directories inside it.
* * <p>
* A {@code PartitionFileInfo} object saves the name of the partition and various properties requires of each file * A {@code PartitionFileInfo} object saves the name of the partition and various properties requires of each file
* required for bootstrapping the metadata table. Saving limited properties reduces the total memory footprint when * required for initializing the metadata table. Saving limited properties reduces the total memory footprint when
* a very large number of files are present in the dataset being bootstrapped. * a very large number of files are present in the dataset being initialized.
*/ */
static class DirectoryInfo implements Serializable { static class DirectoryInfo implements Serializable {
// Relative path of the directory (relative to the base directory) // Relative path of the directory (relative to the base directory)
private final String relativePath; private final String relativePath;
// Map of filenames within this partition to their respective sizes // Map of filenames within this partition to their respective sizes
private HashMap<String, Long> filenameToSizeMap; private final HashMap<String, Long> filenameToSizeMap;
// List of directories within this partition // List of directories within this partition
private final List<Path> subDirectories = new ArrayList<>(); private final List<Path> subDirectories = new ArrayList<>();
// Is this a hoodie partition // Is this a hoodie partition

View File

@@ -97,7 +97,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
return (HoodieWriteableTestTable) super.forCommit(instantTime); return (HoodieWriteableTestTable) super.forCommit(instantTime);
} }
public HoodieWriteableTestTable withInserts(String partition, String fileId, List<HoodieRecord> records, TaskContextSupplier contextSupplier) throws Exception { public Path withInserts(String partition, String fileId, List<HoodieRecord> records, TaskContextSupplier contextSupplier) throws Exception {
FileCreateUtils.createPartitionMetaFile(basePath, partition); FileCreateUtils.createPartitionMetaFile(basePath, partition);
String fileName = baseFileName(currentInstantTime, fileId); String fileName = baseFileName(currentInstantTime, fileId);
@@ -151,7 +151,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
} }
} }
return this; return baseFilePath;
} }
public Map<String, List<HoodieLogFile>> withLogAppends(List<HoodieRecord> records) throws Exception { public Map<String, List<HoodieLogFile>> withLogAppends(List<HoodieRecord> records) throws Exception {

View File

@@ -91,7 +91,7 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
Option<String> inflightInstantTimestamp) { Option<String> inflightInstantTimestamp) {
try { try {
if (enabled) { if (enabled) {
bootstrapIfNeeded(engineContext, dataMetaClient, actionMetadata, inflightInstantTimestamp); initializeIfNeeded(dataMetaClient, actionMetadata, inflightInstantTimestamp);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.error("Failed to initialize metadata table. Disabling the writer.", e); LOG.error("Failed to initialize metadata table. Disabling the writer.", e);

View File

@@ -108,7 +108,8 @@ public class HoodieFlinkWriteableTestTable extends HoodieWriteableTestTable {
} }
public HoodieFlinkWriteableTestTable withInserts(String partition, String fileId, List<HoodieRecord> records) throws Exception { public HoodieFlinkWriteableTestTable withInserts(String partition, String fileId, List<HoodieRecord> records) throws Exception {
return (HoodieFlinkWriteableTestTable) withInserts(partition, fileId, records, new org.apache.hudi.client.FlinkTaskContextSupplier(null)); withInserts(partition, fileId, records, new org.apache.hudi.client.FlinkTaskContextSupplier(null));
return this;
} }
public Map<String, List<HoodieLogFile>> withLogAppends(List<HoodieRecord> records) throws Exception { public Map<String, List<HoodieLogFile>> withLogAppends(List<HoodieRecord> records) throws Exception {

View File

@@ -20,8 +20,7 @@ package org.apache.hudi.index.bloom;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.utils.LazyIterableIterator; import org.apache.hudi.client.utils.LazyIterableIterator;
import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
import org.apache.hudi.common.fs.FSUtils; 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.HoodieKey; import org.apache.hudi.common.model.HoodieKey;
@@ -37,8 +36,6 @@ import org.apache.log4j.Logger;
import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.Function2;
import scala.Tuple2; import scala.Tuple2;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
@@ -113,7 +110,7 @@ public class HoodieMetadataBloomIndexCheckFunction implements
} }
List<Pair<String, String>> partitionNameFileNameList = new ArrayList<>(fileToKeysMap.keySet()); List<Pair<String, String>> partitionNameFileNameList = new ArrayList<>(fileToKeysMap.keySet());
Map<Pair<String, String>, ByteBuffer> fileToBloomFilterMap = Map<Pair<String, String>, BloomFilter> fileToBloomFilterMap =
hoodieTable.getMetadataTable().getBloomFilters(partitionNameFileNameList); hoodieTable.getMetadataTable().getBloomFilters(partitionNameFileNameList);
final AtomicInteger totalKeys = new AtomicInteger(0); final AtomicInteger totalKeys = new AtomicInteger(0);
@@ -126,11 +123,7 @@ public class HoodieMetadataBloomIndexCheckFunction implements
if (!fileToBloomFilterMap.containsKey(partitionPathFileNamePair)) { if (!fileToBloomFilterMap.containsKey(partitionPathFileNamePair)) {
throw new HoodieIndexException("Failed to get the bloom filter for " + partitionPathFileNamePair); throw new HoodieIndexException("Failed to get the bloom filter for " + partitionPathFileNamePair);
} }
final ByteBuffer fileBloomFilterByteBuffer = fileToBloomFilterMap.get(partitionPathFileNamePair); final BloomFilter fileBloomFilter = fileToBloomFilterMap.get(partitionPathFileNamePair);
HoodieDynamicBoundedBloomFilter fileBloomFilter =
new HoodieDynamicBoundedBloomFilter(StandardCharsets.UTF_8.decode(fileBloomFilterByteBuffer).toString(),
BloomFilterTypeCode.DYNAMIC_V0);
List<String> candidateRecordKeys = new ArrayList<>(); List<String> candidateRecordKeys = new ArrayList<>();
hoodieKeyList.forEach(hoodieKey -> { hoodieKeyList.forEach(hoodieKey -> {

View File

@@ -113,7 +113,7 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
}); });
if (enabled) { if (enabled) {
bootstrapIfNeeded(engineContext, dataMetaClient, actionMetadata, inflightInstantTimestamp); initializeIfNeeded(dataMetaClient, actionMetadata, inflightInstantTimestamp);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.error("Failed to initialize metadata table. Disabling the writer.", e); LOG.error("Failed to initialize metadata table. Disabling the writer.", e);

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.client.functional; package org.apache.hudi.client.functional;
import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieAvroRecord;
@@ -47,7 +48,6 @@ import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner; import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner;
import org.apache.hudi.testutils.Assertions; import org.apache.hudi.testutils.Assertions;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.apache.hudi.testutils.MetadataMergeWriteStatus;
@@ -63,6 +63,7 @@ import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.MethodSource;
import java.io.IOException; import java.io.IOException;
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;
@@ -82,17 +83,24 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
@Tag("functional") @Tag("functional")
public class TestHoodieIndex extends HoodieClientTestHarness { public class TestHoodieIndex extends TestHoodieMetadataBase {
private static Stream<Arguments> indexTypeParams() { private static Stream<Arguments> indexTypeParams() {
// IndexType, populateMetaFields, enableMetadataIndex
Object[][] data = new Object[][] { Object[][] data = new Object[][] {
{IndexType.BLOOM, true}, {IndexType.BLOOM, true, true},
{IndexType.GLOBAL_BLOOM, true}, {IndexType.BLOOM, true, false},
{IndexType.SIMPLE, true}, {IndexType.GLOBAL_BLOOM, true, true},
{IndexType.GLOBAL_SIMPLE, true}, {IndexType.GLOBAL_BLOOM, true, false},
{IndexType.SIMPLE, false}, {IndexType.SIMPLE, true, true},
{IndexType.GLOBAL_SIMPLE, false}, {IndexType.SIMPLE, true, false},
{IndexType.BUCKET, false} {IndexType.SIMPLE, false, true},
{IndexType.SIMPLE, false, false},
{IndexType.GLOBAL_SIMPLE, true, true},
{IndexType.GLOBAL_SIMPLE, false, true},
{IndexType.GLOBAL_SIMPLE, false, false},
{IndexType.BUCKET, false, true},
{IndexType.BUCKET, false, false}
}; };
return Stream.of(data).map(Arguments::of); return Stream.of(data).map(Arguments::of);
} }
@@ -103,11 +111,11 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
private HoodieIndex index; private HoodieIndex index;
private HoodieWriteConfig config; private HoodieWriteConfig config;
private void setUp(IndexType indexType, boolean populateMetaFields) throws Exception { private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
setUp(indexType, populateMetaFields, true); setUp(indexType, populateMetaFields, true, enableMetadataIndex);
} }
private void setUp(IndexType indexType, boolean populateMetaFields, boolean rollbackUsingMarkers) throws Exception { private void setUp(IndexType indexType, boolean populateMetaFields, boolean rollbackUsingMarkers, boolean enableMetadataIndex) throws Exception {
this.indexType = indexType; this.indexType = indexType;
initPath(); initPath();
initSparkContexts(); initSparkContexts();
@@ -123,8 +131,13 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
.withRollbackUsingMarkers(rollbackUsingMarkers) .withRollbackUsingMarkers(rollbackUsingMarkers)
.withIndexConfig(indexBuilder.build()) .withIndexConfig(indexBuilder.build())
.withAutoCommit(false) .withAutoCommit(false)
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.withMetadataIndexBloomFilter(enableMetadataIndex)
.withMetadataIndexColumnStats(enableMetadataIndex)
.build())
.withLayoutConfig(HoodieLayoutConfig.newBuilder().fromProperties(indexBuilder.build().getProps()) .withLayoutConfig(HoodieLayoutConfig.newBuilder().fromProperties(indexBuilder.build().getProps())
.withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build()).build(); .withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build())
.build();
writeClient = getHoodieWriteClient(config); writeClient = getHoodieWriteClient(config);
this.index = writeClient.getIndex(); this.index = writeClient.getIndex();
} }
@@ -136,8 +149,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
@ParameterizedTest @ParameterizedTest
@MethodSource("indexTypeParams") @MethodSource("indexTypeParams")
public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populateMetaFields) throws Exception { public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
setUp(indexType, populateMetaFields); setUp(indexType, populateMetaFields, enableMetadataIndex);
String newCommitTime = "001"; String newCommitTime = "001";
int totalRecords = 10 + random.nextInt(20); int totalRecords = 10 + random.nextInt(20);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords); List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
@@ -186,8 +199,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
@ParameterizedTest @ParameterizedTest
@MethodSource("indexTypeParams") @MethodSource("indexTypeParams")
public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean populateMetaFields) throws Exception { public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
setUp(indexType, populateMetaFields); setUp(indexType, populateMetaFields, enableMetadataIndex);
String newCommitTime = "001"; String newCommitTime = "001";
int totalRecords = 10 + random.nextInt(20); int totalRecords = 10 + random.nextInt(20);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords); List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
@@ -236,8 +249,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
@ParameterizedTest @ParameterizedTest
@MethodSource("indexTypeParams") @MethodSource("indexTypeParams")
public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields) throws Exception { public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
setUp(indexType, populateMetaFields, false); setUp(indexType, populateMetaFields, false, enableMetadataIndex);
String newCommitTime = writeClient.startCommit(); String newCommitTime = writeClient.startCommit();
int totalRecords = 20 + random.nextInt(20); int totalRecords = 20 + random.nextInt(20);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords); List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
@@ -286,17 +299,21 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
} }
private static Stream<Arguments> regularIndexTypeParams() { private static Stream<Arguments> regularIndexTypeParams() {
// IndexType, populateMetaFields, enableMetadataIndex
Object[][] data = new Object[][] { Object[][] data = new Object[][] {
{IndexType.BLOOM, true}, // TODO (codope): Enabling metadata index is flaky. Both bloom_filter and col_stats get generated but loading column ranges from the index is failing.
{IndexType.SIMPLE, true} // {IndexType.BLOOM, true, true},
{IndexType.BLOOM, true, false},
{IndexType.SIMPLE, true, true},
{IndexType.SIMPLE, true, false}
}; };
return Stream.of(data).map(Arguments::of); return Stream.of(data).map(Arguments::of);
} }
@ParameterizedTest @ParameterizedTest
@MethodSource("regularIndexTypeParams") @MethodSource("regularIndexTypeParams")
public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean populateMetaFields) throws Exception { public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
setUp(indexType, populateMetaFields); setUp(indexType, populateMetaFields, enableMetadataIndex);
String p1 = "2016/01/31"; String p1 = "2016/01/31";
String p2 = "2015/01/31"; String p2 = "2015/01/31";
String rowKey1 = UUID.randomUUID().toString(); String rowKey1 = UUID.randomUUID().toString();
@@ -320,7 +337,9 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
HoodieRecord record4 = HoodieRecord record4 =
new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
String newCommitTime = writeClient.startCommit();
metaClient = HoodieTableMetaClient.reload(metaClient);
writeClient.upsert(recordRDD, newCommitTime);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable); JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
@@ -330,20 +349,42 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
assertFalse(record.isCurrentLocationKnown()); assertFalse(record.isCurrentLocationKnown());
} }
// We create three parquet file, each having one record. (two different partitions) // We create three parquet files, each having one record (two different partitions)
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
String fileId1 = testTable.addCommit("001").getFileIdWithInserts(p1, record1); final String fileId1 = "fileID1";
String fileId2 = testTable.addCommit("002").getFileIdWithInserts(p1, record2); final String fileId2 = "fileID2";
String fileId3 = testTable.addCommit("003").getFileIdWithInserts(p2, record4); final String fileId3 = "fileID3";
Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
Path baseFilePath = testTable.forCommit("0000001").withInserts(p1, fileId1, Collections.singletonList(record1));
long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(p1, k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation("0000001", WriteOperationType.UPSERT, Arrays.asList(p1, p2),
partitionToFilesNameLengthMap, false, false);
partitionToFilesNameLengthMap.clear();
baseFilePath = testTable.forCommit("0000002").withInserts(p1, fileId2, Collections.singletonList(record2));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(p1, k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation("0000002", WriteOperationType.UPSERT, Arrays.asList(p1, p2),
partitionToFilesNameLengthMap, false, false);
partitionToFilesNameLengthMap.clear();
baseFilePath = testTable.forCommit("0000003").withInserts(p2, fileId3, Collections.singletonList(record4));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(p2, k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation("0000003", WriteOperationType.UPSERT, Arrays.asList(p1, p2),
partitionToFilesNameLengthMap, false, false);
// We do the tag again // We do the tag again
metaClient = HoodieTableMetaClient.reload(metaClient); metaClient = HoodieTableMetaClient.reload(metaClient);
hoodieTable = HoodieSparkTable.create(config, context, metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient);
taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable); taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
List<HoodieRecord> records = taggedRecordRDD.collect();
// Check results // Check results
for (HoodieRecord record : taggedRecordRDD.collect()) { for (HoodieRecord record : records) {
if (record.getRecordKey().equals(rowKey1)) { if (record.getRecordKey().equals(rowKey1)) {
if (record.getPartitionPath().equals(p2)) { if (record.getPartitionPath().equals(p2)) {
assertEquals(record.getCurrentLocation().getFileId(), fileId3); assertEquals(record.getCurrentLocation().getFileId(), fileId3);
@@ -378,12 +419,17 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
@Test @Test
public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath() throws Exception { public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath() throws Exception {
setUp(IndexType.GLOBAL_SIMPLE, true); setUp(IndexType.GLOBAL_SIMPLE, true, true);
config = getConfigBuilder() config = getConfigBuilder()
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
.withGlobalSimpleIndexUpdatePartitionPath(true) .withGlobalSimpleIndexUpdatePartitionPath(true)
.withBloomIndexUpdatePartitionPath(true) .withBloomIndexUpdatePartitionPath(true)
.build()) .build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.enable(true)
.withMetadataIndexBloomFilter(true)
.withMetadataIndexColumnStats(true)
.build())
.build(); .build();
writeClient = getHoodieWriteClient(config); writeClient = getHoodieWriteClient(config);
index = writeClient.getIndex(); index = writeClient.getIndex();
@@ -432,7 +478,10 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
final String file1P1C0 = UUID.randomUUID().toString(); final String file1P1C0 = UUID.randomUUID().toString();
Map<String, List<Pair<String, Integer>>> c1PartitionToFilesNameLengthMap = new HashMap<>(); Map<String, List<Pair<String, Integer>>> c1PartitionToFilesNameLengthMap = new HashMap<>();
c1PartitionToFilesNameLengthMap.put(p1, Collections.singletonList(Pair.of(file1P1C0, 100))); // We have some records to be tagged (two different partitions)
Path baseFilePath = testTable.forCommit("1000").withInserts(p1, file1P1C0, Collections.singletonList(originalRecord));
long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
c1PartitionToFilesNameLengthMap.put(p1, Collections.singletonList(Pair.of(file1P1C0, Integer.valueOf((int) baseFileLength))));
testTable.doWriteOperation("1000", WriteOperationType.INSERT, Arrays.asList(p1), testTable.doWriteOperation("1000", WriteOperationType.INSERT, Arrays.asList(p1),
c1PartitionToFilesNameLengthMap, false, false); c1PartitionToFilesNameLengthMap, false, false);

View File

@@ -18,8 +18,8 @@
package org.apache.hudi.client.functional; package org.apache.hudi.client.functional;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.avro.model.HoodieMetadataRecord;
import org.apache.hudi.client.HoodieTimelineArchiver;
import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCleaningPolicy;
@@ -52,8 +52,9 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter;
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;
import org.apache.hudi.client.HoodieTimelineArchiver;
import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieClientTestHarness;
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 org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
@@ -437,5 +438,4 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
} }
return builder.build(); return builder.build();
} }
} }

View File

@@ -18,8 +18,6 @@
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.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;
@@ -28,6 +26,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieAvroRecord;
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.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
@@ -37,10 +36,14 @@ 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.index.HoodieIndex;
import org.apache.hudi.index.HoodieIndexUtils; 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.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;
@@ -49,10 +52,11 @@ 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.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@@ -61,6 +65,8 @@ 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;
@@ -75,8 +81,13 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
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 = // rangePruning, treeFiltering, bucketizedChecking
new Object[][]{{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}}; Object[][] data = 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);
} }
@@ -87,6 +98,11 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
initFileSystem(); initFileSystem();
// We have some records to be tagged (two different partitions) // We have some records to be tagged (two different partitions)
initMetaClient(); initMetaClient();
HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withIndexConfig(indexBuilder.build())
.build();
writeClient = getHoodieWriteClient(config);
} }
@AfterEach @AfterEach
@@ -112,7 +128,7 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
HoodieBloomIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieBloomIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
// Create some partitions, and put some files // Create some partitions, and put some files
// "2016/01/21": 0 file // "2016/01/21": 0 file
@@ -142,10 +158,40 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
// Still 0, as no valid commit // Still 0, as no valid commit
assertEquals(0, filesList.size()); assertEquals(0, filesList.size());
testTable.addCommit("20160401010101").withInserts("2016/04/01", "2"); final String fileId1 = "1";
testTable.addCommit("20150312101010").withInserts("2015/03/12", "1") final String fileId2 = "2";
.withInserts("2015/03/12", "3", record1) final String fileId3 = "3";
.withInserts("2015/03/12", "4", record2, record3, record4); final String fileId4 = "4";
final Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
String commitTime = "20160401010101";
Path baseFilePath = testTable.forCommit(commitTime).withInserts(partitions.get(1), fileId2, Collections.emptyList());
long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partitions.get(1),
k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Arrays.asList(partitions.get(1)),
partitionToFilesNameLengthMap, false, false);
commitTime = "20150312101010";
partitionToFilesNameLengthMap.clear();
testTable.forCommit(commitTime);
baseFilePath = testTable.withInserts(partitions.get(2), fileId1, Collections.emptyList());
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partitions.get(2),
k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
baseFilePath = testTable.withInserts(partitions.get(2), fileId3, Collections.singletonList(record1));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partitions.get(2),
k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
baseFilePath = testTable.withInserts(partitions.get(2), fileId4, Arrays.asList(record2, record3, record4));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partitions.get(2),
k -> new ArrayList<>()).add(Pair.of(fileId4, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Arrays.asList(partitions.get(2)),
partitionToFilesNameLengthMap, false, false);
filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
assertEquals(4, filesList.size()); assertEquals(4, filesList.size());
@@ -229,9 +275,20 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
// record2, record3). // record2, record3).
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
filter.add(record3.getRecordKey()); filter.add(record3.getRecordKey());
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter, metadataWriter);
String fileId = testTable.addCommit("000").getFileIdWithInserts(partition, record1, record2);
String filename = testTable.getBaseFileNameById(fileId); final Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
final String commitTime = "0000001";
final String fileId = UUID.randomUUID().toString();
Path baseFilePath = testTable.forCommit(commitTime)
.withInserts(partition, fileId, Arrays.asList(record1, record2));
long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partition,
k -> new ArrayList<>()).add(Pair.of(fileId, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition),
partitionToFilesNameLengthMap, false, false);
final String filename = testTable.getBaseFileNameById(fileId);
// The bloom filter contains 3 records // The bloom filter contains 3 records
assertTrue(filter.mightContain(record1.getRecordKey())); assertTrue(filter.mightContain(record1.getRecordKey()));
@@ -305,7 +362,7 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
// Also create the metadata and config // Also create the metadata and config
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
// Let's tag // Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
@@ -316,10 +373,39 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
assertFalse(record.isCurrentLocationKnown()); assertFalse(record.isCurrentLocationKnown());
} }
final Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
final String partition1 = "2016/01/31";
final String partition2 = "2015/01/31";
// We create three parquet file, each having one record. (two different partitions) // We create three parquet file, each having one record. (two different partitions)
String fileId1 = testTable.addCommit("001").getFileIdWithInserts("2016/01/31", record1); final String fileId1 = UUID.randomUUID().toString();
String fileId2 = testTable.addCommit("002").getFileIdWithInserts("2016/01/31", record2); final String commit1 = "0000001";
String fileId3 = testTable.addCommit("003").getFileIdWithInserts("2015/01/31", record4); Path baseFilePath = testTable.forCommit(commit1).withInserts(partition1, fileId1, Collections.singletonList(record1));
long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partition1,
k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commit1, WriteOperationType.UPSERT, Collections.singletonList(partition1),
partitionToFilesNameLengthMap, false, false);
final String fileId2 = UUID.randomUUID().toString();
final String commit2 = "0000002";
baseFilePath = testTable.forCommit(commit2).withInserts(partition1, fileId2, Collections.singletonList(record2));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.clear();
partitionToFilesNameLengthMap.computeIfAbsent(partition1,
k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commit2, WriteOperationType.UPSERT, Collections.singletonList(partition1),
partitionToFilesNameLengthMap, false, false);
final String fileId3 = UUID.randomUUID().toString();
final String commit3 = "0000003";
baseFilePath = testTable.forCommit(commit3).withInserts(partition2, fileId3, Collections.singletonList(record4));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.clear();
partitionToFilesNameLengthMap.computeIfAbsent(partition2,
k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commit3, WriteOperationType.UPSERT, Collections.singletonList(partition2),
partitionToFilesNameLengthMap, false, false);
// We do the tag again // We do the tag again
taggedRecordRDD = tagLocation(bloomIndex, recordRDD, HoodieSparkTable.create(config, context, metaClient)); taggedRecordRDD = tagLocation(bloomIndex, recordRDD, HoodieSparkTable.create(config, context, metaClient));
@@ -327,7 +413,7 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
// Check results // Check results
for (HoodieRecord record : taggedRecordRDD.collect()) { for (HoodieRecord record : taggedRecordRDD.collect()) {
if (record.getRecordKey().equals(rowKey1)) { if (record.getRecordKey().equals(rowKey1)) {
if (record.getPartitionPath().equals("2015/01/31")) { if (record.getPartitionPath().equals(partition2)) {
assertEquals(record.getCurrentLocation().getFileId(), fileId3); assertEquals(record.getCurrentLocation().getFileId(), fileId3);
} else { } else {
assertEquals(record.getCurrentLocation().getFileId(), fileId1); assertEquals(record.getCurrentLocation().getFileId(), fileId1);
@@ -370,7 +456,7 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
// Also create the metadata and config // Also create the metadata and config
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
// Let's tag // Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
@@ -387,10 +473,38 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
assertTrue(!record._2.isPresent()); assertTrue(!record._2.isPresent());
} }
final String partition1 = "2016/01/31";
final String partition2 = "2015/01/31";
final String fileId1 = UUID.randomUUID().toString();
final String fileId2 = UUID.randomUUID().toString();
final String fileId3 = UUID.randomUUID().toString();
final Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
// We create three parquet file, each having one record. (two different partitions) // We create three parquet file, each having one record. (two different partitions)
String fileId1 = testTable.addCommit("001").getFileIdWithInserts("2016/01/31", record1); final String commit1 = "0000001";
String fileId2 = testTable.addCommit("002").getFileIdWithInserts("2016/01/31", record2); Path baseFilePath = testTable.forCommit(commit1).withInserts(partition1, fileId1, Collections.singletonList(record1));
String fileId3 = testTable.addCommit("003").getFileIdWithInserts("2015/01/31", record4); long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partition1,
k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commit1, WriteOperationType.UPSERT, Collections.singletonList(partition1),
partitionToFilesNameLengthMap, false, false);
final String commit2 = "0000002";
partitionToFilesNameLengthMap.clear();
baseFilePath = testTable.forCommit(commit2).withInserts(partition1, fileId2, Collections.singletonList(record2));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partition1,
k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commit2, WriteOperationType.UPSERT, Collections.singletonList(partition1),
partitionToFilesNameLengthMap, false, false);
final String commit3 = "0000003";
partitionToFilesNameLengthMap.clear();
baseFilePath = testTable.forCommit(commit3).withInserts(partition2, fileId3, Collections.singletonList(record4));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(partition2,
k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commit3, WriteOperationType.UPSERT, Collections.singletonList(partition2),
partitionToFilesNameLengthMap, false, false);
// We do the tag again // We do the tag again
metaClient = HoodieTableMetaClient.reload(metaClient); metaClient = HoodieTableMetaClient.reload(metaClient);
@@ -409,7 +523,7 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
assertEquals(fileId1, record._2.get().getRight()); assertEquals(fileId1, record._2.get().getRight());
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) { } else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent()); assertTrue(record._2.isPresent());
if (record._1.getPartitionPath().equals("2015/01/31")) { if (record._1.getPartitionPath().equals(partition2)) {
assertEquals(fileId3, record._2.get().getRight()); assertEquals(fileId3, record._2.get().getRight());
} else { } else {
assertEquals(fileId2, record._2.get().getRight()); assertEquals(fileId2, record._2.get().getRight());

View File

@@ -18,22 +18,25 @@
package org.apache.hudi.index.bloom; package org.apache.hudi.index.bloom;
import org.apache.hudi.client.functional.TestHoodieMetadataBase;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieAvroRecord;
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.WriteOperationType;
import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.testutils.RawTripTestPayload;
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.data.HoodieJavaPairRDD; import org.apache.hudi.data.HoodieJavaPairRDD;
import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.data.HoodieJavaRDD;
import org.apache.hudi.index.HoodieIndex;
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.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;
@@ -41,12 +44,14 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.io.IOException; import java.io.IOException;
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.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.UUID;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import scala.Tuple2; import scala.Tuple2;
@@ -59,7 +64,7 @@ import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { public class TestHoodieGlobalBloomIndex extends TestHoodieMetadataBase {
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieGlobalBloomIndex.class, "/exampleSchema.avsc", true); private static final Schema SCHEMA = getSchemaFromResource(TestHoodieGlobalBloomIndex.class, "/exampleSchema.avsc", true);
@@ -67,7 +72,13 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
public void setUp() throws Exception { public void setUp() throws Exception {
initSparkContexts(); initSparkContexts();
initPath(); initPath();
initFileSystem();
initMetaClient(); initMetaClient();
HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.GLOBAL_BLOOM);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withIndexConfig(indexBuilder.build())
.build();
writeClient = getHoodieWriteClient(config);
} }
@AfterEach @AfterEach
@@ -81,13 +92,15 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
HoodieGlobalBloomIndex index = HoodieGlobalBloomIndex index =
new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
// Create some partitions, and put some files, along with the meta file // Create some partitions, and put some files, along with the meta file
// "2016/01/21": 0 file // "2016/01/21": 0 file
// "2016/04/01": 1 file (2_0_20160401010101.parquet) // "2016/04/01": 1 file (2_0_20160401010101.parquet)
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet) // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12"); final String p1 = "2016/01/21";
final String p2 = "2016/04/01";
final String p3 = "2015/03/12";
RawTripTestPayload rowChange1 = RawTripTestPayload rowChange1 =
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
@@ -107,16 +120,46 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
// 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(p1, p2);
// partitions will NOT be respected by this loadInvolvedFiles(...) call // partitions will NOT be respected by this loadInvolvedFiles(...) call
List<Pair<String, BloomIndexFileInfo>> filesList = index.loadColumnRangesFromFiles(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());
testTable.addCommit("20160401010101").withInserts("2016/04/01", "2"); final String fileId1 = "1";
testTable.addCommit("20150312101010").withInserts("2015/03/12", "1") final String fileId2 = "2";
.withInserts("2015/03/12", "3", record1) final String fileId3 = "3";
.withInserts("2015/03/12", "4", record2, record3, record4); final String fileId4 = "4";
final Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
final String c1 = "20160401010101";
Path baseFilePath = testTable.forCommit(c1).withInserts(p2, fileId2, Collections.emptyList());
long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(p2,
k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(c1, WriteOperationType.UPSERT, Collections.singletonList(p2),
partitionToFilesNameLengthMap, false, false);
final String c2 = "20150312101010";
testTable.forCommit(c2);
baseFilePath = testTable.withInserts(p3, fileId1, Collections.emptyList());
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.clear();
partitionToFilesNameLengthMap.computeIfAbsent(p3,
k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
baseFilePath = testTable.withInserts(p3, fileId3, Collections.singletonList(record1));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(p3,
k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
baseFilePath = testTable.withInserts(p3, fileId4, Arrays.asList(record2, record3, record4));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(p3,
k -> new ArrayList<>()).add(Pair.of(fileId4, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(c2, WriteOperationType.UPSERT, Collections.singletonList(p3),
partitionToFilesNameLengthMap, false, false);
filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
assertEquals(4, filesList.size()); assertEquals(4, filesList.size());
@@ -185,17 +228,21 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
@Test @Test
public void testTagLocation() throws Exception { public void testTagLocation() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(false).build()).build(); .withIndexConfig(HoodieIndexConfig.newBuilder()
HoodieGlobalBloomIndex index = .withIndexType(HoodieIndex.IndexType.GLOBAL_BLOOM)
new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); .withBloomIndexUpdatePartitionPath(false)
.build())
.build();
HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
// Create some partitions, and put some files, along with the meta file // Create some partitions, and put some files, along with the meta file
// "2016/01/21": 0 file // "2016/01/21": 0 file
// "2016/04/01": 1 file (2_0_20160401010101.parquet) // "2016/04/01": 1 file (2_0_20160401010101.parquet)
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet) // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12"); final String partition2 = "2016/04/01";
final String partition3 = "2015/03/12";
RawTripTestPayload rowChange1 = RawTripTestPayload rowChange1 =
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
@@ -223,13 +270,49 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
HoodieRecord record5 = HoodieRecord record5 =
new HoodieAvroRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()), rowChange5); new HoodieAvroRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()), rowChange5);
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5)); final String fileId1 = UUID.randomUUID().toString();
final String fileId2 = UUID.randomUUID().toString();
final String fileId3 = UUID.randomUUID().toString();
final String fileId4 = UUID.randomUUID().toString();
final Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
// 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
String fileId1 = testTable.addCommit("1000").getFileIdWithInserts("2016/04/01", record1); String commitTime = "0000001";
String fileId2 = testTable.addCommit("2000").getFileIdWithInserts("2015/03/12"); Path baseFilePath = testTable.forCommit(commitTime).withInserts(partition2, fileId1, Collections.singletonList(record1));
String fileId3 = testTable.addCommit("3000").getFileIdWithInserts("2015/03/12", record2); long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
String fileId4 = testTable.addCommit("4000").getFileIdWithInserts("2015/03/12", record4); partitionToFilesNameLengthMap.computeIfAbsent(partition2,
k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition2),
partitionToFilesNameLengthMap, false, false);
commitTime = "0000002";
baseFilePath = testTable.forCommit(commitTime).withInserts(partition3, fileId2, Collections.emptyList());
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.clear();
partitionToFilesNameLengthMap.computeIfAbsent(partition3,
k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition3),
partitionToFilesNameLengthMap, false, false);
commitTime = "0000003";
baseFilePath = testTable.forCommit(commitTime).withInserts(partition3, fileId3, Collections.singletonList(record2));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.clear();
partitionToFilesNameLengthMap.computeIfAbsent(partition3,
k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition3),
partitionToFilesNameLengthMap, false, false);
commitTime = "0000004";
baseFilePath = testTable.forCommit(commitTime).withInserts(partition3, fileId4, Collections.singletonList(record4));
baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.clear();
partitionToFilesNameLengthMap.computeIfAbsent(partition3,
k -> new ArrayList<>()).add(Pair.of(fileId4, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition3),
partitionToFilesNameLengthMap, false, false);
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
// partitions will NOT be respected by this loadInvolvedFiles(...) call // partitions will NOT be respected by this loadInvolvedFiles(...) call
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable); JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
@@ -266,12 +349,15 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
public void testTagLocationWhenShouldUpdatePartitionPath() throws Exception { public void testTagLocationWhenShouldUpdatePartitionPath() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder() HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath) .withPath(basePath)
.withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build()) .withIndexConfig(HoodieIndexConfig.newBuilder()
.withIndexType(HoodieIndex.IndexType.GLOBAL_BLOOM)
.withBloomIndexUpdatePartitionPath(true)
.build())
.build(); .build();
HoodieGlobalBloomIndex index = HoodieGlobalBloomIndex index =
new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
final String p1 = "2016/01/31"; final String p1 = "2016/01/31";
final String p2 = "2016/02/28"; final String p2 = "2016/02/28";
@@ -309,7 +395,16 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()), new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()),
incomingPayloadSamePartition); incomingPayloadSamePartition);
testTable.addCommit("1000").getFileIdWithInserts(p1, originalRecord); final String fileId1 = UUID.randomUUID().toString();
final Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
final String commitTime = "0000001";
Path baseFilePath = testTable.forCommit(commitTime).withInserts(p1, fileId1, Collections.singletonList(originalRecord));
long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
partitionToFilesNameLengthMap.computeIfAbsent(p1,
k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Arrays.asList(p1),
partitionToFilesNameLengthMap, false, false);
// test against incoming record with a different partition // test against incoming record with a different partition
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord)); JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.table.HoodieTable;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
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;
@@ -65,7 +66,7 @@ public class HoodieSparkWriteableTestTable extends HoodieWriteableTestTable {
public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema,
HoodieTableMetadataWriter metadataWriter) { HoodieTableMetadataWriter metadataWriter) {
BloomFilter filter = BloomFilterFactory BloomFilter filter = BloomFilterFactory
.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); .createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.DYNAMIC_V0.name());
return of(metaClient, schema, filter, metadataWriter); return of(metaClient, schema, filter, metadataWriter);
} }
@@ -108,11 +109,11 @@ public class HoodieSparkWriteableTestTable extends HoodieWriteableTestTable {
} }
public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception { public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception {
return withInserts(partition, fileId, Arrays.asList(records)); withInserts(partition, fileId, Arrays.asList(records));
}
public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, List<HoodieRecord> records) throws Exception {
super.withInserts(partition, fileId, records, new SparkTaskContextSupplier());
return this; return this;
} }
public Path withInserts(String partition, String fileId, List<HoodieRecord> records) throws Exception {
return super.withInserts(partition, fileId, records, new SparkTaskContextSupplier());
}
} }

View File

@@ -169,6 +169,12 @@ public final class HoodieMetadataConfig extends HoodieConfig {
+ "store the column ranges and will be used for pruning files during the index lookups. " + "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."); + "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)
.sinceVersion("0.11.0")
.withDocumentation("Parallelism to use, when generating column stats index.");
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)
@@ -223,6 +229,10 @@ public final class HoodieMetadataConfig extends HoodieConfig {
return getIntOrDefault(METADATA_INDEX_COLUMN_STATS_FILE_GROUP_COUNT); return getIntOrDefault(METADATA_INDEX_COLUMN_STATS_FILE_GROUP_COUNT);
} }
public int getColumnStatsIndexParallelism() {
return getIntOrDefault(COLUMN_STATS_INDEX_PARALLELISM);
}
public boolean enableMetrics() { public boolean enableMetrics() {
return getBoolean(METRICS_ENABLE); return getBoolean(METRICS_ENABLE);
} }
@@ -285,6 +295,11 @@ public final class HoodieMetadataConfig extends HoodieConfig {
return this; return this;
} }
public Builder withColumnStatsIndexParallelism(int parallelism) {
metadataConfig.setValue(COLUMN_STATS_INDEX_PARALLELISM, String.valueOf(parallelism));
return this;
}
public Builder withMetadataIndexForAllColumns(boolean enable) { public Builder withMetadataIndexForAllColumns(boolean enable) {
metadataConfig.setValue(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS, String.valueOf(enable)); metadataConfig.setValue(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS, String.valueOf(enable));
return this; return this;

View File

@@ -18,12 +18,16 @@
package org.apache.hudi.common.model; package org.apache.hudi.common.model;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Comparator;
import java.util.Objects; import java.util.Objects;
import java.util.function.BiFunction;
/** /**
* Hoodie Range metadata. * Hoodie Range metadata.
*/ */
public class HoodieColumnRangeMetadata<T> { public class HoodieColumnRangeMetadata<T> implements Serializable {
private final String filePath; private final String filePath;
private final String columnName; private final String columnName;
private final T minValue; private final T minValue;
@@ -33,6 +37,20 @@ public class HoodieColumnRangeMetadata<T> {
private final long totalSize; private final long totalSize;
private final long totalUncompressedSize; private final long totalUncompressedSize;
public static final BiFunction<HoodieColumnRangeMetadata<Comparable>, HoodieColumnRangeMetadata<Comparable>, HoodieColumnRangeMetadata<Comparable>> COLUMN_RANGE_MERGE_FUNCTION =
(oldColumnRange, newColumnRange) -> new HoodieColumnRangeMetadata<>(
newColumnRange.getFilePath(),
newColumnRange.getColumnName(),
(Comparable) Arrays.asList(oldColumnRange.getMinValue(), newColumnRange.getMinValue())
.stream().filter(Objects::nonNull).min(Comparator.naturalOrder()).orElse(null),
(Comparable) Arrays.asList(oldColumnRange.getMinValue(), newColumnRange.getMinValue())
.stream().filter(Objects::nonNull).max(Comparator.naturalOrder()).orElse(null),
oldColumnRange.getNullCount() + newColumnRange.getNullCount(),
oldColumnRange.getValueCount() + newColumnRange.getValueCount(),
oldColumnRange.getTotalSize() + newColumnRange.getTotalSize(),
oldColumnRange.getTotalUncompressedSize() + newColumnRange.getTotalUncompressedSize()
);
public HoodieColumnRangeMetadata(final String filePath, final String columnName, final T minValue, final T maxValue, public HoodieColumnRangeMetadata(final String filePath, final String columnName, final T minValue, final T maxValue,
final long nullCount, long valueCount, long totalSize, long totalUncompressedSize) { final long nullCount, long valueCount, long totalSize, long totalUncompressedSize) {
this.filePath = filePath; this.filePath = filePath;
@@ -114,4 +132,18 @@ public class HoodieColumnRangeMetadata<T> {
+ ", totalUncompressedSize=" + totalUncompressedSize + ", totalUncompressedSize=" + totalUncompressedSize
+ '}'; + '}';
} }
/**
* Statistics that is collected in {@link org.apache.hudi.metadata.MetadataPartitionType#COLUMN_STATS} index.
*/
public static final class Stats {
public static final String VALUE_COUNT = "value_count";
public static final String NULL_COUNT = "null_count";
public static final String MIN = "min";
public static final String MAX = "max";
public static final String TOTAL_SIZE = "total_size";
public static final String TOTAL_UNCOMPRESSED_SIZE = "total_uncompressed_size";
private Stats() { }
}
} }

View File

@@ -19,9 +19,12 @@
package org.apache.hudi.common.model; package org.apache.hudi.common.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.hudi.common.util.Option;
import java.io.Serializable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map;
/** /**
* Statistics about a single Hoodie delta log operation. * Statistics about a single Hoodie delta log operation.
@@ -33,6 +36,7 @@ public class HoodieDeltaWriteStat extends HoodieWriteStat {
private long logOffset; private long logOffset;
private String baseFile; private String baseFile;
private List<String> logFiles = new ArrayList<>(); private List<String> logFiles = new ArrayList<>();
private Option<RecordsStats<? extends Map>> recordsStats = Option.empty();
public void setLogVersion(int logVersion) { public void setLogVersion(int logVersion) {
this.logVersion = logVersion; this.logVersion = logVersion;
@@ -69,4 +73,24 @@ public class HoodieDeltaWriteStat extends HoodieWriteStat {
public List<String> getLogFiles() { public List<String> getLogFiles() {
return logFiles; return logFiles;
} }
public void setRecordsStats(RecordsStats<? extends Map> stats) {
recordsStats = Option.of(stats);
}
public Option<RecordsStats<? extends Map>> getRecordsStats() {
return recordsStats;
}
public static class RecordsStats<T> implements Serializable {
private final T recordsStats;
public RecordsStats(T recordsStats) {
this.recordsStats = recordsStats;
}
public T getStats() {
return recordsStats;
}
}
} }

View File

@@ -19,10 +19,10 @@
package org.apache.hudi.metadata; package org.apache.hudi.metadata;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; import org.apache.hudi.avro.model.HoodieMetadataBloomFilter;
import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory;
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;
@@ -40,11 +40,15 @@ import org.apache.hudi.common.util.hash.ColumnIndexID;
import org.apache.hudi.common.util.hash.FileIndexID; import org.apache.hudi.common.util.hash.FileIndexID;
import org.apache.hudi.common.util.hash.PartitionIndexID; 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.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.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
@@ -143,9 +147,8 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
throws IOException { throws IOException {
if (isMetadataTableEnabled) { if (isMetadataTableEnabled) {
try { try {
List<Path> partitionPaths = partitions.stream().map(entry -> new Path(entry)).collect(Collectors.toList()); List<Path> partitionPaths = partitions.stream().map(Path::new).collect(Collectors.toList());
Map<String, FileStatus[]> partitionsFilesMap = fetchAllFilesInPartitionPaths(partitionPaths); return fetchAllFilesInPartitionPaths(partitionPaths);
return partitionsFilesMap;
} catch (Exception e) { } catch (Exception e) {
throw new HoodieMetadataException("Failed to retrieve files in partition from metadata", e); throw new HoodieMetadataException("Failed to retrieve files in partition from metadata", e);
} }
@@ -156,7 +159,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
} }
@Override @Override
public Option<ByteBuffer> getBloomFilter(final String partitionName, final String fileName) public Option<BloomFilter> getBloomFilter(final String partitionName, final String fileName)
throws HoodieMetadataException { throws HoodieMetadataException {
if (!isBloomFilterIndexEnabled) { if (!isBloomFilterIndexEnabled) {
LOG.error("Metadata bloom filter index is disabled!"); LOG.error("Metadata bloom filter index is disabled!");
@@ -164,7 +167,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
} }
final Pair<String, String> partitionFileName = Pair.of(partitionName, fileName); final Pair<String, String> partitionFileName = Pair.of(partitionName, fileName);
Map<Pair<String, String>, ByteBuffer> bloomFilters = getBloomFilters(Collections.singletonList(partitionFileName)); Map<Pair<String, String>, BloomFilter> bloomFilters = getBloomFilters(Collections.singletonList(partitionFileName));
if (bloomFilters.isEmpty()) { if (bloomFilters.isEmpty()) {
LOG.error("Meta index: missing bloom filter for partition: " + partitionName + ", file: " + fileName); LOG.error("Meta index: missing bloom filter for partition: " + partitionName + ", file: " + fileName);
return Option.empty(); return Option.empty();
@@ -175,7 +178,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
} }
@Override @Override
public Map<Pair<String, String>, ByteBuffer> getBloomFilters(final List<Pair<String, String>> partitionNameFileNameList) public Map<Pair<String, String>, BloomFilter> getBloomFilters(final List<Pair<String, String>> partitionNameFileNameList)
throws HoodieMetadataException { throws HoodieMetadataException {
if (!isBloomFilterIndexEnabled) { if (!isBloomFilterIndexEnabled) {
LOG.error("Metadata bloom filter index is disabled!"); LOG.error("Metadata bloom filter index is disabled!");
@@ -202,7 +205,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_BLOOM_FILTERS_METADATA_STR, metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_BLOOM_FILTERS_METADATA_STR,
(timer.endTimer() / partitionIDFileIDStrings.size()))); (timer.endTimer() / partitionIDFileIDStrings.size())));
Map<Pair<String, String>, ByteBuffer> partitionFileToBloomFilterMap = new HashMap<>(); Map<Pair<String, String>, BloomFilter> partitionFileToBloomFilterMap = new HashMap<>();
for (final Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>> entry : hoodieRecordList) { for (final Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>> entry : hoodieRecordList) {
if (entry.getRight().isPresent()) { if (entry.getRight().isPresent()) {
final Option<HoodieMetadataBloomFilter> bloomFilterMetadata = final Option<HoodieMetadataBloomFilter> bloomFilterMetadata =
@@ -210,7 +213,11 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
if (bloomFilterMetadata.isPresent()) { if (bloomFilterMetadata.isPresent()) {
if (!bloomFilterMetadata.get().getIsDeleted()) { if (!bloomFilterMetadata.get().getIsDeleted()) {
ValidationUtils.checkState(fileToKeyMap.containsKey(entry.getLeft())); ValidationUtils.checkState(fileToKeyMap.containsKey(entry.getLeft()));
partitionFileToBloomFilterMap.put(fileToKeyMap.get(entry.getLeft()), bloomFilterMetadata.get().getBloomFilter()); final ByteBuffer bloomFilterByteBuffer = bloomFilterMetadata.get().getBloomFilter();
final String bloomFilterType = bloomFilterMetadata.get().getType();
final BloomFilter bloomFilter = BloomFilterFactory.fromString(
StandardCharsets.UTF_8.decode(bloomFilterByteBuffer).toString(), bloomFilterType);
partitionFileToBloomFilterMap.put(fileToKeyMap.get(entry.getLeft()), bloomFilter);
} }
} else { } else {
LOG.error("Meta index bloom filter missing for: " + fileToKeyMap.get(entry.getLeft())); LOG.error("Meta index bloom filter missing for: " + fileToKeyMap.get(entry.getLeft()));
@@ -269,7 +276,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
/** /**
* Returns a list of all partitions. * Returns a list of all partitions.
*/ */
protected List<String> fetchAllPartitionPaths() throws IOException { protected List<String> fetchAllPartitionPaths() {
HoodieTimer timer = new HoodieTimer().startTimer(); HoodieTimer timer = new HoodieTimer().startTimer();
Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord = getRecordByKey(RECORDKEY_PARTITION_LIST, Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord = getRecordByKey(RECORDKEY_PARTITION_LIST,
MetadataPartitionType.FILES.getPartitionPath()); MetadataPartitionType.FILES.getPartitionPath());

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.metadata; package org.apache.hudi.metadata;
import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
import org.apache.hudi.common.bloom.BloomFilter;
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;
@@ -33,7 +34,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hudi.exception.HoodieMetadataException; 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;
@@ -143,13 +143,13 @@ public class FileSystemBackedTableMetadata implements HoodieTableMetadata {
// no-op // no-op
} }
public Option<ByteBuffer> getBloomFilter(final String partitionName, final String fileName) public Option<BloomFilter> getBloomFilter(final String partitionName, final String fileName)
throws HoodieMetadataException { throws HoodieMetadataException {
throw new HoodieMetadataException("Unsupported operation: getBloomFilter for " + fileName); throw new HoodieMetadataException("Unsupported operation: getBloomFilter for " + fileName);
} }
@Override @Override
public Map<Pair<String, String>, ByteBuffer> getBloomFilters(final List<Pair<String, String>> partitionNameFileNameList) public Map<Pair<String, String>, BloomFilter> getBloomFilters(final List<Pair<String, String>> partitionNameFileNameList)
throws HoodieMetadataException { throws HoodieMetadataException {
throw new HoodieMetadataException("Unsupported operation: getBloomFilters!"); throw new HoodieMetadataException("Unsupported operation: getBloomFilters!");
} }

View File

@@ -18,18 +18,10 @@
package org.apache.hudi.metadata; package org.apache.hudi.metadata;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; import org.apache.hudi.avro.model.HoodieMetadataBloomFilter;
import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
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.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
@@ -37,13 +29,20 @@ 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.hash.ColumnIndexID; import org.apache.hudi.common.util.hash.ColumnIndexID;
import org.apache.hudi.common.util.hash.FileIndexID; import org.apache.hudi.common.util.hash.FileIndexID;
import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.common.util.hash.PartitionIndexID;
import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.io.storage.HoodieHFileReader; import org.apache.hudi.io.storage.HoodieHFileReader;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
@@ -114,7 +113,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
private static final String COLUMN_STATS_FIELD_NULL_COUNT = "nullCount"; 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_VALUE_COUNT = "valueCount";
private static final String COLUMN_STATS_FIELD_TOTAL_SIZE = "totalSize"; 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_FILE_NAME = "fileName";
private static final String COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE = "totalUncompressedSize"; private static final String COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE = "totalUncompressedSize";
private static final String COLUMN_STATS_FIELD_IS_DELETED = FIELD_IS_DELETED; private static final String COLUMN_STATS_FIELD_IS_DELETED = FIELD_IS_DELETED;
@@ -177,7 +176,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
String.format("Valid %s record expected for type: %s", SCHEMA_FIELD_ID_COLUMN_STATS, METADATA_TYPE_COLUMN_STATS)); String.format("Valid %s record expected for type: %s", SCHEMA_FIELD_ID_COLUMN_STATS, METADATA_TYPE_COLUMN_STATS));
} else { } else {
columnStatMetadata = HoodieMetadataColumnStats.newBuilder() columnStatMetadata = HoodieMetadataColumnStats.newBuilder()
.setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_RESOURCE_NAME)) .setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_FILE_NAME))
.setMinValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE)) .setMinValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE))
.setMaxValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE)) .setMaxValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE))
.setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT)) .setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT))
@@ -275,27 +274,25 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
public static HoodieRecord<HoodieMetadataPayload> createBloomFilterMetadataRecord(final String partitionName, public static HoodieRecord<HoodieMetadataPayload> createBloomFilterMetadataRecord(final String partitionName,
final String baseFileName, final String baseFileName,
final String timestamp, final String timestamp,
final String bloomFilterType,
final ByteBuffer bloomFilter, final ByteBuffer bloomFilter,
final boolean isDeleted) { final boolean isDeleted) {
ValidationUtils.checkArgument(!baseFileName.contains(Path.SEPARATOR) checkArgument(!baseFileName.contains(Path.SEPARATOR)
&& FSUtils.isBaseFile(new Path(baseFileName)), && FSUtils.isBaseFile(new Path(baseFileName)),
"Invalid base file '" + baseFileName + "' for MetaIndexBloomFilter!"); "Invalid base file '" + baseFileName + "' for MetaIndexBloomFilter!");
final String bloomFilterIndexKey = new PartitionIndexID(partitionName).asBase64EncodedString() final String bloomFilterIndexKey = new PartitionIndexID(partitionName).asBase64EncodedString()
.concat(new FileIndexID(baseFileName).asBase64EncodedString()); .concat(new FileIndexID(baseFileName).asBase64EncodedString());
HoodieKey key = new HoodieKey(bloomFilterIndexKey, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath()); HoodieKey key = new HoodieKey(bloomFilterIndexKey, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath());
// TODO: HUDI-3203 Get the bloom filter type from the file
HoodieMetadataBloomFilter metadataBloomFilter = HoodieMetadataBloomFilter metadataBloomFilter =
new HoodieMetadataBloomFilter(BloomFilterTypeCode.DYNAMIC_V0.name(), new HoodieMetadataBloomFilter(bloomFilterType, timestamp, bloomFilter, isDeleted);
timestamp, bloomFilter, isDeleted); HoodieMetadataPayload metadataPayload = new HoodieMetadataPayload(key.getRecordKey(), metadataBloomFilter);
HoodieMetadataPayload metadataPayload = new HoodieMetadataPayload(key.getRecordKey(),
metadataBloomFilter);
return new HoodieAvroRecord<>(key, metadataPayload); return new HoodieAvroRecord<>(key, metadataPayload);
} }
@Override @Override
public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) {
ValidationUtils.checkArgument(previousRecord.type == type, checkArgument(previousRecord.type == type,
"Cannot combine " + previousRecord.type + " with " + type); "Cannot combine " + previousRecord.type + " with " + type);
switch (type) { switch (type) {
@@ -314,11 +311,16 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
} }
private HoodieMetadataBloomFilter combineBloomFilterMetadata(HoodieMetadataPayload previousRecord) { private HoodieMetadataBloomFilter combineBloomFilterMetadata(HoodieMetadataPayload previousRecord) {
// Bloom filters are always additive. No need to merge with previous bloom filter
return this.bloomFilterMetadata; return this.bloomFilterMetadata;
} }
private HoodieMetadataColumnStats combineColumnStatsMetadata(HoodieMetadataPayload previousRecord) { private HoodieMetadataColumnStats combineColumnStatsMetadata(HoodieMetadataPayload previousRecord) {
return this.columnStatMetadata; checkArgument(previousRecord.getColumnStatMetadata().isPresent());
checkArgument(getColumnStatMetadata().isPresent());
checkArgument(previousRecord.getColumnStatMetadata().get()
.getFileName().equals(this.columnStatMetadata.getFileName()));
return HoodieTableMetadataUtil.mergeColumnStats(previousRecord.getColumnStatMetadata().get(), this.columnStatMetadata);
} }
@Override @Override
@@ -353,7 +355,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
* Returns the list of filenames added as part of this record. * Returns the list of filenames added as part of this record.
*/ */
public List<String> getFilenames() { public List<String> getFilenames() {
return filterFileInfoEntries(false).map(e -> e.getKey()).sorted().collect(Collectors.toList()); return filterFileInfoEntries(false).map(Map.Entry::getKey).sorted().collect(Collectors.toList());
} }
/** /**
@@ -518,8 +520,6 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
.build()); .build());
return new HoodieAvroRecord<>(key, payload); return new HoodieAvroRecord<>(key, payload);
}); });
} }
@Override @Override
@@ -532,9 +532,9 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
if (type == METADATA_TYPE_BLOOM_FILTER) { if (type == METADATA_TYPE_BLOOM_FILTER) {
checkState(getBloomFilterMetadata().isPresent()); checkState(getBloomFilterMetadata().isPresent());
sb.append("BloomFilter: {"); sb.append("BloomFilter: {");
sb.append("bloom size: " + getBloomFilterMetadata().get().getBloomFilter().array().length).append(", "); sb.append("bloom size: ").append(getBloomFilterMetadata().get().getBloomFilter().array().length).append(", ");
sb.append("timestamp: " + getBloomFilterMetadata().get().getTimestamp()).append(", "); sb.append("timestamp: ").append(getBloomFilterMetadata().get().getTimestamp()).append(", ");
sb.append("deleted: " + getBloomFilterMetadata().get().getIsDeleted()); sb.append("deleted: ").append(getBloomFilterMetadata().get().getIsDeleted());
sb.append("}"); sb.append("}");
} }
if (type == METADATA_TYPE_COLUMN_STATS) { if (type == METADATA_TYPE_COLUMN_STATS) {

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.metadata;
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.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
import org.apache.hudi.common.bloom.BloomFilter;
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;
@@ -31,7 +32,6 @@ 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;
@@ -122,20 +122,20 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
* *
* @param partitionName - Partition name * @param partitionName - Partition name
* @param fileName - File name for which bloom filter needs to be retrieved * @param fileName - File name for which bloom filter needs to be retrieved
* @return BloomFilter byte buffer if available, otherwise empty * @return BloomFilter if available, otherwise empty
* @throws HoodieMetadataException * @throws HoodieMetadataException
*/ */
Option<ByteBuffer> getBloomFilter(final String partitionName, final String fileName) Option<BloomFilter> getBloomFilter(final String partitionName, final String fileName)
throws HoodieMetadataException; throws HoodieMetadataException;
/** /**
* Get bloom filters for files from the metadata table index. * 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 * @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 * @return Map of partition file name pair to its bloom filter
* @throws HoodieMetadataException * @throws HoodieMetadataException
*/ */
Map<Pair<String, String>, ByteBuffer> getBloomFilters(final List<Pair<String, String>> partitionNameFileNameList) Map<Pair<String, String>, BloomFilter> getBloomFilters(final List<Pair<String, String>> partitionNameFileNameList)
throws HoodieMetadataException; throws HoodieMetadataException;
/** /**

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.metadata; package org.apache.hudi.metadata;
import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
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.bloom.BloomFilter;
@@ -43,32 +44,48 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.CollectionUtils;
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.ParquetUtils;
import org.apache.hudi.common.util.StringUtils;
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.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieMetadataException;
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.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
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.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import javax.annotation.Nonnull;
import java.io.IOException; 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.Collections; import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
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.Objects;
import java.util.function.BiFunction; import java.util.function.BiFunction;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldValAsString;
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.COLUMN_RANGE_MERGE_FUNCTION;
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.MAX;
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.MIN;
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.NULL_COUNT;
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.TOTAL_SIZE;
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.TOTAL_UNCOMPRESSED_SIZE;
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.VALUE_COUNT;
import static org.apache.hudi.metadata.HoodieTableMetadata.EMPTY_PARTITION_NAME; import static org.apache.hudi.metadata.HoodieTableMetadata.EMPTY_PARTITION_NAME;
import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME; import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME;
@@ -104,37 +121,27 @@ public class HoodieTableMetadataUtil {
/** /**
* Convert commit action to metadata records for the enabled partition types. * Convert commit action to metadata records for the enabled partition types.
* *
* @param commitMetadata - Commit action metadata * @param commitMetadata - Commit action metadata
* @param dataMetaClient - Meta client for the data table * @param instantTime - Action instant time
* @param isMetaIndexColumnStatsForAllColumns - Do all columns need meta indexing? * @param recordsGenerationParams - Parameters for the record generation
* @param instantTime - Action instant time
* @return Map of partition to metadata records for the commit action * @return Map of partition to metadata records for the commit action
*/ */
public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords( public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
HoodieEngineContext context, List<MetadataPartitionType> enabledPartitionTypes, HoodieEngineContext context, HoodieCommitMetadata commitMetadata, String instantTime,
HoodieCommitMetadata commitMetadata, HoodieTableMetaClient dataMetaClient, MetadataRecordsGenerationParams recordsGenerationParams) {
boolean isMetaIndexColumnStatsForAllColumns, String instantTime) {
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>(); final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
final HoodieData<HoodieRecord> filesPartitionRecordsRDD = context.parallelize( final HoodieData<HoodieRecord> filesPartitionRecordsRDD = context.parallelize(
convertMetadataToFilesPartitionRecords(commitMetadata, instantTime), 1); convertMetadataToFilesPartitionRecords(commitMetadata, instantTime), 1);
partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD);
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) {
final List<HoodieRecord> metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(commitMetadata, final HoodieData<HoodieRecord> metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(context, commitMetadata, instantTime, recordsGenerationParams);
dataMetaClient, instantTime); partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecords);
if (!metadataBloomFilterRecords.isEmpty()) {
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = context.parallelize(metadataBloomFilterRecords, 1);
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
}
} }
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) {
final List<HoodieRecord> metadataColumnStats = convertMetadataToColumnStatsRecords(commitMetadata, context, final HoodieData<HoodieRecord> metadataColumnStatsRDD = convertMetadataToColumnStatsRecords(commitMetadata, context, recordsGenerationParams);
dataMetaClient, isMetaIndexColumnStatsForAllColumns, instantTime); partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
if (!metadataColumnStats.isEmpty()) {
final HoodieData<HoodieRecord> metadataColumnStatsRDD = context.parallelize(metadataColumnStats, 1);
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
}
} }
return partitionToRecordsMap; return partitionToRecordsMap;
} }
@@ -163,7 +170,7 @@ public class HoodieTableMetadataUtil {
String partitionStatName = entry.getKey(); String partitionStatName = entry.getKey();
List<HoodieWriteStat> writeStats = entry.getValue(); List<HoodieWriteStat> writeStats = entry.getValue();
String partition = partitionStatName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionStatName; String partition = getPartition(partitionStatName);
HashMap<String, Long> updatedFilesToSizesMapping = HashMap<String, Long> updatedFilesToSizesMapping =
writeStats.stream().reduce(new HashMap<>(writeStats.size()), writeStats.stream().reduce(new HashMap<>(writeStats.size()),
@@ -206,94 +213,90 @@ public class HoodieTableMetadataUtil {
/** /**
* Convert commit action metadata to bloom filter records. * Convert commit action metadata to bloom filter records.
* *
* @param commitMetadata - Commit action metadata * @param context - Engine context to use
* @param dataMetaClient - Meta client for the data table * @param commitMetadata - Commit action metadata
* @param instantTime - Action instant time * @param instantTime - Action instant time
* @return List of metadata table records * @param recordsGenerationParams - Parameters for bloom filter record generation
* @return HoodieData of metadata table records
*/ */
public static List<HoodieRecord> convertMetadataToBloomFilterRecords(HoodieCommitMetadata commitMetadata, public static HoodieData<HoodieRecord> convertMetadataToBloomFilterRecords(
HoodieTableMetaClient dataMetaClient, HoodieEngineContext context, HoodieCommitMetadata commitMetadata,
String instantTime) { String instantTime, MetadataRecordsGenerationParams recordsGenerationParams) {
List<HoodieRecord> records = new LinkedList<>(); final List<HoodieWriteStat> allWriteStats = commitMetadata.getPartitionToWriteStats().values().stream()
commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { .flatMap(entry -> entry.stream()).collect(Collectors.toList());
final String partition = partitionStatName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionStatName; if (allWriteStats.isEmpty()) {
Map<String, Long> newFiles = new HashMap<>(writeStats.size()); return context.emptyHoodieData();
writeStats.forEach(hoodieWriteStat -> { }
// No action for delta logs
if (hoodieWriteStat instanceof HoodieDeltaWriteStat) {
return;
}
String pathWithPartition = hoodieWriteStat.getPath(); final int parallelism = Math.max(Math.min(allWriteStats.size(), recordsGenerationParams.getBloomIndexParallelism()), 1);
if (pathWithPartition == null) { HoodieData<HoodieWriteStat> allWriteStatsRDD = context.parallelize(allWriteStats, parallelism);
// Empty partition return allWriteStatsRDD.flatMap(hoodieWriteStat -> {
LOG.error("Failed to find path in write stat to update metadata table " + hoodieWriteStat); final String partition = hoodieWriteStat.getPartitionPath();
return;
}
int offset = partition.equals(NON_PARTITIONED_NAME) ? (pathWithPartition.startsWith("/") ? 1 : 0) :
partition.length() + 1;
final String fileName = pathWithPartition.substring(offset); // For bloom filter index, delta writes do not change the base file bloom filter entries
if (!FSUtils.isBaseFile(new Path(fileName))) { if (hoodieWriteStat instanceof HoodieDeltaWriteStat) {
return; return Collections.emptyListIterator();
} }
ValidationUtils.checkState(!newFiles.containsKey(fileName), "Duplicate files in HoodieCommitMetadata");
final Path writeFilePath = new Path(dataMetaClient.getBasePath(), pathWithPartition); String pathWithPartition = hoodieWriteStat.getPath();
if (pathWithPartition == null) {
// Empty partition
LOG.error("Failed to find path in write stat to update metadata table " + hoodieWriteStat);
return Collections.emptyListIterator();
}
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 Collections.emptyListIterator();
}
final Path writeFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition);
try (HoodieFileReader<IndexedRecord> fileReader =
HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), writeFilePath)) {
try { try {
HoodieFileReader<IndexedRecord> fileReader = final BloomFilter fileBloomFilter = fileReader.readBloomFilter();
HoodieFileReaderFactory.getFileReader(dataMetaClient.getHadoopConf(), writeFilePath); if (fileBloomFilter == null) {
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); LOG.error("Failed to read bloom filter for " + writeFilePath);
return; return Collections.emptyListIterator();
} }
ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes());
HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord(
partition, fileName, instantTime, recordsGenerationParams.getBloomFilterType(), bloomByteBuffer, false);
return Collections.singletonList(record).iterator();
} catch (Exception e) {
LOG.error("Failed to read bloom filter for " + writeFilePath);
return Collections.emptyListIterator();
} finally {
fileReader.close(); fileReader.close();
} catch (IOException e) {
LOG.error("Failed to get bloom filter for file: " + writeFilePath + ", write stat: " + hoodieWriteStat);
} }
}); } catch (IOException e) {
LOG.error("Failed to get bloom filter for file: " + writeFilePath + ", write stat: " + hoodieWriteStat);
}
return Collections.emptyListIterator();
}); });
return records;
} }
/** /**
* Convert the clean action to metadata records. * Convert the clean action to metadata records.
*/ */
public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords( public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
HoodieEngineContext engineContext, List<MetadataPartitionType> enabledPartitionTypes, HoodieEngineContext engineContext, HoodieCleanMetadata cleanMetadata,
HoodieCleanMetadata cleanMetadata, HoodieTableMetaClient dataMetaClient, String instantTime) { MetadataRecordsGenerationParams recordsGenerationParams, String instantTime) {
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>(); final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
final HoodieData<HoodieRecord> filesPartitionRecordsRDD = engineContext.parallelize( final HoodieData<HoodieRecord> filesPartitionRecordsRDD = engineContext.parallelize(
convertMetadataToFilesPartitionRecords(cleanMetadata, instantTime), 1); convertMetadataToFilesPartitionRecords(cleanMetadata, instantTime), 1);
partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD);
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) {
final List<HoodieRecord> metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(cleanMetadata, final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = convertMetadataToBloomFilterRecords(cleanMetadata, engineContext, instantTime, recordsGenerationParams);
engineContext, instantTime); partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
if (!metadataBloomFilterRecords.isEmpty()) {
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1);
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
}
} }
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) {
final List<HoodieRecord> metadataColumnStats = convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, final HoodieData<HoodieRecord> metadataColumnStatsRDD = convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, recordsGenerationParams);
dataMetaClient); partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
if (!metadataColumnStats.isEmpty()) {
final HoodieData<HoodieRecord> metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1);
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
}
} }
return partitionToRecordsMap; return partitionToRecordsMap;
@@ -311,7 +314,7 @@ public class HoodieTableMetadataUtil {
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) -> {
final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; final String partition = getPartition(partitionName);
// Files deleted from a partition // Files deleted from a partition
List<String> deletedFiles = partitionMetadata.getDeletePathPatterns(); List<String> deletedFiles = partitionMetadata.getDeletePathPatterns();
HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(), HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(),
@@ -329,14 +332,16 @@ public class HoodieTableMetadataUtil {
/** /**
* Convert clean metadata to bloom filter index records. * Convert clean metadata to bloom filter index records.
* *
* @param cleanMetadata - Clean action metadata * @param cleanMetadata - Clean action metadata
* @param engineContext - Engine context * @param engineContext - Engine context
* @param instantTime - Clean action instant time * @param instantTime - Clean action instant time
* @param recordsGenerationParams - Parameters for bloom filter record generation
* @return List of bloom filter index records for the clean metadata * @return List of bloom filter index records for the clean metadata
*/ */
public static List<HoodieRecord> convertMetadataToBloomFilterRecords(HoodieCleanMetadata cleanMetadata, public static HoodieData<HoodieRecord> convertMetadataToBloomFilterRecords(HoodieCleanMetadata cleanMetadata,
HoodieEngineContext engineContext, HoodieEngineContext engineContext,
String instantTime) { String instantTime,
MetadataRecordsGenerationParams recordsGenerationParams) {
List<Pair<String, String>> deleteFileList = new ArrayList<>(); List<Pair<String, String>> deleteFileList = new ArrayList<>();
cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> {
// Files deleted from a partition // Files deleted from a partition
@@ -349,23 +354,24 @@ public class HoodieTableMetadataUtil {
}); });
}); });
return engineContext.map(deleteFileList, deleteFileInfo -> { final int parallelism = Math.max(Math.min(deleteFileList.size(), recordsGenerationParams.getBloomIndexParallelism()), 1);
return HoodieMetadataPayload.createBloomFilterMetadataRecord( HoodieData<Pair<String, String>> deleteFileListRDD = engineContext.parallelize(deleteFileList, parallelism);
deleteFileInfo.getLeft(), deleteFileInfo.getRight(), instantTime, ByteBuffer.allocate(0), true); return deleteFileListRDD.map(deleteFileInfoPair -> HoodieMetadataPayload.createBloomFilterMetadataRecord(
}, 1).stream().collect(Collectors.toList()); deleteFileInfoPair.getLeft(), deleteFileInfoPair.getRight(), instantTime, StringUtils.EMPTY_STRING,
ByteBuffer.allocate(0), true));
} }
/** /**
* Convert clean metadata to column stats index records. * Convert clean metadata to column stats index records.
* *
* @param cleanMetadata - Clean action metadata * @param cleanMetadata - Clean action metadata
* @param engineContext - Engine context * @param engineContext - Engine context
* @param datasetMetaClient - data table meta client * @param recordsGenerationParams - Parameters for bloom filter record generation
* @return List of column stats index records for the clean metadata * @return List of column stats index records for the clean metadata
*/ */
public static List<HoodieRecord> convertMetadataToColumnStatsRecords(HoodieCleanMetadata cleanMetadata, public static HoodieData<HoodieRecord> convertMetadataToColumnStatsRecords(HoodieCleanMetadata cleanMetadata,
HoodieEngineContext engineContext, HoodieEngineContext engineContext,
HoodieTableMetaClient datasetMetaClient) { MetadataRecordsGenerationParams recordsGenerationParams) {
List<Pair<String, String>> deleteFileList = new ArrayList<>(); List<Pair<String, String>> deleteFileList = new ArrayList<>();
cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> {
// Files deleted from a partition // Files deleted from a partition
@@ -373,54 +379,42 @@ public class HoodieTableMetadataUtil {
deletedFiles.forEach(entry -> deleteFileList.add(Pair.of(partition, entry))); deletedFiles.forEach(entry -> deleteFileList.add(Pair.of(partition, entry)));
}); });
List<String> latestColumns = getLatestColumns(datasetMetaClient); final List<String> columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled());
return engineContext.flatMap(deleteFileList, final int parallelism = Math.max(Math.min(deleteFileList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1);
deleteFileInfo -> { HoodieData<Pair<String, String>> deleteFileListRDD = engineContext.parallelize(deleteFileList, parallelism);
if (deleteFileInfo.getRight().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { return deleteFileListRDD.flatMap(deleteFileInfoPair -> {
return getColumnStats(deleteFileInfo.getKey(), deleteFileInfo.getValue(), datasetMetaClient, if (deleteFileInfoPair.getRight().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
latestColumns, true); return getColumnStats(deleteFileInfoPair.getLeft(), deleteFileInfoPair.getRight(), recordsGenerationParams.getDataMetaClient(), columnsToIndex, true).iterator();
} }
return Stream.empty(); return Collections.emptyListIterator();
}, 1).stream().collect(Collectors.toList()); });
} }
/** /**
* Convert restore action metadata to metadata table records. * Convert restore action metadata to metadata table records.
*/ */
public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords( public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
HoodieEngineContext engineContext, List<MetadataPartitionType> enabledPartitionTypes, HoodieEngineContext engineContext, HoodieActiveTimeline metadataTableTimeline, HoodieRestoreMetadata restoreMetadata,
HoodieActiveTimeline metadataTableTimeline, HoodieRestoreMetadata restoreMetadata, MetadataRecordsGenerationParams recordsGenerationParams, String instantTime, Option<String> lastSyncTs) {
HoodieTableMetaClient dataMetaClient, String instantTime, Option<String> lastSyncTs) {
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>(); final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
final Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>(); final Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
final Map<String, List<String>> partitionToDeletedFiles = new HashMap<>(); final Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
processRestoreMetadata(metadataTableTimeline, restoreMetadata, processRestoreMetadata(metadataTableTimeline, restoreMetadata, partitionToAppendedFiles, partitionToDeletedFiles, lastSyncTs);
partitionToAppendedFiles, partitionToDeletedFiles, lastSyncTs); final HoodieData<HoodieRecord> filesPartitionRecordsRDD =
engineContext.parallelize(convertFilesToFilesPartitionRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore"), 1);
final HoodieData<HoodieRecord> filesPartitionRecordsRDD = engineContext.parallelize(
convertFilesToFilesPartitionRecords(partitionToDeletedFiles,
partitionToAppendedFiles, instantTime, "Restore"), 1);
partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD);
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) {
final List<HoodieRecord> metadataBloomFilterRecords = convertFilesToBloomFilterRecords( final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD =
engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); convertFilesToBloomFilterRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams, instantTime);
if (!metadataBloomFilterRecords.isEmpty()) { partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1);
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
}
} }
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) {
final List<HoodieRecord> metadataColumnStats = convertFilesToColumnStatsRecords( final HoodieData<HoodieRecord> metadataColumnStatsRDD = convertFilesToColumnStatsRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams);
engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
if (!metadataColumnStats.isEmpty()) {
final HoodieData<HoodieRecord> metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1);
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
}
} }
return partitionToRecordsMap; return partitionToRecordsMap;
} }
@@ -436,44 +430,35 @@ public class HoodieTableMetadataUtil {
Map<String, Map<String, Long>> partitionToAppendedFiles, Map<String, Map<String, Long>> partitionToAppendedFiles,
Map<String, List<String>> partitionToDeletedFiles, Map<String, List<String>> partitionToDeletedFiles,
Option<String> lastSyncTs) { Option<String> lastSyncTs) {
restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> { restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> rms.forEach(rm -> processRollbackMetadata(metadataTableTimeline, rm,
rms.forEach(rm -> processRollbackMetadata(metadataTableTimeline, rm, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs)));
partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs));
});
} }
/** /**
* Convert rollback action metadata to metadata table records. * Convert rollback action metadata to metadata table records.
*/ */
public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords( public static Map<MetadataPartitionType, HoodieData<HoodieRecord>> convertMetadataToRecords(
HoodieEngineContext engineContext, List<MetadataPartitionType> enabledPartitionTypes, HoodieEngineContext engineContext, HoodieActiveTimeline metadataTableTimeline,
HoodieActiveTimeline metadataTableTimeline, HoodieRollbackMetadata rollbackMetadata, HoodieRollbackMetadata rollbackMetadata, MetadataRecordsGenerationParams recordsGenerationParams,
HoodieTableMetaClient dataMetaClient, String instantTime, Option<String> lastSyncTs, boolean wasSynced) { String instantTime, Option<String> lastSyncTs, boolean wasSynced) {
final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>(); final Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
Map<String, List<String>> partitionToDeletedFiles = new HashMap<>(); Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>(); Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
List<HoodieRecord> filesPartitionRecords = convertMetadataToRollbackRecords(metadataTableTimeline, rollbackMetadata,
partitionToDeletedFiles, partitionToAppendedFiles, instantTime, lastSyncTs, wasSynced); List<HoodieRecord> filesPartitionRecords =
convertMetadataToRollbackRecords(metadataTableTimeline, rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, instantTime, lastSyncTs, wasSynced);
final HoodieData<HoodieRecord> rollbackRecordsRDD = engineContext.parallelize(filesPartitionRecords, 1); final HoodieData<HoodieRecord> rollbackRecordsRDD = engineContext.parallelize(filesPartitionRecords, 1);
partitionToRecordsMap.put(MetadataPartitionType.FILES, rollbackRecordsRDD); partitionToRecordsMap.put(MetadataPartitionType.FILES, rollbackRecordsRDD);
if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) {
final List<HoodieRecord> metadataBloomFilterRecords = convertFilesToBloomFilterRecords( final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD =
engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); convertFilesToBloomFilterRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams, instantTime);
if (!metadataBloomFilterRecords.isEmpty()) { partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
final HoodieData<HoodieRecord> metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1);
partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD);
}
} }
if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) {
final List<HoodieRecord> metadataColumnStats = convertFilesToColumnStatsRecords( final HoodieData<HoodieRecord> metadataColumnStatsRDD = convertFilesToColumnStatsRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams);
engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
if (!metadataColumnStats.isEmpty()) {
final HoodieData<HoodieRecord> metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1);
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
}
} }
return partitionToRecordsMap; return partitionToRecordsMap;
@@ -594,7 +579,7 @@ public class HoodieTableMetadataUtil {
partitionToDeletedFiles.forEach((partitionName, deletedFiles) -> { partitionToDeletedFiles.forEach((partitionName, deletedFiles) -> {
fileChangeCount[0] += deletedFiles.size(); fileChangeCount[0] += deletedFiles.size();
final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; final String partition = getPartition(partitionName);
Option<Map<String, Long>> filesAdded = Option.empty(); Option<Map<String, Long>> filesAdded = Option.empty();
if (partitionToAppendedFiles.containsKey(partitionName)) { if (partitionToAppendedFiles.containsKey(partitionName)) {
@@ -607,7 +592,7 @@ public class HoodieTableMetadataUtil {
}); });
partitionToAppendedFiles.forEach((partitionName, appendedFileMap) -> { partitionToAppendedFiles.forEach((partitionName, appendedFileMap) -> {
final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; final String partition = getPartition(partitionName);
fileChangeCount[1] += appendedFileMap.size(); fileChangeCount[1] += appendedFileMap.size();
// Validate that no appended file has been deleted // Validate that no appended file has been deleted
@@ -628,82 +613,133 @@ public class HoodieTableMetadataUtil {
} }
/** /**
* Convert rollback action metadata to bloom filter index records. * Returns partition name for the given path.
*
* @param path
* @return
*/ */
private static List<HoodieRecord> convertFilesToBloomFilterRecords(HoodieEngineContext engineContext, static String getPartition(@Nonnull String path) {
HoodieTableMetaClient dataMetaClient, return EMPTY_PARTITION_NAME.equals(path) ? NON_PARTITIONED_NAME : path;
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. * Convert added and deleted files metadata to bloom filter index records.
*/ */
private static List<HoodieRecord> convertFilesToColumnStatsRecords(HoodieEngineContext engineContext, public static HoodieData<HoodieRecord> convertFilesToBloomFilterRecords(HoodieEngineContext engineContext,
HoodieTableMetaClient datasetMetaClient, Map<String, List<String>> partitionToDeletedFiles,
Map<String, List<String>> partitionToDeletedFiles, Map<String, Map<String, Long>> partitionToAppendedFiles,
Map<String, Map<String, Long>> partitionToAppendedFiles, MetadataRecordsGenerationParams recordsGenerationParams,
String instantTime) { String instantTime) {
List<HoodieRecord> records = new LinkedList<>(); HoodieData<HoodieRecord> allRecordsRDD = engineContext.emptyHoodieData();
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( List<Pair<String, List<String>>> partitionToDeletedFilesList = partitionToDeletedFiles.entrySet()
(appendedFile, size) -> { .stream().map(e -> Pair.of(e.getKey(), e.getValue())).collect(Collectors.toList());
final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; int parallelism = Math.max(Math.min(partitionToDeletedFilesList.size(), recordsGenerationParams.getBloomIndexParallelism()), 1);
if (appendedFile.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { HoodieData<Pair<String, List<String>>> partitionToDeletedFilesRDD = engineContext.parallelize(partitionToDeletedFilesList, parallelism);
final String filePathWithPartition = partitionName + "/" + appendedFile;
records.addAll(getColumnStats(partition, filePathWithPartition, datasetMetaClient, HoodieData<HoodieRecord> deletedFilesRecordsRDD = partitionToDeletedFilesRDD.flatMap(partitionToDeletedFilesPair -> {
latestColumns, false).collect(Collectors.toList())); final String partitionName = partitionToDeletedFilesPair.getLeft();
final List<String> deletedFileList = partitionToDeletedFilesPair.getRight();
return deletedFileList.stream().flatMap(deletedFile -> {
if (!FSUtils.isBaseFile(new Path(deletedFile))) {
return Stream.empty();
}
final String partition = getPartition(partitionName);
return Stream.<HoodieRecord>of(HoodieMetadataPayload.createBloomFilterMetadataRecord(
partition, deletedFile, instantTime, StringUtils.EMPTY_STRING, ByteBuffer.allocate(0), true));
}).iterator();
});
allRecordsRDD = allRecordsRDD.union(deletedFilesRecordsRDD);
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.getBloomIndexParallelism()), 1);
HoodieData<Pair<String, Map<String, Long>>> partitionToAppendedFilesRDD = engineContext.parallelize(partitionToAppendedFilesList, parallelism);
HoodieData<HoodieRecord> appendedFilesRecordsRDD = partitionToAppendedFilesRDD.flatMap(partitionToAppendedFilesPair -> {
final String partitionName = partitionToAppendedFilesPair.getLeft();
final Map<String, Long> appendedFileMap = partitionToAppendedFilesPair.getRight();
final String partition = getPartition(partitionName);
return appendedFileMap.entrySet().stream().flatMap(appendedFileLengthPairEntry -> {
final String appendedFile = appendedFileLengthPairEntry.getKey();
if (!FSUtils.isBaseFile(new Path(appendedFile))) {
return Stream.empty();
}
final String pathWithPartition = partitionName + "/" + appendedFile;
final Path appendedFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition);
try (HoodieFileReader<IndexedRecord> fileReader =
HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), appendedFilePath)) {
final BloomFilter fileBloomFilter = fileReader.readBloomFilter();
if (fileBloomFilter == null) {
LOG.error("Failed to read bloom filter for " + appendedFilePath);
return Stream.empty();
} }
})); ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes());
return records; HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord(
partition, appendedFile, instantTime, recordsGenerationParams.getBloomFilterType(), bloomByteBuffer, false);
return Stream.of(record);
} catch (IOException e) {
LOG.error("Failed to get bloom filter for file: " + appendedFilePath);
}
return Stream.empty();
}).iterator();
});
allRecordsRDD = allRecordsRDD.union(appendedFilesRecordsRDD);
return allRecordsRDD;
}
/**
* Convert added and deleted action metadata to column stats index records.
*/
public static HoodieData<HoodieRecord> convertFilesToColumnStatsRecords(HoodieEngineContext engineContext,
Map<String, List<String>> partitionToDeletedFiles,
Map<String, Map<String, Long>> partitionToAppendedFiles,
MetadataRecordsGenerationParams recordsGenerationParams) {
HoodieData<HoodieRecord> allRecordsRDD = engineContext.emptyHoodieData();
final List<String> columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled());
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);
HoodieData<HoodieRecord> deletedFilesRecordsRDD = partitionToDeletedFilesRDD.flatMap(partitionToDeletedFilesPair -> {
final String partitionName = partitionToDeletedFilesPair.getLeft();
final String partition = getPartition(partitionName);
final List<String> deletedFileList = partitionToDeletedFilesPair.getRight();
return deletedFileList.stream().flatMap(deletedFile -> {
final String filePathWithPartition = partitionName + "/" + deletedFile;
return getColumnStats(partition, filePathWithPartition, recordsGenerationParams.getDataMetaClient(), 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);
HoodieData<HoodieRecord> appendedFilesRecordsRDD = partitionToAppendedFilesRDD.flatMap(partitionToAppendedFilesPair -> {
final String partitionName = partitionToAppendedFilesPair.getLeft();
final String partition = getPartition(partitionName);
final Map<String, Long> appendedFileMap = partitionToAppendedFilesPair.getRight();
return appendedFileMap.entrySet().stream().flatMap(appendedFileNameLengthEntry -> {
if (!FSUtils.isBaseFile(new Path(appendedFileNameLengthEntry.getKey()))
|| !appendedFileNameLengthEntry.getKey().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
return Stream.empty();
}
final String filePathWithPartition = partitionName + "/" + appendedFileNameLengthEntry.getKey();
return getColumnStats(partition, filePathWithPartition, recordsGenerationParams.getDataMetaClient(), columnsToIndex, false);
}).iterator();
});
allRecordsRDD = allRecordsRDD.union(appendedFilesRecordsRDD);
return allRecordsRDD;
} }
/** /**
@@ -768,7 +804,7 @@ public class HoodieTableMetadataUtil {
if (timeline.empty()) { if (timeline.empty()) {
final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION,
HoodieActiveTimeline.createNewInstantTime()); HoodieActiveTimeline.createNewInstantTime());
timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails); timeline = new HoodieDefaultTimeline(Stream.of(instant), metaClient.getActiveTimeline()::getInstantDetails);
} }
return new HoodieTableFileSystemView(metaClient, timeline); return new HoodieTableFileSystemView(metaClient, timeline);
} }
@@ -796,20 +832,16 @@ public class HoodieTableMetadataUtil {
} 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(Comparator.comparing(FileSlice::getFileId)).collect(Collectors.toList());
} }
public static List<HoodieRecord> convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata, public static HoodieData<HoodieRecord> convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata,
HoodieEngineContext engineContext, HoodieEngineContext engineContext,
HoodieTableMetaClient dataMetaClient, MetadataRecordsGenerationParams recordsGenerationParams) {
boolean isMetaIndexColumnStatsForAllColumns,
String instantTime) {
try { try {
List<HoodieWriteStat> allWriteStats = commitMetadata.getPartitionToWriteStats().values().stream() List<HoodieWriteStat> allWriteStats = commitMetadata.getPartitionToWriteStats().values().stream()
.flatMap(entry -> entry.stream()).collect(Collectors.toList()); .flatMap(entry -> entry.stream()).collect(Collectors.toList());
return HoodieTableMetadataUtil.createColumnStatsFromWriteStats(engineContext, dataMetaClient, allWriteStats, return HoodieTableMetadataUtil.createColumnStatsFromWriteStats(engineContext, allWriteStats, recordsGenerationParams);
isMetaIndexColumnStatsForAllColumns);
} catch (Exception e) { } catch (Exception e) {
throw new HoodieException("Failed to generate column stats records for metadata table ", e); throw new HoodieException("Failed to generate column stats records for metadata table ", e);
} }
@@ -818,30 +850,20 @@ public class HoodieTableMetadataUtil {
/** /**
* Create column stats from write status. * Create column stats from write status.
* *
* @param engineContext - Engine context * @param engineContext - Engine context
* @param datasetMetaClient - Dataset meta client * @param allWriteStats - Write status to convert
* @param allWriteStats - Write status to convert * @param recordsGenerationParams - Parameters for columns stats record generation
* @param isMetaIndexColumnStatsForAllColumns - Are all columns enabled for indexing
*/ */
public static List<HoodieRecord> createColumnStatsFromWriteStats(HoodieEngineContext engineContext, public static HoodieData<HoodieRecord> createColumnStatsFromWriteStats(HoodieEngineContext engineContext,
HoodieTableMetaClient datasetMetaClient, List<HoodieWriteStat> allWriteStats,
List<HoodieWriteStat> allWriteStats, MetadataRecordsGenerationParams recordsGenerationParams) {
boolean isMetaIndexColumnStatsForAllColumns) throws Exception {
if (allWriteStats.isEmpty()) { if (allWriteStats.isEmpty()) {
return Collections.emptyList(); return engineContext.emptyHoodieData();
} }
final List<String> columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled());
List<HoodieWriteStat> prunedWriteStats = allWriteStats.stream().filter(writeStat -> { final int parallelism = Math.max(Math.min(allWriteStats.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1);
return !(writeStat instanceof HoodieDeltaWriteStat); HoodieData<HoodieWriteStat> allWriteStatsRDD = engineContext.parallelize(allWriteStats, parallelism);
}).collect(Collectors.toList()); return allWriteStatsRDD.flatMap(writeStat -> translateWriteStatToColumnStats(writeStat, recordsGenerationParams.getDataMetaClient(), columnsToIndex).iterator());
if (prunedWriteStats.isEmpty()) {
return Collections.emptyList();
}
return engineContext.flatMap(prunedWriteStats,
writeStat -> translateWriteStatToColumnStats(writeStat, datasetMetaClient,
getLatestColumns(datasetMetaClient, isMetaIndexColumnStatsForAllColumns)),
prunedWriteStats.size());
} }
/** /**
@@ -850,10 +872,10 @@ public class HoodieTableMetadataUtil {
* @param datasetMetaClient - Data table meta client * @param datasetMetaClient - Data table meta client
* @param isMetaIndexColumnStatsForAllColumns - Is column stats indexing enabled for all columns * @param isMetaIndexColumnStatsForAllColumns - Is column stats indexing enabled for all columns
*/ */
private static List<String> getLatestColumns(HoodieTableMetaClient datasetMetaClient, boolean isMetaIndexColumnStatsForAllColumns) { private static List<String> getColumnsToIndex(HoodieTableMetaClient datasetMetaClient, boolean isMetaIndexColumnStatsForAllColumns) {
if (!isMetaIndexColumnStatsForAllColumns if (!isMetaIndexColumnStatsForAllColumns
|| datasetMetaClient.getCommitsTimeline().filterCompletedInstants().countInstants() < 1) { || datasetMetaClient.getCommitsTimeline().filterCompletedInstants().countInstants() < 1) {
return Collections.singletonList(datasetMetaClient.getTableConfig().getRecordKeyFieldProp()); return Arrays.asList(datasetMetaClient.getTableConfig().getRecordKeyFieldProp().split(","));
} }
TableSchemaResolver schemaResolver = new TableSchemaResolver(datasetMetaClient); TableSchemaResolver schemaResolver = new TableSchemaResolver(datasetMetaClient);
@@ -867,27 +889,42 @@ public class HoodieTableMetadataUtil {
} }
} }
private static List<String> getLatestColumns(HoodieTableMetaClient datasetMetaClient) { public static HoodieMetadataColumnStats mergeColumnStats(HoodieMetadataColumnStats oldColumnStats, HoodieMetadataColumnStats newColumnStats) {
return getLatestColumns(datasetMetaClient, false); ValidationUtils.checkArgument(oldColumnStats.getFileName().equals(newColumnStats.getFileName()));
if (newColumnStats.getIsDeleted()) {
return newColumnStats;
}
return HoodieMetadataColumnStats.newBuilder()
.setFileName(newColumnStats.getFileName())
.setMinValue(Stream.of(oldColumnStats.getMinValue(), newColumnStats.getMinValue()).filter(Objects::nonNull).min(Comparator.naturalOrder()).orElse(null))
.setMaxValue(Stream.of(oldColumnStats.getMinValue(), newColumnStats.getMinValue()).filter(Objects::nonNull).max(Comparator.naturalOrder()).orElse(null))
.setValueCount(oldColumnStats.getValueCount() + newColumnStats.getValueCount())
.setNullCount(oldColumnStats.getNullCount() + newColumnStats.getNullCount())
.setTotalSize(oldColumnStats.getTotalSize() + newColumnStats.getTotalSize())
.setTotalUncompressedSize(oldColumnStats.getTotalUncompressedSize() + newColumnStats.getTotalUncompressedSize())
.setIsDeleted(newColumnStats.getIsDeleted())
.build();
} }
public static Stream<HoodieRecord> translateWriteStatToColumnStats(HoodieWriteStat writeStat, public static Stream<HoodieRecord> translateWriteStatToColumnStats(HoodieWriteStat writeStat,
HoodieTableMetaClient datasetMetaClient, HoodieTableMetaClient datasetMetaClient,
List<String> latestColumns) { List<String> columnsToIndex) {
return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, latestColumns, false); if (writeStat instanceof HoodieDeltaWriteStat && ((HoodieDeltaWriteStat) writeStat).getRecordsStats().isPresent()) {
Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangeMap = ((HoodieDeltaWriteStat) writeStat).getRecordsStats().get().getStats();
List<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList = new ArrayList<>(columnRangeMap.values());
return HoodieMetadataPayload.createColumnStatsRecords(writeStat.getPartitionPath(), columnRangeMetadataList, false);
}
return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, columnsToIndex,false);
} }
private static Stream<HoodieRecord> getColumnStats(final String partitionPath, final String filePathWithPartition, private static Stream<HoodieRecord> getColumnStats(final String partitionPath, final String filePathWithPartition,
HoodieTableMetaClient datasetMetaClient, HoodieTableMetaClient datasetMetaClient,
List<String> columns, boolean isDeleted) { List<String> columnsToIndex,
final String partition = partitionPath.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionPath; boolean isDeleted) {
final String partition = getPartition(partitionPath);
final int offset = partition.equals(NON_PARTITIONED_NAME) ? (filePathWithPartition.startsWith("/") ? 1 : 0) final int offset = partition.equals(NON_PARTITIONED_NAME) ? (filePathWithPartition.startsWith("/") ? 1 : 0)
: partition.length() + 1; : partition.length() + 1;
final String fileName = filePathWithPartition.substring(offset); final String fileName = filePathWithPartition.substring(offset);
if (!FSUtils.isBaseFile(new Path(fileName))) {
return Stream.empty();
}
if (filePathWithPartition.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { if (filePathWithPartition.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
List<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList = new ArrayList<>(); List<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList = new ArrayList<>();
@@ -895,13 +932,13 @@ public class HoodieTableMetadataUtil {
if (!isDeleted) { if (!isDeleted) {
try { try {
columnRangeMetadataList = new ParquetUtils().readRangeFromParquetMetadata( columnRangeMetadataList = new ParquetUtils().readRangeFromParquetMetadata(
datasetMetaClient.getHadoopConf(), fullFilePath, columns); datasetMetaClient.getHadoopConf(), fullFilePath, columnsToIndex);
} catch (Exception e) { } catch (Exception e) {
LOG.error("Failed to read column stats for " + fullFilePath, e); LOG.error("Failed to read column stats for " + fullFilePath, e);
} }
} else { } else {
columnRangeMetadataList = columnRangeMetadataList =
columns.stream().map(entry -> new HoodieColumnRangeMetadata<Comparable>(fileName, columnsToIndex.stream().map(entry -> new HoodieColumnRangeMetadata<Comparable>(fileName,
entry, null, null, 0, 0, 0, 0)) entry, null, null, 0, 0, 0, 0))
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
@@ -941,4 +978,72 @@ public class HoodieTableMetadataUtil {
} }
} }
/**
* Accumulates column range metadata for the given field and updates the column range map.
*
* @param field - column for which statistics will be computed
* @param filePath - data file path
* @param columnRangeMap - old column range statistics, which will be merged in this computation
* @param columnToStats - map of column to map of each stat and its value
*/
public static void accumulateColumnRanges(Schema.Field field, String filePath,
Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangeMap,
Map<String, Map<String, Object>> columnToStats) {
Map<String, Object> columnStats = columnToStats.get(field.name());
HoodieColumnRangeMetadata<Comparable> columnRangeMetadata = new HoodieColumnRangeMetadata<>(
filePath,
field.name(),
String.valueOf(columnStats.get(MIN)),
String.valueOf(columnStats.get(MAX)),
Long.parseLong(columnStats.getOrDefault(NULL_COUNT, 0).toString()),
Long.parseLong(columnStats.getOrDefault(VALUE_COUNT, 0).toString()),
Long.parseLong(columnStats.getOrDefault(TOTAL_SIZE, 0).toString()),
Long.parseLong(columnStats.getOrDefault(TOTAL_UNCOMPRESSED_SIZE, 0).toString())
);
columnRangeMap.merge(field.name(), columnRangeMetadata, COLUMN_RANGE_MERGE_FUNCTION);
}
/**
* Aggregates column stats for each field.
*
* @param record - current record
* @param schema - write schema
* @param columnToStats - map of column to map of each stat and its value which gets updates in this method
* @param consistentLogicalTimestampEnabled - flag to deal with logical timestamp type when getting column value
*/
public static void aggregateColumnStats(IndexedRecord record, Schema schema,
Map<String, Map<String, Object>> columnToStats,
boolean consistentLogicalTimestampEnabled) {
if (!(record instanceof GenericRecord)) {
throw new HoodieIOException("Record is not a generic type to get column range metadata!");
}
schema.getFields().forEach(field -> {
Map<String, Object> columnStats = columnToStats.getOrDefault(field.name(), new HashMap<>());
final String fieldVal = getNestedFieldValAsString((GenericRecord) record, field.name(), true, consistentLogicalTimestampEnabled);
// update stats
final int fieldSize = fieldVal == null ? 0 : fieldVal.length();
columnStats.put(TOTAL_SIZE, Long.parseLong(columnStats.getOrDefault(TOTAL_SIZE, 0).toString()) + fieldSize);
columnStats.put(TOTAL_UNCOMPRESSED_SIZE, Long.parseLong(columnStats.getOrDefault(TOTAL_UNCOMPRESSED_SIZE, 0).toString()) + fieldSize);
if (!StringUtils.isNullOrEmpty(fieldVal)) {
// set the min value of the field
if (!columnStats.containsKey(MIN)) {
columnStats.put(MIN, fieldVal);
}
if (fieldVal.compareTo(String.valueOf(columnStats.get(MIN))) < 0) {
columnStats.put(MIN, fieldVal);
}
// set the max value of the field
if (fieldVal.compareTo(String.valueOf(columnStats.getOrDefault(MAX, ""))) > 0) {
columnStats.put(MAX, fieldVal);
}
// increment non-null value count
columnStats.put(VALUE_COUNT, Long.parseLong(columnStats.getOrDefault(VALUE_COUNT, 0).toString()) + 1);
} else {
// increment null value count
columnStats.put(NULL_COUNT, Long.parseLong(columnStats.getOrDefault(NULL_COUNT, 0).toString()) + 1);
}
});
}
} }

View File

@@ -0,0 +1,72 @@
/*
* 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.metadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import java.io.Serializable;
import java.util.List;
/**
* Encapsulates all parameters required to generate metadata index for enabled index types.
*/
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 int columnStatsIndexParallelism;
MetadataRecordsGenerationParams(HoodieTableMetaClient dataMetaClient, List<MetadataPartitionType> enabledPartitionTypes, String bloomFilterType, int bloomIndexParallelism,
boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism) {
this.dataMetaClient = dataMetaClient;
this.enabledPartitionTypes = enabledPartitionTypes;
this.bloomFilterType = bloomFilterType;
this.bloomIndexParallelism = bloomIndexParallelism;
this.isAllColumnStatsIndexEnabled = isAllColumnStatsIndexEnabled;
this.columnStatsIndexParallelism = columnStatsIndexParallelism;
}
public HoodieTableMetaClient getDataMetaClient() {
return dataMetaClient;
}
public List<MetadataPartitionType> getEnabledPartitionTypes() {
return enabledPartitionTypes;
}
public String getBloomFilterType() {
return bloomFilterType;
}
public boolean isAllColumnStatsIndexEnabled() {
return isAllColumnStatsIndexEnabled;
}
public int getBloomIndexParallelism() {
return bloomIndexParallelism;
}
public int getColumnStatsIndexParallelism() {
return columnStatsIndexParallelism;
}
}

View File

@@ -709,7 +709,7 @@ public class HoodieMetadataTableValidator implements Serializable {
.map(entry -> BloomFilterData.builder() .map(entry -> BloomFilterData.builder()
.setPartitionPath(entry.getKey().getKey()) .setPartitionPath(entry.getKey().getKey())
.setFilename(entry.getKey().getValue()) .setFilename(entry.getKey().getValue())
.setBloomFilter(entry.getValue()) .setBloomFilter(ByteBuffer.wrap(entry.getValue().serializeToString().getBytes()))
.build()) .build())
.sorted() .sorted()
.collect(Collectors.toList()); .collect(Collectors.toList());