diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java index 04dedc5dfb676..9406ac91c23b7 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java @@ -42,7 +42,7 @@ public class HoodieCLI { public static CLIState state = CLIState.INIT; public static String basePath; protected static HoodieTableMetaClient tableMetadata; - public static HoodieTableMetaClient syncTableMetadata; + public static HoodieTableMetaClient metaClient; public static TimelineLayoutVersion layoutVersion; private static TempViewProvider tempViewProvider; diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodiePrompt.java b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodiePrompt.java index 44405df5bfac6..1bcfd094a7b68 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodiePrompt.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodiePrompt.java @@ -40,7 +40,7 @@ public String getPrompt() { case TABLE: return "hudi:" + tableName + "->"; case SYNC: - return "hudi:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->"; + return "hudi:" + tableName + " <==> " + HoodieCLI.metaClient.getTableConfig().getTableName() + "->"; default: return "hudi:" + tableName + "->"; } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java index 9517234a0bb60..30f09843f55ec 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java @@ -442,10 +442,10 @@ public String compareCommits(@CliOption(key = {"path"}, help = "Path of the tabl @CliCommand(value = "commits sync", help = "Compare commits with another Hoodie table") public String syncCommits(@CliOption(key = {"path"}, help = "Path of the table to compare to") final String path) { - HoodieCLI.syncTableMetadata = HoodieTableMetaClient.builder().setConf(HoodieCLI.conf).setBasePath(path).build(); + HoodieCLI.metaClient = HoodieTableMetaClient.builder().setConf(HoodieCLI.conf).setBasePath(path).build(); HoodieCLI.state = HoodieCLI.CLIState.SYNC; return "Load sync state between " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " and " - + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); + + HoodieCLI.metaClient.getTableConfig().getTableName(); } /* diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java index 66c2eb02159e4..4eea191c42096 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java @@ -56,10 +56,10 @@ public String validateSync( @CliOption(key = {"hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") final String hivePass) throws Exception { - if (HoodieCLI.syncTableMetadata == null) { + if (HoodieCLI.metaClient == null) { throw new HoodieException("Sync validate request target table not null."); } - HoodieTableMetaClient target = HoodieCLI.syncTableMetadata; + HoodieTableMetaClient target = HoodieCLI.metaClient; HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline(); HoodieTableMetaClient source = HoodieCLI.getTableMetaClient(); HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 11af7f9e9fe0d..0a4997d556984 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -60,6 +60,7 @@ import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.exception.HoodieSavepointException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metrics.HoodieMetrics; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; @@ -187,6 +188,7 @@ public boolean commitStats(String instantTime, List stats, Opti lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); try { preCommit(instantTime, metadata); + table.getMetadataWriter().ifPresent(w -> ((HoodieTableMetadataWriter)w).update(metadata, instantTime)); commit(table, commitActionType, instantTime, metadata, stats); postCommit(table, metadata, instantTime, extraMetadata); LOG.info("Committed " + instantTime); @@ -241,10 +243,6 @@ protected void preCommit(String instantTime, HoodieCommitMetadata metadata) { // TODO : Conflict resolution is not supported for Flink & Java engines } - protected void syncTableMetadata() { - // no-op - } - /** * Filter out HoodieRecords that already exists in the output folder. This is useful in deduplication. * @@ -399,14 +397,6 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp HoodieTableMetaClient metaClient) { setOperationType(writeOperationType); this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient); - this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)), lastCompletedTxnAndMetadata - .isPresent() - ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); - try { - syncTableMetadata(); - } finally { - this.txnManager.endTransaction(); - } this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); } @@ -435,7 +425,6 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); archiveLog.archiveIfRequired(context); autoCleanOnCommit(); - syncTableMetadata(); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } finally { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java index 80a412010203e..39f397ab170d6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java @@ -58,7 +58,7 @@ public static Option resolveWriteConflictIfAny(final Hoodi if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy(); Stream instantStream = resolutionStrategy.getCandidateInstants(table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant); - final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.get()); + final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.orElse(new HoodieCommitMetadata())); instantStream.forEach(instant -> { try { ConcurrentOperation otherOperation = new ConcurrentOperation(instant, table.getMetaClient()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java index 162dc61138e0a..a9bccea073eb6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java @@ -209,6 +209,11 @@ public HoodieLockConfig.Builder withLockProvider(Class l return this; } + public HoodieLockConfig.Builder withLockProviderClass(String lockProviderClassName) { + lockConfig.setValue(LOCK_PROVIDER_CLASS_PROP, lockProviderClassName); + return this; + } + public HoodieLockConfig.Builder withHiveDatabaseName(String databaseName) { lockConfig.setValue(HIVE_DATABASE_NAME_PROP, databaseName); return this; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 6e87257b3f428..736b91ef30be2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1220,10 +1220,6 @@ public boolean useFileListingMetadata() { return metadataConfig.useFileListingMetadata(); } - public boolean getFileListingMetadataVerify() { - return metadataConfig.validateFileListingMetadata(); - } - public int getMetadataInsertParallelism() { return getInt(HoodieMetadataConfig.METADATA_INSERT_PARALLELISM_PROP); } @@ -1305,6 +1301,17 @@ public boolean allowEmptyCommit() { return getBooleanOrDefault(ALLOW_EMPTY_COMMIT); } + /** + * Record level index configs. + */ + public boolean isRecordLevelIndexEnabled() { + return metadataConfig.isRecordLevelIndexEnabled(); + } + + public int getRecordLevelIndexShardCount() { + return metadataConfig.getRecordLevelIndexShardCount(); + } + public static class Builder { protected final HoodieWriteConfig writeConfig = new HoodieWriteConfig(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 2e1915ff20431..91e6248f45d9c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -22,9 +22,11 @@ import org.apache.hudi.PublicAPIClass; import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.table.HoodieTable; @@ -42,10 +44,26 @@ @PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) public abstract class HoodieIndex implements Serializable { + // Metric names + protected static final String UPDATE_DURATION = "update.duration"; + protected static final String UPDATE_COUNT = "update.count"; + protected static final String INSERT_COUNT = "insert.count"; + protected static final String DELETE_COUNT = "delete.count"; + protected static final String TAG_LOCATION_DURATION = "tag.duration"; + protected static final String TAG_LOCATION_COUNT = "tag.count"; + protected static final String TAG_LOCATION_NUM_PARTITIONS = "tag.num_partitions"; + protected static final String TAG_LOCATION_HITS = "tag.hits"; + + // Metric registry + protected Option registry; + protected final HoodieWriteConfig config; protected HoodieIndex(HoodieWriteConfig config) { this.config = config; + if (config.getTableName() != null) { + this.registry = Option.of(Registry.getRegistry(config.getTableName() + "." + this.getClass().getSimpleName())); + } } /** @@ -104,6 +122,6 @@ public void close() { } public enum IndexType { - HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE + HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE, RECORD_LEVEL } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 9cd5301fb5c8f..30157a91313aa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -19,10 +19,10 @@ package org.apache.hudi.metadata; import org.apache.hudi.avro.model.HoodieCleanMetadata; -import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -32,15 +32,22 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -49,7 +56,6 @@ import org.apache.hudi.config.HoodieMetricsConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hadoop.conf.Configuration; @@ -65,9 +71,11 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP; +import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; @@ -76,7 +84,7 @@ * called Metadata Table. This table is created by listing files and partitions (first time) * and kept in sync using the instants on the main dataset. */ -public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMetadataWriter { +public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMetadataWriter { private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadataWriter.class); @@ -86,10 +94,12 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta protected HoodieBackedTableMetadata metadata; protected HoodieTableMetaClient metaClient; + protected HoodieTableMetaClient datasetMetaClient; protected Option metrics; protected boolean enabled; protected SerializableConfiguration hadoopConf; protected final transient HoodieEngineContext engineContext; + protected TransactionManager txnManager; protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) { @@ -110,24 +120,30 @@ protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteC ValidationUtils.checkArgument(!this.metadataWriteConfig.useFileListingMetadata(), "File listing cannot be used for Metadata Table"); initRegistry(); - HoodieTableMetaClient datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(datasetWriteConfig.getBasePath()).build(); - initialize(engineContext, datasetMetaClient); - if (enabled) { - // This is always called even in case the table was created for the first time. This is because - // initFromFilesystem() does file listing and hence may take a long time during which some new updates - // may have occurred on the table. Hence, calling this always ensures that the metadata is brought in sync - // with the active timeline. - HoodieTimer timer = new HoodieTimer().startTimer(); - syncFromInstants(datasetMetaClient); - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.SYNC_STR, timer.endTimer())); - } + this.datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(datasetWriteConfig.getBasePath()).build(); + initTransactionManager(); + initTableMetadata(); + initialize(engineContext); } else { enabled = false; this.metrics = Option.empty(); } } - protected abstract void initRegistry(); + /** + * Initialize the {@code TransactionManager} to use for metadata table. + * + * In HUDI multi writer mode, each operation will sync to metadata table before completion. Metadata table has common + * base and log files to update for each operation. So we can only support serialized operations. + */ + private void initTransactionManager() { + // The lock location should be different from the dataset + Properties properties = new Properties(); + properties.putAll(datasetWriteConfig.getProps()); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, properties.getProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, datasetWriteConfig.getBasePath() + "/.hoodie/.locks") + "/metadata"); + HoodieWriteConfig txConfig = HoodieWriteConfig.newBuilder().withProperties(properties).build(); + this.txnManager = new TransactionManager(txConfig, datasetMetaClient.getFs()); + } /** * Create a {@code HoodieWriteConfig} to use for the Metadata Table. @@ -210,19 +226,6 @@ public HoodieBackedTableMetadata metadata() { return metadata; } - /** - * Initialize the metadata table if it does not exist. Update the metadata to bring it in sync with the file system. - * - * This can happen in two ways: - * 1. If the metadata table did not exist, then file and partition listing is used - * 2. If the metadata table exists, the instants from active timeline are read in order and changes applied - * - * The above logic has been chosen because it is faster to perform #1 at scale rather than read all the Instants - * which are large in size (AVRO or JSON encoded and not compressed) and incur considerable IO for de-serialization - * and decoding. - */ - protected abstract void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient); - protected void initTableMetadata() { try { if (this.metadata != null) { @@ -237,7 +240,6 @@ protected void initTableMetadata() { } protected void bootstrapIfNeeded(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) throws IOException { - HoodieTimer timer = new HoodieTimer().startTimer(); boolean exists = datasetMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); boolean rebootstrap = false; if (exists) { @@ -264,35 +266,82 @@ protected void bootstrapIfNeeded(HoodieEngineContext engineContext, HoodieTableM exists = false; } + // If there is no commit on the dataset yet, use the SOLO_COMMIT_TIMESTAMP as the instant time for initial commit + // Otherwise, we use the latest commit timestamp. + String createInstantTime = datasetMetaClient.getActiveTimeline().getReverseOrderedInstants().findFirst() + .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + + Option> partitionsBootstrapped = Option.empty(); + + // Bootstrap file listing if required if (!exists) { - // Initialize for the first time by listing partitions and files directly from the file system - if (bootstrapFromFilesystem(engineContext, datasetMetaClient)) { - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); + HoodieTimer timer = new HoodieTimer().startTimer(); + partitionsBootstrapped = bootstrapFileListings(engineContext, datasetMetaClient, createInstantTime); + if (partitionsBootstrapped.isPresent()) { + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_FILE_LISTING_STR, timer.endTimer())); + } else if (datasetWriteConfig.isRecordLevelIndexEnabled()) { + // Since record level index is enabled, we need the file listing bootstrap to succeed. + throw new HoodieMetadataException("Failed to bootstrap file listing for Metadata Table"); + } else { + // Bootstrap will be attempted next time. + return; + } + } + + // Bootstrap record level index if required + if (datasetWriteConfig.isRecordLevelIndexEnabled()) { + HoodieTimer timer = new HoodieTimer().startTimer(); + final long foundShardCount = HoodieTableMetadataUtil.loadPartitionShards(metaClient, MetadataPartitionType.RECORD_LEVEL_INDEX.partitionPath()).size(); + // The number of existing shards should be 0 if bootstrap failed as the commit would not have succeeded. We + // do not support changing the number of shards here. + ValidationUtils.checkState(foundShardCount == 0 || foundShardCount == datasetWriteConfig.getRecordLevelIndexShardCount(), + String.format("Found %d shards for record index which is different than configured count %d", foundShardCount, + datasetWriteConfig.getRecordLevelIndexShardCount())); + + if (foundShardCount == 0) { + exists = false; + + // If we did not bootstrap file listing, we need to get the list of partitions + if (!partitionsBootstrapped.isPresent()) { + partitionsBootstrapped = Option.of(FSUtils.getAllPartitionPaths(engineContext, datasetMetaClient.getBasePath(), false, datasetWriteConfig.shouldAssumeDatePartitioning())); + } + + if (bootstrapRecordLevelIndex(engineContext, datasetMetaClient, createInstantTime, partitionsBootstrapped.get())) { + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_RECORD_INDEX_STR, timer.endTimer())); + } else { + // Failure to bootstrap is a fatal error as we wont be able to tag records for commit and it may lead to + // duplicates. + throw new HoodieMetadataException("Failed to bootstrap file listing for Metadata Table"); + } + } } + + if (!exists) { + performInsert(createInstantTime); + } } /** * Initialize the Metadata Table by listing files and partitions from the file system. * * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + * @param createInstantTime InstantTime to use for the commit */ - private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) throws IOException { + private Option> bootstrapFileListings(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient, + String createInstantTime) throws IOException { 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 Option pendingInstantOption = Option.fromJavaOptional(datasetMetaClient.getActiveTimeline() .getReverseOrderedInstants().filter(i -> !i.isCompleted()).findFirst()); - if (pendingInstantOption.isPresent()) { - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BOOTSTRAP_ERR_STR, 1)); - LOG.warn("Cannot bootstrap metadata table as operation is in progress: " + pendingInstantOption.get()); - return false; - } + // TODO: disabled for testing + //if (pendingInstantOption.isPresent()) { + // metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BOOTSTRAP_ERR_STR, 1)); + // LOG.warn("Cannot bootstrap metadata table as operation is in progress: " + pendingInstantOption.get()); + // return Option.empty(); + //} - // If there is no commit on the dataset yet, use the SOLO_COMMIT_TIMESTAMP as the instant time for initial commit - // Otherwise, we use the latest commit timestamp. - String createInstantTime = datasetMetaClient.getActiveTimeline().getReverseOrderedInstants().findFirst() - .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime); HoodieTableMetaClient.withPropertyBuilder() @@ -304,6 +353,7 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); initTableMetadata(); + initializeShards(datasetMetaClient, MetadataPartitionType.FILES.partitionPath(), createInstantTime, 1); // List all partitions in the basePath of the containing dataset LOG.info("Initializing metadata table by using file listings in " + datasetWriteConfig.getBasePath()); @@ -337,10 +387,63 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi }); LOG.info("Committing " + partitionToFileStatus.size() + " partitions and " + stats[0] + " files to metadata"); - update(commitMetadata, createInstantTime); + List records = HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, createInstantTime); + queueForUpdate(records, MetadataPartitionType.FILES, createInstantTime); + + return Option.of(partitionToFileStatus.keySet().stream().sorted().collect(Collectors.toList())); + } + + /** + * Bootstrap the record level index. + * + * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + * @param createInstantTime InstantTime to use for the commit + * @param partitions List of partitions from which the data files are to be read + */ + private boolean bootstrapRecordLevelIndex(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient, + String createInstantTime, List partitions) throws IOException { + ValidationUtils.checkState(enabled, "Record level index cannot be initialized as Metadata Table is not enabled"); + ValidationUtils.checkState(datasetWriteConfig.isRecordLevelIndexEnabled(), "Record level index cannot be initialized as it is not enabled"); + + HoodieTimer timer = new HoodieTimer().startTimer(); + + // Initialize the shards + final int numShards = datasetWriteConfig.getRecordLevelIndexShardCount(); + initializeShards(datasetMetaClient, MetadataPartitionType.RECORD_LEVEL_INDEX.partitionPath(), createInstantTime, numShards); + + BaseFileOnlyView fsView = getTableFileSystemView(); + + // TODO: scale is an issue here. Remove this limit later + partitions = partitions.subList(partitions.size() - Math.min(partitions.size(), 2), partitions.size()); + + // Collect the list of base files present + List> partitionBaseFilePairs = engineContext.flatMap(partitions, partition -> { + return fsView.getLatestBaseFiles(partition).map(basefile -> Pair.of(partition, basefile.getFileName())); + }, partitions.size()); + + LOG.info("Initializing record level index from " + partitionBaseFilePairs.size() + " base files in " + + partitions.size() + " partitions"); + + // Collect record keys from the files in parallel + if (!partitionBaseFilePairs.isEmpty()) { + Pair recordKeysWithCount = readRecordKeysFromBaseFiles(engineContext, partitionBaseFilePairs); + queueForUpdate(recordKeysWithCount.getLeft(), MetadataPartitionType.RECORD_LEVEL_INDEX, createInstantTime); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.RECORDINDEX_BOOTSTRAP_COUNT_STR, recordKeysWithCount.getRight())); + } + + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.RECORDINDEX_SHARD_COUNT_STR, numShards)); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.READ_RECORDKEYS_BOOTSTRAP_STR, timer.endTimer())); return true; } + private void performInsert(String instantTime) { + commit(instantTime, true); + } + + private void performUpdate(String instantTime) { + commit(instantTime, false); + } + /** * Function to find hoodie partitions and list files in them in parallel. * @@ -396,37 +499,56 @@ private Map> getPartitionsToFilesMapping(HoodieTableMet } /** - * Sync the Metadata Table from the instants created on the dataset. + * Initialize shards for a partition. * - * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + * Each shard is a single log file with the following format: + * ABCD + * where ABCD are digits. This allows up to 9999 shards. + * + * Example: + * fc9f18eb-6049-4f47-bc51-23884bef0001 + * fc9f18eb-6049-4f47-bc51-23884bef0002 */ - private void syncFromInstants(HoodieTableMetaClient datasetMetaClient) { - ValidationUtils.checkState(enabled, "Metadata table cannot be synced as it is not enabled"); - // (re) init the metadata for reading. - initTableMetadata(); - try { - List instantsToSync = metadata.findInstantsToSyncForWriter(); - if (instantsToSync.isEmpty()) { - return; - } - - LOG.info("Syncing " + instantsToSync.size() + " instants to metadata table: " + instantsToSync); - - // Read each instant in order and sync it to metadata table - for (HoodieInstant instant : instantsToSync) { - LOG.info("Syncing instant " + instant + " to metadata table"); - - Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(datasetMetaClient, instant, getLatestSyncedInstantTime()); - if (records.isPresent()) { - commit(records.get(), MetadataPartitionType.FILES.partitionPath(), instant.getTimestamp()); - } + private void initializeShards(HoodieTableMetaClient datasetMetaClient, String partition, String instantTime, + int shardCount) throws IOException { + ValidationUtils.checkArgument(shardCount <= 9999, "Maximum 9999 shards are supported."); + + final String newFileId = FSUtils.createNewFileIdPfx(); + final String newFileIdPrefix = newFileId.substring(0, 32); + final HashMap blockHeader = new HashMap<>(); + blockHeader.put(HeaderMetadataType.INSTANT_TIME, instantTime); + final HoodieDeleteBlock block = new HoodieDeleteBlock(new HoodieKey[0], blockHeader); + + LOG.info(String.format("Creating %d shards for partition %s with base fileId %s at instant time %s", + shardCount, partition, newFileId, instantTime)); + for (int i = 0; i < shardCount; ++i) { + // Generate a indexed fileId for each shard and write a log block into it to create the file. + final String shardFileId = String.format("%s%04d", newFileIdPrefix, i + 1); + ValidationUtils.checkArgument(newFileId.length() == shardFileId.length(), "FileId should be of length " + newFileId.length()); + try { + HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() + .onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), partition)) + .withFileId(shardFileId).overBaseCommit(instantTime) + .withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION) + .withFileSize(0L) + .withSizeThreshold(metadataWriteConfig.getLogFileMaxSize()) + .withFs(datasetMetaClient.getFs()) + .withRolloverLogWriteToken(FSUtils.makeWriteToken(0, 0, 0)) + .withLogWriteToken(FSUtils.makeWriteToken(0, 0, 0)) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); + writer.appendBlock(block); + writer.close(); + } catch (InterruptedException e) { + throw new IOException("Failed to created record level index shard " + shardFileId, e); } - initTableMetadata(); - } catch (IOException ioe) { - throw new HoodieIOException("Unable to sync instants from data to metadata table.", ioe); } } + protected String getShardFileName(String fileId, int shardIndex) { + ValidationUtils.checkArgument(shardIndex <= 9999, "Maximum 9999 shards are supported."); + return String.format("%s%04d", fileId.substring(0, 32), shardIndex + 1); + } + /** * Update from {@code HoodieCommitMetadata}. * @@ -436,22 +558,15 @@ private void syncFromInstants(HoodieTableMetaClient datasetMetaClient) { @Override public void update(HoodieCommitMetadata commitMetadata, String instantTime) { if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime); - commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); - } - } - - /** - * Update from {@code HoodieCleanerPlan}. - * - * @param cleanerPlan {@code HoodieCleanerPlan} - * @param instantTime Timestamp at which the clean plan was generated - */ - @Override - public void update(HoodieCleanerPlan cleanerPlan, String instantTime) { - if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanerPlan, instantTime); - commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)), + Option.empty()); + try { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime); + queueForUpdate(records, MetadataPartitionType.FILES, instantTime); + performUpdate(instantTime); + } finally { + this.txnManager.endTransaction(); + } } } @@ -464,8 +579,15 @@ public void update(HoodieCleanerPlan cleanerPlan, String instantTime) { @Override public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime); - commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)), + Option.empty()); + try { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime); + queueForUpdate(records, MetadataPartitionType.FILES, instantTime); + performUpdate(instantTime); + } finally { + this.txnManager.endTransaction(); + } } } @@ -478,8 +600,15 @@ public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { @Override public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(restoreMetadata, instantTime, metadata.getSyncedInstantTime()); - commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)), + Option.empty()); + try { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(restoreMetadata, instantTime, metadata.getSyncedInstantTime()); + queueForUpdate(records, MetadataPartitionType.FILES, instantTime); + performUpdate(instantTime); + } finally { + this.txnManager.endTransaction(); + } } } @@ -492,8 +621,27 @@ public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { @Override public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) { if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(rollbackMetadata, instantTime, metadata.getSyncedInstantTime()); - commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)), + Option.empty()); + try { + // Is this rollback of an instant that has been synced to the metadata table? + String rollbackInstant = rollbackMetadata.getCommitsRollback().get(0); + boolean wasSynced = metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, rollbackInstant)); + if (!wasSynced) { + // A compaction may have taken place on metadata table which would have included this instant being rolled back. + Option latestCompaction = metadata.getLatestCompactionTime(); + if (latestCompaction.isPresent()) { + wasSynced = HoodieTimeline.compareTimestamps(rollbackInstant, HoodieTimeline.LESSER_THAN_OR_EQUALS, latestCompaction.get()); + } + } + + List records = HoodieTableMetadataUtil.convertMetadataToRecords(rollbackMetadata, instantTime, + metadata.getSyncedInstantTime(), wasSynced); + queueForUpdate(records, MetadataPartitionType.FILES, instantTime); + performUpdate(instantTime); + } finally { + this.txnManager.endTransaction(); + } } } @@ -504,9 +652,48 @@ public void close() throws Exception { } } + protected abstract void initRegistry(); + + /** + * Initialize the metadata table if it does not exist. + * + * If the metadata table did not exist, then file and partition listing is used to bootstrap the table. + */ + protected abstract void initialize(HoodieEngineContext engineContext); + + /** + * Returns a {@code HoodieTableFileSystemView} on the dataset. + */ + protected abstract BaseFileOnlyView getTableFileSystemView(); + + /** + * Queue the given records for an update to the given partition. + * + * Each Metadata Table partition has fixed number of shards. The records should be sharded by their record-key so that + * they are written to the correct shard. + * + * @param records The records to be queued + * @param partitionName The partition to which the records are to be written. + * @param instantTime The timestamp to use for the deltacommit. + */ + protected abstract void queueForUpdate(R records, MetadataPartitionType partitionType, String instantTime); + + protected abstract void queueForUpdate(List records, MetadataPartitionType partitionType, String instantTime); + + /** + * Read the record keys from the given base files in parallel and return records. + * + * @param engineContext + * @param partitionBaseFilePairs + */ + protected abstract Pair readRecordKeysFromBaseFiles(HoodieEngineContext engineContext, + List> partitionBaseFilePairs); + /** - * Commit the {@code HoodieRecord}s to Metadata Table as a new delta-commit. + * Commit the previously queued records to Metadata Table as a new delta-commit. * + * @param instantTime The instant time for the delta-commit + * @param isInsert If true, perform an insert. If false, perform an update */ - protected abstract void commit(List records, String partitionName, String instantTime); + protected abstract void commit(String instantTime, boolean isInsert); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index 1b02a3b92b353..b125182fa0f2f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -18,12 +18,16 @@ package org.apache.hudi.metadata; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieCleanMetadata; -import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieMetadataException; import java.io.Serializable; @@ -32,18 +36,36 @@ */ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { - void update(HoodieCommitMetadata commitMetadata, String instantTime); - - void update(HoodieCleanerPlan cleanerPlan, String instantTime); + // Update the metadata table due to a COMMIT operation + void update(HoodieCommitMetadata option, String instantTime); + // Update the metadata table due to a CLEAN operation void update(HoodieCleanMetadata cleanMetadata, String instantTime); + // Update the metadata table due to a RESTORE operation void update(HoodieRestoreMetadata restoreMetadata, String instantTime); + // Update the metadata table due to a ROLLBACK operation void update(HoodieRollbackMetadata rollbackMetadata, String instantTime); /** * Return the timestamp of the latest instant synced to the metadata table. */ Option getLatestSyncedInstantTime(); + + /** + * Remove the metadata table for the dataset. + * + * @param basePath base path of the dataset + * @param context + */ + static void removeMetadataTable(String basePath, HoodieEngineContext context) { + final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); + FileSystem fs = FSUtils.getFs(metadataTablePath, context.getHadoopConf().get()); + try { + fs.delete(new Path(metadataTablePath), true); + } catch (Exception e) { + throw new HoodieMetadataException("Failed to remove metadata table from path " + metadataTablePath, e); + } + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 19cc010bfb148..574d3f77faa34 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -58,6 +58,7 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; 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.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -66,6 +67,7 @@ import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.log4j.LogManager; @@ -98,7 +100,7 @@ public abstract class HoodieTable implem protected final HoodieIndex index; private SerializableConfiguration hadoopConfiguration; protected final TaskContextSupplier taskContextSupplier; - private final HoodieTableMetadata metadata; + private final HoodieTableMetadata metadataReader; private transient FileSystemViewManager viewManager; protected final transient HoodieEngineContext context; @@ -110,10 +112,10 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(config.getMetadataConfig().getProps()) .build(); - this.metadata = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath(), + this.metadataReader = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue()); - this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadata); + this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadataReader); this.metaClient = metaClient; this.index = getIndex(config, context); this.taskContextSupplier = context.getTaskContextSupplier(); @@ -123,7 +125,7 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo private synchronized FileSystemViewManager getViewManager() { if (null == viewManager) { - viewManager = FileSystemViewManager.createViewManager(getContext(), config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadata); + viewManager = FileSystemViewManager.createViewManager(getContext(), config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadataReader); } return viewManager; } @@ -679,4 +681,13 @@ public HoodieEngineContext getContext() { // to engine context, and it ends up being null (as its not serializable and marked transient here). return context == null ? new HoodieLocalEngineContext(hadoopConfiguration.get()) : context; } + + public HoodieTableMetadata getMetadataReader() { + return metadataReader; + } + + public Option getMetadataWriter() { + ValidationUtils.checkArgument(!config.useFileListingMetadata(), "Metadata Table support not enabled in this Table"); + return Option.empty(); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index 35dbd9e819bbe..12a3d54ee19e0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -200,20 +200,19 @@ private Stream getInstantsToArchive() { .collect(Collectors.groupingBy(i -> Pair.of(i.getTimestamp(), HoodieInstant.getComparableAction(i.getAction())))); - // If metadata table is enabled, do not archive instants which are more recent that the latest synced - // instant on the metadata table. This is required for metadata table sync. + // If metadata table is enabled, do not archive instants which are more recent that the last compaction on the + // metadata table. if (config.useFileListingMetadata()) { try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(), config.getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) { - Option lastSyncedInstantTime = tableMetadata.getSyncedInstantTime(); - - if (lastSyncedInstantTime.isPresent()) { - LOG.info("Limiting archiving of instants to last synced instant on metadata table at " + lastSyncedInstantTime.get()); - instants = instants.filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.LESSER_THAN, - lastSyncedInstantTime.get())); - } else { - LOG.info("Not archiving as there is no instants yet on the metadata table"); + Option latestCompactionTime = tableMetadata.getLatestCompactionTime(); + if (!latestCompactionTime.isPresent()) { + LOG.info("Not archiving as there is no compaction yet on the metadata table"); instants = Stream.empty(); + } else { + LOG.info("Limiting archiving of instants to latest compaction on metadata table at " + latestCompactionTime.get()); + instants = instants.filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.LESSER_THAN, + latestCompactionTime.get())); } } catch (Exception e) { throw new HoodieException("Error limiting instant archival based on metadata table", e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java index 6c776cfb077ae..9dd76d7fe3fcb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java @@ -21,7 +21,11 @@ import java.io.Serializable; import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -46,4 +50,20 @@ public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, } public abstract R execute(); + + protected final void syncTableMetadata(HoodieCommitMetadata metadata) { + table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + } + + protected final void syncTableMetadata(HoodieCleanMetadata metadata) { + table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + } + + protected final void syncTableMetadata(HoodieRollbackMetadata metadata) { + table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + } + + protected final void syncTableMetadata(HoodieRestoreMetadata metadata) { + table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java index acc3cdc6793ab..f83ddfb50fbdd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java @@ -115,6 +115,8 @@ private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstan cleanStats ); + syncTableMetadata(metadata); + table.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant, TimelineMetadataUtils.serializeCleanMetadata(metadata)); LOG.info("Marked clean started on " + inflightInstant.getTimestamp() + " as complete"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 4b519ed92b4aa..ce6ed5db303c7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -175,10 +175,6 @@ protected void finalizeWrite(String instantTime, List stats, Ho } } - protected void syncTableMetadata() { - // No Op - } - /** * By default, return the writer schema in Write Config for storing in commit. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 3b722a7a41ac9..ef18d245514ff 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -92,6 +92,7 @@ private HoodieRestoreMetadata finishRestore(Map doRollbackAndGetStats() { protected void finishRollback(HoodieRollbackMetadata rollbackMetadata) throws HoodieIOException { try { + // TODO: Potential error here - rollbacks have already completed here so if the syncTableMetadata fails, + // metadata table will be left in an inconsistent state. This is because we do not use the inflight + // state for rollback. + syncTableMetadata(rollbackMetadata); + table.getActiveTimeline().createNewInstant( new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.ROLLBACK_ACTION, instantTime)); table.getActiveTimeline().saveAsComplete( diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java index c6a1527e292b0..674a488b7a7e7 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java @@ -28,6 +28,7 @@ import org.apache.hudi.exception.HoodieLockException; import java.io.IOException; +import java.io.Serializable; import java.util.concurrent.TimeUnit; import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY; @@ -39,12 +40,12 @@ * create operation. This lock does not support cleaning/expiring the lock after a failed write hence cannot be used * in production environments. */ -public class FileSystemBasedLockProviderTestClass implements LockProvider { +public class FileSystemBasedLockProviderTestClass implements LockProvider, Serializable { private static final String LOCK_NAME = "acquired"; private String lockPath; - private FileSystem fs; + private transient FileSystem fs; protected LockConfiguration lockConfiguration; public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfiguration, final Configuration configuration) { @@ -55,7 +56,7 @@ public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfigur public void acquireLock() { try { - fs.create(new Path(lockPath + "/" + LOCK_NAME)).close(); + fs.create(new Path(lockPath + "/" + LOCK_NAME), false).close(); } catch (IOException e) { throw new HoodieIOException("Failed to acquire lock", e); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index 5b751e4f1eaec..1ec25763df052 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -42,7 +42,6 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.FlinkHoodieIndex; import org.apache.hudi.index.HoodieIndex; @@ -52,8 +51,6 @@ import org.apache.hudi.io.FlinkMergeHandle; import org.apache.hudi.io.HoodieWriteHandle; import org.apache.hudi.io.MiniBatchHandle; -import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; @@ -389,16 +386,6 @@ protected HoodieTable>, List, List records, String partitionName, String instantTime) { - ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); - List recordRDD = prepRecords(records, partitionName); - - try (HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(engineContext, metadataWriteConfig, true)) { - writeClient.startCommitWithTime(instantTime); - writeClient.transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, instantTime); - - List statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime); - statuses.forEach(writeStatus -> { - if (writeStatus.hasErrors()) { - throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime); - } - }); - writeClient.commit(instantTime, statuses, Option.empty(), HoodieActiveTimeline.DELTA_COMMIT_ACTION, Collections.emptyMap()); - // trigger cleaning, compaction, with suffixes based on the same instant time. This ensures that any future - // delta commits synced over will not have an instant time lesser than the last completed instant on the - // metadata table. - if (writeClient.scheduleCompactionAtInstant(instantTime + "001", Option.empty())) { - writeClient.compact(instantTime + "001"); - } - writeClient.clean(instantTime + "002"); - } - - // Update total size of the metadata and count of base/log files - metrics.ifPresent(m -> { - try { - Map stats = m.getStats(false, metaClient, metadata); - m.updateMetrics(Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)), - Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE)), - Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)), - Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES))); - } catch (HoodieIOException e) { - LOG.error("Could not publish metadata size metrics", e); - } - }); - } - /** * Return the timestamp of the latest instant synced. *

@@ -188,4 +144,66 @@ private List prepRecords(List records, String partit return records.stream().map(r -> r.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId))).collect(Collectors.toList()); } + + @Override + protected BaseFileOnlyView getTableFileSystemView() { + // TODO + throw new UnsupportedOperationException(); + } + + @Override + protected void queueForUpdate(Object records, MetadataPartitionType partitionType, String instantTime) { + // TODO + throw new UnsupportedOperationException(); + } + + @Override + protected void queueForUpdate(List records, MetadataPartitionType partitionType, String instantTime) { + // TODO + throw new UnsupportedOperationException(); + } + + @Override + protected Pair readRecordKeysFromBaseFiles(HoodieEngineContext engineContext, List partitionBaseFilePairs) { + // TODO + throw new UnsupportedOperationException(); + } + + @Override + protected void commit(String instantTime, boolean isInsert) { + // TODO + /* + ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); + List recordRDD = prepRecords(records, partitionName); + + try (HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(engineContext, metadataWriteConfig, true)) { + writeClient.startCommitWithTime(instantTime); + writeClient.transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, instantTime); + + List statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime); + statuses.forEach(writeStatus -> { + if (writeStatus.hasErrors()) { + throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime); + } + }); + writeClient.commit(instantTime, statuses, Option.empty(), HoodieActiveTimeline.DELTA_COMMIT_ACTION, Collections.emptyMap()); + // trigger cleaning, compaction, with suffixes based on the same instant time. This ensures that any future + // delta commits synced over will not have an instant time lesser than the last completed instant on the + // metadata table. + if (writeClient.scheduleCompactionAtInstant(instantTime + "001", Option.empty())) { + writeClient.compact(instantTime + "001"); + } + writeClient.clean(instantTime + "002"); + } + + // Update total size of the metadata and count of base/log files + metrics.ifPresent(m -> { + try { + m.updateSizeMetrics(metaClient, metadata); + } catch (HoodieIOException e) { + LOG.error("Could not publish metadata size metrics", e); + } + }); + */ + } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java index 5cfd28be2c1dc..a5320bcbca0f6 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java @@ -141,13 +141,14 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta result.setWriteStats(writeStats); // Finalize write finalizeWrite(instantTime, writeStats, result); - syncTableMetadata(); try { LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + syncTableMetadata(metadata); + activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java index af57a9e5c36dc..6aa9430585671 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java @@ -43,7 +43,9 @@ public void run(HoodieTableMetaClient metaClient, HoodieTableVersion toVersion, @Override protected void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { - if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) { + if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.TWO) { + // TODO: + } else if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) { new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime); } else { throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true); @@ -52,7 +54,9 @@ protected void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVers @Override protected void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { - if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) { + if (fromVersion == HoodieTableVersion.TWO && toVersion == HoodieTableVersion.ONE) { + // TODO: + } else if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) { new OneToZeroDowngradeHandler().downgrade(config, context, instantTime); } else { throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java index aec84a50e18af..999e31ebb9dfb 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -206,6 +206,8 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + syncTableMetadata(metadata); + activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java index 54259c94175a3..cc661230b06fb 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java @@ -52,8 +52,7 @@ protected List getAllExistingFileIds(String partitionPath) { protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeResult) { Map> partitionToExistingFileIds = new HashMap<>(); List partitionPaths = FSUtils.getAllPartitionPaths(context, - table.getMetaClient().getBasePath(), config.useFileListingMetadata(), - config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()); + table.getMetaClient().getBasePath(), config.useFileListingMetadata(), config.shouldAssumeDatePartitioning()); if (partitionPaths != null && partitionPaths.size() > 0) { partitionToExistingFileIds = context.mapToPair(partitionPaths, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index a88da49dcb405..5294afc34b0c9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -42,10 +42,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieCommitException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndex; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.hudi.table.BulkInsertPartitioner; @@ -79,7 +77,7 @@ public class SparkRDDWriteClient extends private static final Logger LOG = LogManager.getLogger(SparkRDDWriteClient.class); public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { - super(context, clientConfig); + this(context, clientConfig, Option.empty()); } @Deprecated @@ -96,6 +94,11 @@ public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeC public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, Option timelineService) { super(context, writeConfig, timelineService); + if (config.useFileListingMetadata()) { + // If the metadata table does not exist, it should be bootstrapped here + // TODO: Check if we can remove this requirement - auto bootstrap on commit + SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context); + } } /** @@ -299,6 +302,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD writeStats = writeStatuses.map(WriteStatus::getStat).collect(); finalizeWrite(table, compactionCommitTime, writeStats); + table.getMetadataWriter().ifPresent(w -> w.update(metadata, compactionCommitTime)); LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata); SparkCompactHelpers.newInstance().completeInflightCompaction(table, compactionCommitTime, metadata); @@ -444,16 +448,6 @@ private HoodieTable>, JavaRDD, JavaRDD extends SparkHoodieIndex { + private static final Logger LOG = LogManager.getLogger(SparkRecordLevelIndex.class); + + public SparkRecordLevelIndex(HoodieWriteConfig config) { + super(config); + } + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + final int numShards = config.getRecordLevelIndexShardCount(); + JavaRDD> y = recordRDD.keyBy(r -> HoodieTableMetadataUtil.keyToShard(r.getRecordKey(), numShards)) + .partitionBy(new PartitionIdPassthrough(numShards)) + .map(t -> t._2); + ValidationUtils.checkState(y.getNumPartitions() <= numShards); + + registry.ifPresent(r -> r.add(TAG_LOCATION_NUM_PARTITIONS, y.getNumPartitions())); + return y.mapPartitions(new LocationTagFunction(hoodieTable, registry)); + } + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + public JavaRDD updateLocation(JavaRDD writeStatusRDD, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + HoodieTimer timer = new HoodieTimer().startTimer(); + long[] counts = { 0L, 0L, 0L }; // insert, update, delete + JavaRDD indexUpdateRDD = writeStatusRDD.flatMap(writeStatus -> { + List records = new LinkedList<>(); + for (HoodieRecord writtenRecord : writeStatus.getWrittenRecords()) { + if (!writeStatus.isErrored(writtenRecord.getKey())) { + HoodieRecord indexRecord; + HoodieKey key = writtenRecord.getKey(); + ValidationUtils.checkState(!key.getRecordKey().contains("Option")); + Option newLocation = writtenRecord.getNewLocation(); + if (newLocation.isPresent()) { + if (writtenRecord.getCurrentLocation() != null) { + // Update + counts[1] += 1; + // TODO: updates are not currently supported but are required for clustering use-case. We should make + // sure that if the fileID has changed then we update it. + // TODO: How to differentiate dupes here? + continue; + } else { + // Insert + counts[0] += 1; + } + + indexRecord = HoodieMetadataPayload.createRecordLevelIndexRecord(key.getRecordKey(), key.getPartitionPath(), + newLocation.get().getFileId(), newLocation.get().getInstantTime()); + } else { + // Delete existing index for a deleted record + counts[2] += 1; + indexRecord = HoodieMetadataPayload.createRecordLevelIndexDelete(key.getRecordKey()); + } + + records.add(indexRecord); + } + } + + + return records.iterator(); + }); + + // TODO: Find a better way to enqueue records on the driver side for commit to metadata table + if (hoodieTable.getMetadataWriter().isPresent()) { + SparkHoodieBackedTableMetadataWriter metadataWriter = (SparkHoodieBackedTableMetadataWriter) hoodieTable.getMetadataWriter().get(); + metadataWriter.queueForUpdate(indexUpdateRDD, MetadataPartitionType.RECORD_LEVEL_INDEX, ""); + } + + registry.ifPresent(r -> r.add(UPDATE_DURATION, timer.endTimer())); + registry.ifPresent(r -> r.add(UPDATE_COUNT, counts[1])); + registry.ifPresent(r -> r.add(INSERT_COUNT, counts[0])); + registry.ifPresent(r -> r.add(DELETE_COUNT, counts[2])); + return writeStatusRDD; + } + + @Override + public boolean rollbackCommit(String instantTime) { + // TODO: needs to be implemented + throw new UnsupportedOperationException(); + } + + @Override + public boolean isGlobal() { + return true; + } + + @Override + public boolean canIndexLogFiles() { + return false; + } + + @Override + public boolean isImplicitWithStorage() { + return false; + } + + /** + * Function that tags each HoodieRecord with an existing location, if known. + */ + class LocationTagFunction implements FlatMapFunction>, HoodieRecord> { + HoodieTable hoodieTable; + Option registry; + + public LocationTagFunction(HoodieTable>, JavaRDD, JavaRDD> hoodieTable, + Option registry2) { + this.hoodieTable = hoodieTable; + this.registry = registry2; + } + + @Override + public Iterator> call(Iterator> hoodieRecordIterator) { + HoodieTimer timer = new HoodieTimer().startTimer(); + List> taggedRecords = new ArrayList<>(); + Map keyToIndexMap = new HashMap<>(); + while (hoodieRecordIterator.hasNext()) { + HoodieRecord rec = hoodieRecordIterator.next(); + ValidationUtils.checkState(!rec.getRecordKey().contains("Option")); + keyToIndexMap.put(rec.getRecordKey(), taggedRecords.size()); + taggedRecords.add(rec); + } + + Set recordKeys = keyToIndexMap.keySet().stream().sorted().collect(Collectors.toSet()); + Map recordIndexInfo = hoodieTable.getMetadataReader().readRecordLevelIndex(recordKeys); + + for (Entry e : recordIndexInfo.entrySet()) { + HoodieRecord rec = taggedRecords.get(keyToIndexMap.get(e.getKey())); + rec.unseal(); + rec.setCurrentLocation(e.getValue()); + rec.seal(); + } + + registry.ifPresent(r -> r.add(TAG_LOCATION_DURATION, timer.endTimer())); + registry.ifPresent(r -> r.add(TAG_LOCATION_COUNT, recordKeys.size())); + registry.ifPresent(r -> r.add(TAG_LOCATION_HITS, recordIndexInfo.size())); + return taggedRecords.iterator(); + } + } +} + + diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index c014e8be57976..d6567d550587a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -21,40 +21,46 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.TableFileSystemView; 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.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.storage.StorageLevel; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; -public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter { +public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter> { private static final Logger LOG = LogManager.getLogger(SparkHoodieBackedTableMetadataWriter.class); + private List> recordsQueuedForCommit; public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig, HoodieEngineContext context) { return new SparkHoodieBackedTableMetadataWriter(conf, writeConfig, context); @@ -80,7 +86,7 @@ protected void initRegistry() { } @Override - protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) { + protected void initialize(HoodieEngineContext engineContext) { try { metrics.map(HoodieMetadataMetrics::registry).ifPresent(registry -> { if (registry instanceof DistributedRegistry) { @@ -99,39 +105,47 @@ protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClie } @Override - protected void commit(List records, String partitionName, String instantTime) { + protected void commit(String instantTime, boolean isInsert) { ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); - JavaRDD recordRDD = prepRecords(records, partitionName); + ValidationUtils.checkState(recordsQueuedForCommit != null); + + JavaSparkContext jsc = ((HoodieSparkEngineContext) engineContext).getJavaSparkContext(); + JavaRDD recordRDD = recordsQueuedForCommit.isEmpty() ? jsc.emptyRDD() : recordsQueuedForCommit.get(0); + for (int index = 1; index < recordsQueuedForCommit.size(); ++index) { + recordRDD = recordRDD.union(recordsQueuedForCommit.get(index)); + } + + // Ensure each record is tagged + recordRDD.foreach(r -> { + if (r.getCurrentLocation() == null) { + throw new HoodieMetadataException("Record is not tagged with a location"); + } + }); + recordsQueuedForCommit.clear(); try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { writeClient.startCommitWithTime(instantTime); - List statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect(); + List statuses; + if (isInsert) { + statuses = writeClient.insertPreppedRecords(recordRDD, instantTime).collect(); + } else { + statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect(); + } statuses.forEach(writeStatus -> { if (writeStatus.hasErrors()) { throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime); } }); - // trigger cleaning, compaction, with suffixes based on the same instant time. This ensures that any future - // delta commits synced over will not have an instant time lesser than the last completed instant on the - // metadata table. - if (writeClient.scheduleCompactionAtInstant(instantTime + "001", Option.empty())) { - writeClient.compact(instantTime + "001"); - } - writeClient.clean(instantTime + "002"); + + // reload timeline + metaClient.reloadActiveTimeline(); + + compactIfNecessary(writeClient, instantTime); + cleanIfNecessary(writeClient, instantTime); } // Update total size of the metadata and count of base/log files - metrics.ifPresent(m -> { - try { - Map stats = m.getStats(false, metaClient, metadata); - m.updateMetrics(Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)), - Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE)), - Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)), - Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES))); - } catch (HoodieIOException e) { - LOG.error("Could not publish metadata size metrics", e); - } - }); + metrics.ifPresent(m -> m.updateSizeMetrics(metaClient, metadata)); } /** @@ -152,49 +166,110 @@ public Option getLatestSyncedInstantTime() { } /** - * Tag each record with the location. + * Perform a compaction on the Metadata Table. + * + * Cases to be handled: + * 1. We cannot perform compaction if there are previous inflight operations on the dataset. This is because + * a compacted metadata base file at time Tx should represent all the actions on the dataset till time Tx. * - * Since we only read the latest base file in a partition, we tag the records with the instant time of the latest - * base file. + * 2. In multi-writer scenario, a parallel operation with a greater instantTime may have completed creating a + * deltacommit. */ - private JavaRDD prepRecords(List records, String partitionName) { - HoodieTable table = HoodieSparkTable.create(metadataWriteConfig, engineContext); - TableFileSystemView.SliceView fsView = table.getSliceView(); - List baseFiles = fsView.getLatestFileSlices(partitionName) - .map(FileSlice::getBaseFile) - .filter(Option::isPresent) - .map(Option::get) - .collect(Collectors.toList()); - - // All the metadata fits within a single base file - if (partitionName.equals(MetadataPartitionType.FILES.partitionPath())) { - if (baseFiles.size() > 1) { - throw new HoodieMetadataException("Multiple base files found in metadata partition"); - } + private void compactIfNecessary(SparkRDDWriteClient writeClient, String instantTime) { + String latestDeltacommitTime = metaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant() + .get().getTimestamp(); + List pendingInstants = datasetMetaClient.reloadActiveTimeline().filterInflightsAndRequested() + .findInstantsBefore(latestDeltacommitTime).getInstants().collect(Collectors.toList()); + + if (!pendingInstants.isEmpty()) { + LOG.info(String.format("Cannot compact metadata table as there are %d inflight instants before latest deltacommit %s: %s", + pendingInstants.size(), latestDeltacommitTime, Arrays.toString(pendingInstants.toArray()))); + return; } + // Trigger compaction with suffixes based on the same instant time. This ensures that any future + // delta commits synced over will not have an instant time lesser than the last completed instant on the + // metadata table. + final String compactionInstantTime = latestDeltacommitTime + "001"; + if (writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty())) { + writeClient.compact(compactionInstantTime); + } + } + + private void cleanIfNecessary(SparkRDDWriteClient writeClient, String instantTime) { + // Trigger cleaning with suffixes based on the same instant time. This ensures that any future + // delta commits synced over will not have an instant time lesser than the last completed instant on the + // metadata table. + writeClient.clean(instantTime + "002"); + } + + @Override + protected TableFileSystemView.BaseFileOnlyView getTableFileSystemView() { + HoodieTable table = HoodieSparkTable.create(datasetWriteConfig, engineContext); + return (TableFileSystemView.BaseFileOnlyView)table.getFileSystemView(); + } + + @Override + protected Pair, Long> readRecordKeysFromBaseFiles(HoodieEngineContext engineContext, + List> partitionBaseFilePairs) { JavaSparkContext jsc = ((HoodieSparkEngineContext) engineContext).getJavaSparkContext(); - String fileId; - String instantTime; - if (!baseFiles.isEmpty()) { - fileId = baseFiles.get(0).getFileId(); - instantTime = baseFiles.get(0).getCommitTime(); - } else { - // If there is a log file then we can assume that it has the data - List logFiles = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()) - .map(FileSlice::getLatestLogFile) - .filter(Option::isPresent) - .map(Option::get) - .collect(Collectors.toList()); - if (logFiles.isEmpty()) { - // No base and log files. All are new inserts - return jsc.parallelize(records, 1); + final String datasetBasePath = datasetWriteConfig.getBasePath(); + final SerializableConfiguration conf = new SerializableConfiguration(hadoopConf.get()); + JavaRDD recordRDD = jsc.parallelize(partitionBaseFilePairs, partitionBaseFilePairs.size()).flatMap(p -> { + final String partition = p.getKey(); + final String filename = p.getValue(); + Path dataFilePath = new Path(datasetBasePath, partition + Path.SEPARATOR + filename); + + final String fileId = FSUtils.getFileId(filename); + final String instantTime = FSUtils.getCommitTime(filename); + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(conf.get(), dataFilePath); + Iterator recordKeyIterator = reader.getRecordKeyIterator(); + + return new Iterator() { + @Override + public boolean hasNext() { + return recordKeyIterator.hasNext(); + } + + @Override + public HoodieRecord next() { + return HoodieMetadataPayload.createRecordLevelIndexRecord(recordKeyIterator.next(), partition, fileId, instantTime); + } + }; + }); + + if (recordRDD.getStorageLevel() == StorageLevel.NONE()) { + recordRDD.persist(StorageLevel.MEMORY_AND_DISK()); + } + + return Pair.of(recordRDD, recordRDD.count()); + } + + @Override + public void queueForUpdate(JavaRDD records, MetadataPartitionType partitionType, String instantTime) { + List shards = HoodieTableMetadataUtil.loadPartitionShards(metaClient, partitionType.partitionPath()); + + JavaRDD taggedRecordRDD = records.map(r -> { + int shardIndex = Math.abs(HoodieTableMetadataUtil.keyToShard(r.getRecordKey(), shards.size())); + HoodieRecordLocation loc = new HoodieRecordLocation(shards.get(shardIndex).getBaseInstantTime(), shards.get(shardIndex).getFileId()); + HoodieRecord taggedRecord = HoodieIndexUtils.getTaggedRecord(r, Option.of(loc)); + if (taggedRecord.getCurrentLocation() == null) { + throw new HoodieMetadataException("Tagged record does not have a location set"); } - fileId = logFiles.get(0).getFileId(); - instantTime = logFiles.get(0).getBaseCommitTime(); + return taggedRecord; + }); + + if (recordsQueuedForCommit == null) { + recordsQueuedForCommit = new ArrayList<>(); } - return jsc.parallelize(records, 1).map(r -> r.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId))); + recordsQueuedForCommit.add(taggedRecordRDD); + } + + @Override + protected void queueForUpdate(List records, MetadataPartitionType partitionType, String instantTime) { + JavaSparkContext jsc = ((HoodieSparkEngineContext) engineContext).getJavaSparkContext(); + queueForUpdate(jsc.parallelize(records, 1), partitionType, instantTime); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index 70a57b79e0f43..334f71a2e376e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -29,9 +30,12 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndex; - +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.spark.api.java.JavaRDD; public abstract class HoodieSparkTable @@ -66,4 +70,20 @@ public static HoodieSparkTable create(HoodieW protected HoodieIndex>, JavaRDD, JavaRDD> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { return SparkHoodieIndex.createIndex(config); } + + @Override + public Option getMetadataWriter() { + if (!config.useFileListingMetadata()) { + return Option.empty(); + } + + try { + if (!metaClient.getFs().exists(new Path(HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath())))) { + return Option.empty(); + } + return Option.of(SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context)); + } catch (Exception e) { + throw new HoodieMetadataException("Could not create metadata table writer", e); + } + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 821b3071e145c..56cc0d8a3877c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -59,13 +59,10 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieKeyGeneratorException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; import org.apache.hudi.io.HoodieBootstrapHandle; import org.apache.hudi.keygen.KeyGeneratorInterface; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; -import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -226,17 +223,6 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta LOG.info("Committing metadata bootstrap !!"); } - @Override - protected void syncTableMetadata() { - // Open up the metadata table again, for syncing - try (HoodieTableMetadataWriter writer = - SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) { - LOG.info("Successfully synced to metadata table"); - } catch (Exception e) { - throw new HoodieMetadataException("Error syncing to metadata table.", e); - } - } - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List stats) { String actionType = table.getMetaClient().getCommitActionType(); LOG.info("Committing " + instantTime + ", action Type " + actionType); @@ -252,7 +238,6 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta // Finalize write finalizeWrite(instantTime, stats, result); - syncTableMetadata(); // add in extra metadata if (extraMetadata.isPresent()) { extraMetadata.get().forEach(metadata::addMetadata); @@ -260,6 +245,8 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit()); metadata.setOperationType(operationType); + syncTableMetadata(metadata); + try { activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index 2bc1f0302798e..b4d3cd69a5457 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -40,7 +40,6 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.execution.SparkLazyInsertIterable; import org.apache.hudi.io.CreateHandleFactory; @@ -49,8 +48,6 @@ import org.apache.hudi.io.storage.HoodieConcatHandle; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; -import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; @@ -239,7 +236,7 @@ protected JavaRDD updateIndex(JavaRDD writeStatusRDD, result.setWriteStatuses(statuses); return statuses; } - + protected void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { updateIndex(writeStatusRDD, result); result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result)); @@ -264,13 +261,13 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta result.setWriteStats(writeStats); // Finalize write finalizeWrite(instantTime, writeStats, result); - syncTableMetadata(); try { - LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + syncTableMetadata(metadata); + activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); @@ -354,17 +351,6 @@ protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, } } - @Override - public void syncTableMetadata() { - // Open up the metadata table again, for syncing - try (HoodieTableMetadataWriter writer = - SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) { - LOG.info("Successfully synced to metadata table"); - } catch (Exception e) { - throw new HoodieMetadataException("Error syncing to metadata table.", e); - } - } - @Override public Iterator> handleInsert(String idPfx, Iterator> recordItr) throws Exception { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java new file mode 100644 index 0000000000000..36b32a639b679 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java @@ -0,0 +1,39 @@ +/* + * 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.table.upgrade; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; + +/** + * Upgrade handle to assist in upgrading hoodie table from version 1 to 2. + */ +public class OneToTwoUpgradeHandler implements UpgradeHandler { + + @Override + public void upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) { + if (config.useFileListingMetadata()) { + // Metadata Table in version 1 is asynchronous and in version 2 is synchronous. Synchronous table will not + // sync any instants not already synced. So its simpler to re-bootstrap the table. Also, the schema for the + // table has been updated and is not backward compatible. + HoodieTableMetadataWriter.removeMetadataTable(config.getBasePath(), context); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java index accdee221ff73..f64687b1668a9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java @@ -48,7 +48,9 @@ public void run(HoodieTableMetaClient metaClient, @Override protected void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { - if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) { + if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.TWO) { + new OneToTwoUpgradeHandler().upgrade(config, context, instantTime); + } else if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) { new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime); } else { throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true); @@ -57,7 +59,9 @@ protected void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVers @Override protected void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { - if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) { + if (fromVersion == HoodieTableVersion.TWO && toVersion == HoodieTableVersion.ONE) { + new TwoToOneDowngradeHandler().downgrade(config, context, instantTime); + } else if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) { new OneToZeroDowngradeHandler().downgrade(config, context, instantTime); } else { throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java new file mode 100644 index 0000000000000..69b5285b945e7 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java @@ -0,0 +1,39 @@ +/* + * 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.table.upgrade; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; + +/** + * Downgrade handle to assist in downgrading hoodie table from version 2 to 1. + */ +public class TwoToOneDowngradeHandler implements DowngradeHandler { + + @Override + public void downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) { + if (config.useFileListingMetadata()) { + // Metadata Table in version 2 is synchronous and in version 1 is asynchronous. Downgrading to synchronous + // removes the checks in code to decide whether to use a LogBlock or not. Also, the schema for the + // table has been updated and is not forward compatible. Hence, we need to delete the table. + HoodieTableMetadataWriter.removeMetadataTable(config.getBasePath(), context); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 513bac14b2efd..0cd9862138fa7 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -18,10 +18,37 @@ package org.apache.hudi.client.functional; +import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.HoodieWriteResult; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.fs.FSUtils; @@ -33,21 +60,28 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieMetricsConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metadata.FileSystemBackedTableMetadata; @@ -58,15 +92,13 @@ import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade; import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -74,22 +106,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; - @Tag("functional") public class TestHoodieBackedMetadata extends HoodieClientTestHarness { @@ -140,26 +156,66 @@ public void testMetadataTableBootstrap() throws Exception { assertThrows(TableNotFoundException.class, () -> HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build()); } + HoodieTableMetaClient metadataTableMetaClient; + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + + // Create an inflight commit on the table + String inflightCommitTime = HoodieActiveTimeline.createNewInstantTime(); + testTable.addInflightCommit(inflightCommitTime); + // Metadata table should not be created if any non-complete instants are present String secondCommitTime = HoodieActiveTimeline.createNewInstantTime(); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(false, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(true, true, false, HoodieFailedWritesCleaningPolicy.LAZY).build())) { client.startCommitWithTime(secondCommitTime); - client.insert(jsc.parallelize(dataGen.generateUpdates(secondCommitTime, 2)), secondCommitTime); - // AutoCommit is false so no bootstrap - client.syncTableMetadata(); + client.insert(jsc.emptyRDD(), secondCommitTime); + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not be created"); assertThrows(TableNotFoundException.class, () -> HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build()); - // rollback this commit - client.rollback(secondCommitTime); } - // Metadata table created when enabled by config & sync is called - secondCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Rollback the inflight commit so that the metadata table can finally be created + testTable.removeCommit(inflightCommitTime); + + // Metadata table created when enabled by config + String thirdCommitTime = HoodieActiveTimeline.createNewInstantTime(); + String fourthCommitTime = HoodieActiveTimeline.createNewInstantTime(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + client.startCommitWithTime(thirdCommitTime); + client.insert(jsc.parallelize(dataGen.generateUpdates(thirdCommitTime, 2)), thirdCommitTime); + + metadataTableMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); + assertTrue(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not be created"); + assertTrue(metadataTableMetaClient.getActiveTimeline().countInstants() == 2); // bootstrap and thirdCommit + + client.startCommitWithTime(fourthCommitTime); + client.insert(jsc.emptyRDD(), fourthCommitTime); + assertTrue(metadataTableMetaClient.getActiveTimeline().countInstants() == 2); // bootstrap, thirdCommit, fourthCommit + + validateMetadata(client); + } + + /** + + // Delete the thirdCommitTime and fourthCommitTime instants and introduce a new commit. This should trigger a rebootstrap + // of the metadata table as un-synched instants have been "archived". + testTable.removeCommit(inflightCommitTime); + final String metadataTableMetaPath = metadataTableBasePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME; + assertTrue(fs.exists(new Path(metadataTableMetaPath, HoodieTimeline.makeDeltaFileName(thirdCommitTime)))); + assertTrue(fs.exists(new Path(metadataTableMetaPath, HoodieTimeline.makeDeltaFileName(fourthCommitTime)))); + LOG.info("=--------------------------------------------- " + new Path(metaClient.getMetaPath(), "{" + thirdCommitTime + "," + fourthCommitTime + "}.*")); + Arrays.stream(fs.globStatus(new Path(metaClient.getMetaPath(), "{" + firstCommitTime + "," + secondCommitTime + "," + thirdCommitTime + "," + fourthCommitTime + "}.*"))).forEach(s -> { + LOG.info("----------------------------------------------------------------------------- " + s); + try { + fs.delete(s.getPath(), false); + } catch (IOException e) { + LOG.warn("Error when deleting instant " + s + ": " + e); + } + }); + + String fifthCommitTime = HoodieActiveTimeline.createNewInstantTime(); try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { - client.startCommitWithTime(secondCommitTime); - client.insert(jsc.parallelize(dataGen.generateUpdates(secondCommitTime, 2)), secondCommitTime); - client.syncTableMetadata(); - assertTrue(fs.exists(new Path(metadataTableBasePath))); + client.startCommitWithTime(fifthCommitTime); + client.insert(jsc.emptyRDD(), fifthCommitTime); validateMetadata(client); } @@ -168,7 +224,7 @@ public void testMetadataTableBootstrap() throws Exception { final String metadataTableMetaPath = metadataTableBasePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME; assertTrue(fs.exists(new Path(metadataTableMetaPath, HoodieTimeline.makeDeltaFileName(secondCommitTime)))); - Arrays.stream(fs.listStatus(new Path(metaClient.getMetaPath()))).filter(status -> status.getPath().getName().matches("^\\d+\\..*")) + Arrays.stream(fs.listStatus(new Path(metadataTableMetaClient.getMetaPath()))).filter(status -> status.getPath().getName().matches("^\\d+\\..*")) .forEach(status -> { try { fs.delete(status.getPath(), false); @@ -179,9 +235,8 @@ public void testMetadataTableBootstrap() throws Exception { String thirdCommitTime = HoodieActiveTimeline.createNewInstantTime(); try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { - client.startCommitWithTime(thirdCommitTime); - client.insert(jsc.parallelize(dataGen.generateUpdates(thirdCommitTime, 2)), thirdCommitTime); - client.syncTableMetadata(); + client.startCommitWithTime(sixthCommitTime); + client.insert(jsc.parallelize(dataGen.generateUpdates(sixthCommitTime, 2)), sixthCommitTime); assertTrue(fs.exists(new Path(metadataTableBasePath))); validateMetadata(client); @@ -190,6 +245,7 @@ public void testMetadataTableBootstrap() throws Exception { assertFalse(fs.exists(new Path(metadataTableMetaPath, HoodieTimeline.makeDeltaFileName(secondCommitTime)))); assertTrue(fs.exists(new Path(metadataTableMetaPath, HoodieTimeline.makeDeltaFileName(thirdCommitTime)))); } + */ } /** @@ -218,7 +274,7 @@ public void testOnlyValidPartitionsAdded() throws Exception { .addCommit("002").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10, 10); final HoodieWriteConfig writeConfig = - getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.NEVER, true, true, false) + getWriteConfigBuilder(true, true, false, HoodieFailedWritesCleaningPolicy.NEVER) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).withDirectoryFilterRegex(filterDirRegex).build()).build(); try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { client.startCommitWithTime("005"); @@ -337,6 +393,64 @@ public void testTableOperations(HoodieTableType tableType) throws Exception { } } + /** + * Test multi-writer on metadata table with optimistic concurrency. + */ + @Test + public void testMetadataMultiWriter() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); + HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class).build()) + .withProperties(properties) + .build(); + + ExecutorService executors = Executors.newFixedThreadPool(dataGen.getPartitionPaths().length); + // Create clients in advance + SparkRDDWriteClient[] writeClients = new SparkRDDWriteClient[dataGen.getPartitionPaths().length]; + for (int i = 0; i < dataGen.getPartitionPaths().length; ++i) { + writeClients[i] = new SparkRDDWriteClient(engineContext, writeConfig); + } + + // Parallel commits for separate partitions + List futures = new LinkedList<>(); + for (int i = 0; i < dataGen.getPartitionPaths().length; ++i) { + final int index = i; + String newCommitTime = "00" + (index + 1); + Future future = executors.submit(() -> { + List records = dataGen.generateInsertsForPartition(newCommitTime, 100, dataGen.getPartitionPaths()[index]); + writeClients[index].startCommitWithTime(newCommitTime); + List writeStatuses = writeClients[index].insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + }); + futures.add(future); + } + + // Wait for all commits to complete + for (Future future : futures) { + future.get(); + } + + // Ensure all commits were synced to the Metadata Table + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); + assertEquals(metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().countInstants(), 4); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "001"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "002"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "003"))); + + // Compaction may occur if the commits completed in order + assertTrue(metadataMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().countInstants() <= 1); + + // Validation + validateMetadata(writeClients[0]); + } + /** * Test rollback of various table operations sync to Metadata Table correctly. */ @@ -363,7 +477,6 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { assertNoWriteErrors(writeStatuses); validateMetadata(client); client.rollback(newCommitTime); - client.syncTableMetadata(); validateMetadata(client); // Write 3 (updates) + Rollback of updates @@ -374,7 +487,6 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { assertNoWriteErrors(writeStatuses); validateMetadata(client); client.rollback(newCommitTime); - client.syncTableMetadata(); validateMetadata(client); // Rollback of updates and inserts @@ -385,7 +497,6 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { assertNoWriteErrors(writeStatuses); validateMetadata(client); client.rollback(newCommitTime); - client.syncTableMetadata(); validateMetadata(client); // Rollback of Compaction @@ -405,7 +516,6 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { assertNoWriteErrors(writeStatuses); validateMetadata(client); client.rollback(newCommitTime); - client.syncTableMetadata(); validateMetadata(client); // Rollback of Clean @@ -413,7 +523,6 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { client.clean(newCommitTime); validateMetadata(client); client.rollback(newCommitTime); - client.syncTableMetadata(); validateMetadata(client); } @@ -427,7 +536,6 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); client.rollback(newCommitTime); - client.syncTableMetadata(); validateMetadata(client); } @@ -441,44 +549,78 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); client.rollback(newCommitTime); - client.syncTableMetadata(); validateMetadata(client); } } /** - * Test when syncing rollback to metadata if the commit being rolled back has not been synced that essentially a no-op occurs to metadata. - * Once explicit sync is called, metadata should match. + * Test that manual rollbacks work correctly and enough timeline history is maintained on the metadata table + * timeline. */ @ParameterizedTest @EnumSource(HoodieTableType.class) - public void testRollbackUnsyncedCommit(HoodieTableType tableType) throws Exception { + @Disabled + public void testManualRollbacks(HoodieTableType tableType) throws Exception { init(tableType); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Setting to archive more aggressively on the Metadata Table than the Dataset + final int maxDeltaCommitsBeforeCompaction = 4; + final int minArchiveCommitsMetadata = 2; + final int minArchiveCommitsDataset = 4; + HoodieWriteConfig config = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) + .archiveCommitsWith(minArchiveCommitsMetadata, minArchiveCommitsMetadata + 1).retainCommits(1) + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(minArchiveCommitsDataset, minArchiveCommitsDataset + 1) + .retainCommits(1).retainFileVersions(1).withAutoClean(false).withAsyncClean(true).build()) + .build(); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, config)) { // Initialize table with metadata String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); List records = dataGen.generateInserts(newCommitTime, 20); client.startCommitWithTime(newCommitTime); List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); + + // Perform multiple commits + for (int i = 1; i < 10; ++i) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + if (i == 1) { + records = dataGen.generateInserts(newCommitTime, 5); + } else { + records = dataGen.generateUpdates(newCommitTime, 2); + } + client.startCommitWithTime(newCommitTime); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + } validateMetadata(client); - } - String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { - // Commit with metadata disabled - client.startCommitWithTime(newCommitTime); - List records = dataGen.generateUpdates(newCommitTime, 10); - List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - assertNoWriteErrors(writeStatuses); - client.rollback(newCommitTime); - } - try (SparkRDDWriteClient client = new SparkRDDWriteClient<>(engineContext, getWriteConfig(true, true))) { - assertFalse(metadata(client).isInSync()); - client.syncTableMetadata(); + // We can only rollback those commits whose deltacommit have not been archived yet. + int numRollbacks = 0; + boolean exceptionRaised = false; + + metaClient.reloadActiveTimeline(); + List allInstants = metaClient.getCommitsAndCompactionTimeline().getReverseOrderedInstants() + .collect(Collectors.toList()); + for (HoodieInstant instantToRollback : allInstants) { + try { + client.rollback(instantToRollback.getTimestamp()); + ++numRollbacks; + } catch (HoodieMetadataException e) { + exceptionRaised = true; + break; + } + } validateMetadata(client); + + assertTrue(exceptionRaised, "Rollback of archived instants should fail"); + // Since each rollback also creates a deltacommit, we can only support rolling back of half of the original + // instants present before rollback started. + assertTrue(numRollbacks >= Math.max(minArchiveCommitsDataset, minArchiveCommitsMetadata) / 2, + "Rollbacks of non archived instants should work"); } } @@ -521,7 +663,6 @@ public void testSync(HoodieTableType tableType) throws Exception { assertNoWriteErrors(writeStatuses); validateMetadata(client); - assertTrue(metadata(client).isInSync()); } // Various table operations without metadata table enabled @@ -535,7 +676,6 @@ public void testSync(HoodieTableType tableType) throws Exception { records = dataGen.generateUniqueUpdates(newCommitTime, 5); writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); - assertTrue(metadata(client).isInSync()); // updates and inserts newCommitTime = HoodieActiveTimeline.createNewInstantTime(); @@ -543,21 +683,18 @@ public void testSync(HoodieTableType tableType) throws Exception { records = dataGen.generateUpdates(newCommitTime, 10); writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); - assertTrue(metadata(client).isInSync()); // Compaction if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); client.compact(newCommitTime); - assertTrue(metadata(client).isInSync()); } // Savepoint restoreToInstant = newCommitTime; if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) { client.savepoint("hoodie", "metadata test"); - assertTrue(metadata(client).isInSync()); } // Record a timestamp for creating an inflight instance for sync testing @@ -570,12 +707,10 @@ public void testSync(HoodieTableType tableType) throws Exception { JavaRDD deleteKeys = jsc.parallelize(records, 1).map(r -> r.getKey()); client.startCommitWithTime(newCommitTime); client.delete(deleteKeys, newCommitTime); - assertTrue(metadata(client).isInSync()); // Clean newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.clean(newCommitTime); - assertTrue(metadata(client).isInSync()); // updates newCommitTime = HoodieActiveTimeline.createNewInstantTime(); @@ -583,7 +718,6 @@ public void testSync(HoodieTableType tableType) throws Exception { records = dataGen.generateUniqueUpdates(newCommitTime, 10); writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); - assertTrue(metadata(client).isInSync()); // insert overwrite to test replacecommit newCommitTime = HoodieActiveTimeline.createNewInstantTime(); @@ -592,7 +726,6 @@ public void testSync(HoodieTableType tableType) throws Exception { HoodieWriteResult replaceResult = client.insertOverwrite(jsc.parallelize(records, 1), newCommitTime); writeStatuses = replaceResult.getWriteStatuses().collect(); assertNoWriteErrors(writeStatuses); - assertTrue(metadata(client).isInSync()); } // If there is an incomplete operation, the Metadata Table is not updated beyond that operations but the @@ -602,7 +735,6 @@ public void testSync(HoodieTableType tableType) throws Exception { try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { // Restore cannot be done until the metadata table is in sync. See HUDI-1502 for details - client.syncTableMetadata(); // Table should sync only before the inflightActionTimestamp HoodieBackedTableMetadataWriter writer = @@ -612,11 +744,10 @@ public void testSync(HoodieTableType tableType) throws Exception { // Reader should sync to all the completed instants HoodieTableMetadata metadata = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(), client.getConfig().getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue()); - assertEquals(metadata.getSyncedInstantTime().get(), newCommitTime); + // TODO assertEquals(metadata.getSyncedInstantTimeForReader().get(), newCommitTime); // Remove the inflight instance holding back table sync fs.delete(inflightCleanPath, false); - client.syncTableMetadata(); writer = (HoodieBackedTableMetadataWriter)SparkHoodieBackedTableMetadataWriter.create(hadoopConf, client.getConfig(), context); @@ -631,62 +762,228 @@ public void testSync(HoodieTableType tableType) throws Exception { // Enable metadata table and ensure it is synced try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { client.restoreToInstant(restoreToInstant); - assertFalse(metadata(client).isInSync()); newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.startCommitWithTime(newCommitTime); - client.syncTableMetadata(); validateMetadata(client); - assertTrue(metadata(client).isInSync()); } } /** - * Instants on Metadata Table should be archived as per config. Metadata Table should be automatically compacted as per config. + * Ensure that the reader only reads completed instants. + * @throws IOException + */ + @Test + public void testReader() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + List records; + List writeStatuses; + String[] commitTimestamps = {HoodieActiveTimeline.createNewInstantTime(), HoodieActiveTimeline.createNewInstantTime(), + HoodieActiveTimeline.createNewInstantTime(), HoodieActiveTimeline.createNewInstantTime()}; + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + for (int i = 0; i < commitTimestamps.length; ++i) { + records = dataGen.generateInserts(commitTimestamps[i], 5); + client.startCommitWithTime(commitTimestamps[i]); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamps[i]).collect(); + assertNoWriteErrors(writeStatuses); + } + + // Ensure we can see files from each commit + Set timelineTimestamps = getAllFiles(metadata(client)).stream().map(p -> p.getName()).map(n -> FSUtils.getCommitTime(n)).collect(Collectors.toSet()); + assertEquals(timelineTimestamps.size(), commitTimestamps.length); + for (int i = 0; i < commitTimestamps.length; ++i) { + assertTrue(timelineTimestamps.contains(commitTimestamps[i])); + } + + // mark each commit as incomplete and ensure files are not seen + for (int i = 0; i < commitTimestamps.length; ++i) { + FileCreateUtils.deleteCommit(basePath, commitTimestamps[i]); + timelineTimestamps = getAllFiles(metadata(client)).stream().map(p -> p.getName()).map(n -> FSUtils.getCommitTime(n)).collect(Collectors.toSet()); + assertEquals(timelineTimestamps.size(), commitTimestamps.length - 1); + for (int j = 0; j < commitTimestamps.length; ++j) { + assertTrue(j == i || timelineTimestamps.contains(commitTimestamps[j])); + } + FileCreateUtils.createCommit(basePath, commitTimestamps[i]); + } + + // Test multiple incomplete commits + FileCreateUtils.deleteCommit(basePath, commitTimestamps[0]); + FileCreateUtils.deleteCommit(basePath, commitTimestamps[2]); + timelineTimestamps = getAllFiles(metadata(client)).stream().map(p -> p.getName()).map(n -> FSUtils.getCommitTime(n)).collect(Collectors.toSet()); + assertEquals(timelineTimestamps.size(), commitTimestamps.length - 2); + for (int j = 0; j < commitTimestamps.length; ++j) { + assertTrue(j == 0 || j == 2 || timelineTimestamps.contains(commitTimestamps[j])); + } + + // Test no completed commits + for (int i = 0; i < commitTimestamps.length; ++i) { + FileCreateUtils.deleteCommit(basePath, commitTimestamps[i]); + } + timelineTimestamps = getAllFiles(metadata(client)).stream().map(p -> p.getName()).map(n -> FSUtils.getCommitTime(n)).collect(Collectors.toSet()); + assertEquals(timelineTimestamps.size(), 0); + } + } + + /** + * Instants on Metadata Table should be archived as per config but we always keep atlest the number of instants + * as on the dataset. + * + * Metadata Table should be automatically compacted as per config. */ @Test public void testCleaningArchivingAndCompaction() throws Exception { init(HoodieTableType.COPY_ON_WRITE); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - final int maxDeltaCommitsBeforeCompaction = 4; + final int maxDeltaCommitsBeforeCompaction = 3; HoodieWriteConfig config = getWriteConfigBuilder(true, true, false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) - .archiveCommitsWith(6, 8).retainCommits(1) + .archiveCommitsWith(40, 60).retainCommits(1) .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) - // don't archive the data timeline at all. - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(Integer.MAX_VALUE - 1, Integer.MAX_VALUE) - .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER) + .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(false).build()) .build(); List records; + String newCommitTime; try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, config)) { - for (int i = 1; i < 10; ++i) { - String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); - if (i == 1) { - records = dataGen.generateInserts(newCommitTime, 5); - } else { - records = dataGen.generateUpdates(newCommitTime, 2); - } + // Some initial commits so compaction is not triggered. + // 1 deltacommit will be from bootstrap. So we can perform maxDeltaCommitsBeforeCompaction - 2 more commits before + // compaction will be attempted. + for (int i = 0; i < maxDeltaCommitsBeforeCompaction - 2; ++i) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); client.startCommitWithTime(newCommitTime); - List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - assertNoWriteErrors(writeStatuses); - validateMetadata(client); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + } + + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); + HoodieTableMetaClient datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(config.getBasePath()).build(); + + // There should not be any compaction yet and we have not performed more than maxDeltaCommitsBeforeCompaction + // deltacommits (1 will be due to bootstrap) + HoodieActiveTimeline metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 0); + assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), maxDeltaCommitsBeforeCompaction - 1); + assertEquals(datasetMetaClient.getArchivedTimeline().reload().countInstants(), 0); + + // Next commit will initiate a compaction + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 1); + assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), maxDeltaCommitsBeforeCompaction + 1); + assertEquals(datasetMetaClient.getArchivedTimeline().reload().countInstants(), 0); + + // More than maxDeltaCommitsBeforeCompaction commits + String inflightCommitTime = newCommitTime; + for (int i = 0; i < maxDeltaCommitsBeforeCompaction + 1; ++i) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + if (i == 0) { + // Mark this commit inflight so compactions dont take place + FileCreateUtils.deleteCommit(basePath, newCommitTime); + FileCreateUtils.createInflightCommit(basePath, newCommitTime); + inflightCommitTime = newCommitTime; + } } + + // Ensure no more compactions took place due to the leftover inflight commit + metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 1); + assertEquals(metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(), 2 * maxDeltaCommitsBeforeCompaction + 1 /* clean */); + + // Complete commit + FileCreateUtils.createCommit(basePath, inflightCommitTime); + + // Next commit should lead to compaction + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + + // Ensure compactions took place + metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 2); + assertEquals(metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(), 2 * maxDeltaCommitsBeforeCompaction + 2 /* cleans */); + + assertTrue(datasetMetaClient.getArchivedTimeline().reload().countInstants() > 0); + + validateMetadata(client); } - HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); - HoodieTableMetaClient datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(config.getBasePath()).build(); - HoodieActiveTimeline metadataTimeline = metadataMetaClient.getActiveTimeline(); - // check that there are compactions. - assertTrue(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants() > 0); - // check that cleaning has, once after each compaction. - assertTrue(metadataTimeline.getCleanerTimeline().filterCompletedInstants().countInstants() > 0); + /* // ensure archiving has happened long numDataCompletedInstants = datasetMetaClient.getActiveTimeline().filterCompletedInstants().countInstants(); long numDeltaCommits = metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(); assertTrue(numDeltaCommits < numDataCompletedInstants, "Must have less delta commits than total completed instants on data timeline."); + */ + } + + @Test + public void testUpgradeDowngrade() throws IOException { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Perform a commit. This should bootstrap the metadata table with latest version. + List records; + List writeStatuses; + String commitTimestamp = HoodieActiveTimeline.createNewInstantTime(); + HoodieWriteConfig writeConfig = getWriteConfig(true, true); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { + records = dataGen.generateInserts(commitTimestamp, 5); + client.startCommitWithTime(commitTimestamp); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamp).collect(); + assertNoWriteErrors(writeStatuses); + } + + // Metadata table should have been bootstrapped + assertTrue(fs.exists(new Path(metadataTableBasePath)), "Metadata table should exist"); + FileStatus oldStatus = fs.getFileStatus(new Path(metadataTableBasePath)); + + // set hoodie.table.version to 1 in hoodie.properties file + changeTableVersion(HoodieTableVersion.ONE); + + // With next commit the table should be deleted (as part of upgrade) + commitTimestamp = HoodieActiveTimeline.createNewInstantTime(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + records = dataGen.generateInserts(commitTimestamp, 5); + client.startCommitWithTime(commitTimestamp); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamp).collect(); + assertNoWriteErrors(writeStatuses); + } + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not exist"); + + // With next commit the table should be re-bootstrapped (currently in the constructor. To be changed) + commitTimestamp = HoodieActiveTimeline.createNewInstantTime(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + records = dataGen.generateInserts(commitTimestamp, 5); + client.startCommitWithTime(commitTimestamp); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamp).collect(); + assertNoWriteErrors(writeStatuses); + } + + initMetaClient(); + assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.TWO.versionCode()); + assertTrue(fs.exists(new Path(metadataTableBasePath)), "Metadata table should exist"); + FileStatus newStatus = fs.getFileStatus(new Path(metadataTableBasePath)); + assertTrue(oldStatus.getModificationTime() < newStatus.getModificationTime()); + + // Test downgrade by running the downgrader + new SparkUpgradeDowngrade(metaClient, writeConfig, context).run(metaClient, HoodieTableVersion.ONE, writeConfig, context, null); + + assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.ONE.versionCode()); + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not exist"); } /** @@ -771,93 +1068,18 @@ public void testMetadataMetrics() throws Exception { assertNoWriteErrors(writeStatuses); validateMetadata(client); - Registry metricsRegistry = Registry.getRegistry("HoodieMetadata"); - assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".count")); - assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".totalDuration")); - assertTrue(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_STR + ".count") >= 1L); - assertTrue(metricsRegistry.getAllCounts().containsKey("basefile.size")); - assertTrue(metricsRegistry.getAllCounts().containsKey("logfile.size")); - assertTrue(metricsRegistry.getAllCounts().containsKey("basefile.count")); - assertTrue(metricsRegistry.getAllCounts().containsKey("logfile.count")); - } - } - - /** - * Test when reading from metadata table which is out of sync with dataset that results are still consistent. - */ - @Test - public void testMetadataOutOfSync() throws Exception { - init(HoodieTableType.COPY_ON_WRITE); - HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - - SparkRDDWriteClient unsyncedClient = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true)); - - // Enable metadata so table is initialized - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { - // Perform Bulk Insert - String newCommitTime = "001"; - client.startCommitWithTime(newCommitTime); - List records = dataGen.generateInserts(newCommitTime, 20); - client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); - } - - // Perform commit operations with metadata disabled - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { - // Perform Insert - String newCommitTime = "002"; - client.startCommitWithTime(newCommitTime); - List records = dataGen.generateInserts(newCommitTime, 20); - client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); - - // Perform Upsert - newCommitTime = "003"; - client.startCommitWithTime(newCommitTime); - records = dataGen.generateUniqueUpdates(newCommitTime, 20); - client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - - // Compaction - if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { - newCommitTime = "004"; - client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); - client.compact(newCommitTime); - } - } - - assertFalse(metadata(unsyncedClient).isInSync()); - validateMetadata(unsyncedClient); - - // Perform clean operation with metadata disabled - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { - // One more commit needed to trigger clean so upsert and compact - String newCommitTime = "005"; - client.startCommitWithTime(newCommitTime); - List records = dataGen.generateUpdates(newCommitTime, 20); - client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - - if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { - newCommitTime = "006"; - client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); - client.compact(newCommitTime); - } - - // Clean - newCommitTime = "007"; - client.clean(newCommitTime); - } - - assertFalse(metadata(unsyncedClient).isInSync()); - validateMetadata(unsyncedClient); - - // Perform restore with metadata disabled - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { - client.restoreToInstant("004"); + Registry metricsRegistry = Registry.getRegistry(client.getConfig().getTableName() + ".HoodieMetadata"); + assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_FILE_LISTING_STR + ".count")); + assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_FILE_LISTING_STR + ".totalDuration")); + assertTrue(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_FILE_LISTING_STR + ".count") >= 1L); + final String prefix = MetadataPartitionType.FILES.partitionPath() + "."; + assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)); + assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_LOG_FILES)); + assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)); + assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE)); } - - assertFalse(metadata(unsyncedClient).isInSync()); - validateMetadata(unsyncedClient); } - /** * Validate the metadata tables contents to ensure it matches what is on the file system. */ @@ -964,7 +1186,6 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException .sum(); assertEquals(metadataFilenames.size(), numFiles); } catch (IOException e) { - // TODO Auto-generated catch block e.printStackTrace(); assertTrue(false, "Exception should not be raised: " + e); } @@ -976,10 +1197,8 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException // Validate write config for metadata table HoodieWriteConfig metadataWriteConfig = metadataWriter.getWriteConfig(); assertFalse(metadataWriteConfig.useFileListingMetadata(), "No metadata table for metadata table"); - assertFalse(metadataWriteConfig.getFileListingMetadataVerify(), "No verify for metadata table"); // Metadata table should be in sync with the dataset - assertTrue(metadata(client).isInSync()); HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); // Metadata table is MOR @@ -993,8 +1212,8 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory // in the .hoodie folder. List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath), - false, false, false); - Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); + false, false); + assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); // Metadata table should automatically compact and clean // versions are +1 as autoclean / compaction happens end of commits @@ -1011,6 +1230,22 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException LOG.info("Validation time=" + timer.endTimer()); } + /** + * Returns the list of all files in the dataset by iterating over the metadata table. + * @throws IOException + * @throws IllegalArgumentException + */ + private List getAllFiles(HoodieTableMetadata metadata) throws Exception { + List allfiles = new LinkedList<>(); + for (String partition : metadata.getAllPartitionPaths()) { + for (FileStatus status : metadata.getAllFilesInPartition(new Path(basePath, partition))) { + allfiles.add(status.getPath()); + } + } + + return allfiles; + } + private HoodieBackedTableMetadataWriter metadataWriter(SparkRDDWriteClient client) { return (HoodieBackedTableMetadataWriter) SparkHoodieBackedTableMetadataWriter .create(hadoopConf, client.getConfig(), new HoodieSparkEngineContext(jsc)); @@ -1035,16 +1270,17 @@ private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileList } private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { - return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics); + return getWriteConfigBuilder(autoCommit, useFileListingMetadata, enableMetrics, HoodieFailedWritesCleaningPolicy.EAGER); } - private HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { + private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics, + HoodieFailedWritesCleaningPolicy failedWritesPolicy) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) .withAutoCommit(autoCommit) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) - .withFailedWritesCleaningPolicy(policy) + .withFailedWritesCleaningPolicy(failedWritesPolicy) .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build()) .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") @@ -1058,6 +1294,14 @@ private HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleani .withExecutorMetrics(true).usePrefix("unit-test").build()); } + private void changeTableVersion(HoodieTableVersion version) throws IOException { + metaClient.getTableConfig().setTableVersion(version); + Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE); + try (FSDataOutputStream os = metaClient.getFs().create(propertyFile)) { + metaClient.getTableConfig().getProps().store(os, ""); + } + } + @Override protected HoodieTableType getTableType() { return tableType; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 9c4059a519504..3f1442ef0cd91 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -84,8 +84,10 @@ private static Stream indexTypeParams() { {IndexType.GLOBAL_BLOOM, true}, {IndexType.SIMPLE, true}, {IndexType.GLOBAL_SIMPLE, true}, + {IndexType.RECORD_LEVEL, true}, {IndexType.SIMPLE, false}, - {IndexType.GLOBAL_SIMPLE, false} + {IndexType.GLOBAL_SIMPLE, false}, + {IndexType.RECORD_LEVEL, false} }; return Stream.of(data).map(Arguments::of); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java index 79c415a4bc268..608d9ca07aa8c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java @@ -52,7 +52,7 @@ public class TestAsyncCompaction extends CompactionTestBase { private HoodieWriteConfig getConfig(Boolean autoCommit) { return getConfigBuilder(autoCommit) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).validate(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) .build(); } diff --git a/hudi-common/src/main/avro/HoodieMetadata.avsc b/hudi-common/src/main/avro/HoodieMetadata.avsc index bf85587a3a7ac..5cd57d196a54f 100644 --- a/hudi-common/src/main/avro/HoodieMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieMetadata.avsc @@ -23,14 +23,16 @@ "fields": [ { "name": "key", - "type": "string" + "type": "string", + "avro.java.string": "String" }, { "name": "type", "doc": "Type of the metadata record", "type": "int" }, - { "name": "filesystemMetadata", + { + "name": "filesystemMetadata", "doc": "Contains information about partitions and files within the dataset", "type": ["null", { "type": "map", @@ -51,6 +53,44 @@ ] } }] + }, + { + "name": "recordIndexMetadata", + "doc": "Contains information about record keys and their location in the dataset", + "type": ["null", { + "type": "record", + "name": "HoodieRecordIndexInfo", + "fields": [ + { + "name": "partition", + "type": "string", + "doc": "Partition which contains the record", + "avro.java.string": "String" + }, + { + "name": "fileIdHighBits", + "type": "long", + "doc": "fileId which contains the record (high 64 bits)", + "avro.java.string": "String" + }, + { + "name": "fileIdLowBits", + "type": "long", + "doc": "fileId which contains the record (low 64 bits)", + "avro.java.string": "String" + }, + { + "name": "fileIndex", + "type": "int", + "doc": "index of the file" + }, + { + "name": "instantTime", + "type": "int", + "doc": "Timestamp at which record was added" + } + ] + }] } ] } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 11a75b6c4dfab..509eafa59141d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -19,6 +19,9 @@ package org.apache.hudi.common.config; import javax.annotation.concurrent.Immutable; + +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; + import java.io.File; import java.io.FileReader; import java.io.IOException; @@ -44,13 +47,6 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.7.0") .withDocumentation("Enable the internal metadata table which serves table metadata like level file listings"); - // Validate contents of Metadata Table on each access against the actual filesystem - public static final ConfigProperty METADATA_VALIDATE_PROP = ConfigProperty - .key(METADATA_PREFIX + ".validate") - .defaultValue(false) - .sinceVersion("0.7.0") - .withDocumentation("Validate contents of metadata table on each access; e.g against the actual listings from lake storage"); - public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false; // Enable metrics for internal Metadata Table @@ -121,6 +117,42 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.7.0") .withDocumentation("Parallelism to use, when listing the table on lake storage."); + public static final ConfigProperty RECORD_LEVEL_INDEX_ENABLE_PROP = ConfigProperty + .key(METADATA_PREFIX + ".record.index.enabled") + .defaultValue(false) + .sinceVersion("0.10.0") + .withDocumentation("Enable Record Level Index"); + + public static final ConfigProperty RECORD_LEVEL_INDEX_SHARD_COUNT_PROP = ConfigProperty + .key(METADATA_PREFIX + ".record.index.enabled") + .defaultValue(10) + .sinceVersion("0.10.0") + .withDocumentation("Number of shards to use for Record Level Index"); + + public static final ConfigProperty SPILLABLE_MAP_DIR_PROP = ConfigProperty + .key(METADATA_PREFIX + ".spillable.dir") + .defaultValue(FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue()) + .sinceVersion("0.10.0") + .withDocumentation("Directory where the spillable maps are saved"); + + public static final ConfigProperty MAX_CACHE_MEMORY_PROP = ConfigProperty + .key(METADATA_PREFIX + ".max.cache.memory") + .defaultValue(256 * 1024 * 1024L) + .sinceVersion("0.10.0") + .withDocumentation("Max memory to use to cache records read from metadata"); + + public static final ConfigProperty MAX_READER_MEMORY_PROP = ConfigProperty + .key(METADATA_PREFIX + ".max.reader.memory") + .defaultValue(1024 * 1024 * 1024L) + .sinceVersion("0.10.0") + .withDocumentation("Max memory to use for the reader to read from metadata"); + + public static final ConfigProperty MAX_READER_BUFFER_SIZE_PROP = ConfigProperty + .key(METADATA_PREFIX + ".max.reader.buffer.size") + .defaultValue(10 * 1024 * 1024) + .sinceVersion("0.10.0") + .withDocumentation("Max memory to use for the reader buffer while merging log blocks"); + private HoodieMetadataConfig() { super(); } @@ -141,10 +173,6 @@ public boolean useFileListingMetadata() { return getBoolean(METADATA_ENABLE_PROP); } - public boolean validateFileListingMetadata() { - return getBoolean(METADATA_VALIDATE_PROP); - } - public boolean enableMetrics() { return getBoolean(METADATA_METRICS_ENABLE_PROP); } @@ -153,6 +181,30 @@ public String getDirectoryFilterRegex() { return getString(DIRECTORY_FILTER_REGEX); } + public boolean isRecordLevelIndexEnabled() { + return useFileListingMetadata() && getBoolean(RECORD_LEVEL_INDEX_ENABLE_PROP); + } + + public int getRecordLevelIndexShardCount() { + return getInt(RECORD_LEVEL_INDEX_SHARD_COUNT_PROP); + } + + public String getSplliableMapDir() { + return getString(SPILLABLE_MAP_DIR_PROP); + } + + public long getMaxRecordCacheMemory() { + return getLong(MAX_CACHE_MEMORY_PROP); + } + + public long getMaxReaderMemory() { + return getLong(MAX_READER_MEMORY_PROP); + } + + public int getMaxReaderBufferSize() { + return getInt(MAX_READER_BUFFER_SIZE_PROP); + } + public static class Builder { private final HoodieMetadataConfig metadataConfig = new HoodieMetadataConfig(); @@ -179,11 +231,6 @@ public Builder enableMetrics(boolean enableMetrics) { return this; } - public Builder validate(boolean validate) { - metadataConfig.setValue(METADATA_VALIDATE_PROP, String.valueOf(validate)); - return this; - } - public Builder withInsertParallelism(int parallelism) { metadataConfig.setValue(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism)); return this; @@ -225,6 +272,36 @@ public Builder withDirectoryFilterRegex(String regex) { return this; } + public Builder withRecordLevelIndexEnabled(boolean enabled) { + metadataConfig.setValue(RECORD_LEVEL_INDEX_ENABLE_PROP, String.valueOf(enabled)); + return this; + } + + public Builder withSpillableMapDir(String dir) { + metadataConfig.setValue(SPILLABLE_MAP_DIR_PROP, dir); + return this; + } + + public Builder withMaxRecordCacheMemory(long mem) { + metadataConfig.setValue(MAX_CACHE_MEMORY_PROP, String.valueOf(mem)); + return this; + } + + public Builder withMaxReaderMemory(long mem) { + metadataConfig.setValue(MAX_READER_MEMORY_PROP, String.valueOf(mem)); + return this; + } + + public Builder withMaxReaderBufferSize(long mem) { + metadataConfig.setValue(MAX_READER_BUFFER_SIZE_PROP, String.valueOf(mem)); + return this; + } + + public Builder withRecordLevelIndexShardCount(int count) { + metadataConfig.setValue(RECORD_LEVEL_INDEX_SHARD_COUNT_PROP, String.valueOf(count)); + return this; + } + public HoodieMetadataConfig build() { metadataConfig.setDefaults(HoodieMetadataConfig.class.getName()); return metadataConfig; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index cb778e6826506..a7d31128db2b9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -253,11 +253,10 @@ public static void processFiles(FileSystem fs, String basePathStr, Function getAllPartitionPaths(HoodieEngineContext engineContext, String basePathStr, - boolean useFileListingFromMetadata, boolean verifyListings, + boolean useFileListingFromMetadata, boolean assumeDatePartitioning) { HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() .enable(useFileListingFromMetadata) - .validate(verifyListings) .withAssumeDatePartitioning(assumeDatePartitioning) .build(); try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java index eb2e200de6bde..847a29d5ed00a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java @@ -30,7 +30,9 @@ public enum HoodieTableVersion { // < 0.6.0 versions ZERO(0), // 0.6.0 onwards - ONE(1); + ONE(1), + // 0.9.0 onwards + TWO(2); private final int versionCode; @@ -43,7 +45,7 @@ public int versionCode() { } public static HoodieTableVersion current() { - return ONE; + return TWO; } static HoodieTableVersion versionFromCode(int versionCode) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index 4d51fbb855a29..5f945ac5a74e0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -174,50 +174,59 @@ public static class Builder extends AbstractHoodieLogRecordScanner.Builder { protected ExternalSpillableMap.DiskMapType diskMapType = HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue(); protected boolean isBitCaskDiskMapCompressionEnabled = HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue(); // incremental filtering - private Option instantRange = Option.empty(); + protected Option instantRange = Option.empty(); // auto scan default true private boolean autoScan = true; + @Override public Builder withFileSystem(FileSystem fs) { this.fs = fs; return this; } + @Override public Builder withBasePath(String basePath) { this.basePath = basePath; return this; } + @Override public Builder withLogFilePaths(List logFilePaths) { this.logFilePaths = logFilePaths; return this; } + @Override public Builder withReaderSchema(Schema schema) { this.readerSchema = schema; return this; } + @Override public Builder withLatestInstantTime(String latestInstantTime) { this.latestInstantTime = latestInstantTime; return this; } + @Override public Builder withReadBlocksLazily(boolean readBlocksLazily) { this.readBlocksLazily = readBlocksLazily; return this; } + @Override public Builder withReverseReader(boolean reverseReader) { this.reverseReader = reverseReader; return this; } + @Override public Builder withBufferSize(int bufferSize) { this.bufferSize = bufferSize; return this; } + @Override public Builder withInstantRange(Option instantRange) { this.instantRange = instantRange; return this; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index 6d2682a4ffa09..aaa7e5ac0cae4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieHFileReader; import org.apache.log4j.LogManager; @@ -119,8 +118,13 @@ protected byte[] serializeRecords() throws IOException { recordKey = record.get(keyField.pos()).toString(); } byte[] recordBytes = HoodieAvroUtils.indexedRecordToBytes(record); - ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey), - "Writing multiple records with same key not supported for " + this.getClass().getName()); + // TODO: Disabled this validation check for testing. We will have to handle the duplicates + //ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey), + // "Writing multiple records with same key not supported for " + this.getClass().getName()); + if (sortedRecordsMap.containsKey(recordKey)) { + continue; + } + sortedRecordsMap.put(recordKey, recordBytes); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index d4bf2dd209f5d..552adfa8f3490 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -229,7 +229,14 @@ public HoodieTimeline getCleanerTimeline() { */ public HoodieTimeline getRollbackTimeline() { return new HoodieDefaultTimeline(filterInstantsByAction(ROLLBACK_ACTION), - (Function> & Serializable) this::getInstantDetails); + (Function> & Serializable) this::getInstantDetails); + } + + /** + * Get only the rollback and restore action (inflight and completed) in the active timeline. + */ + public HoodieTimeline getRollbackAndRestoreTimeline() { + return getTimelineOfActions(CollectionUtils.createSet(ROLLBACK_ACTION, RESTORE_ACTION)); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java index fefe7eb7e5cc6..ef490ea60d80f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java @@ -37,6 +37,8 @@ public interface HoodieFileReader { public Iterator getRecordIterator(Schema readerSchema) throws IOException; + public Iterator getRecordKeyIterator() throws IOException; + default Iterator getRecordIterator() throws IOException { return getRecordIterator(getSchema()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index b954e57e77c7e..e0af6a2e04dfe 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -312,4 +312,24 @@ public void readFully(long position, byte[] buffer, int offset, int length) thro read(position, buffer, offset, length); } } + + @Override + public Iterator getRecordKeyIterator() throws IOException { + final HFileScanner scanner = reader.getScanner(false, false); + return new Iterator() { + @Override + public boolean hasNext() { + try { + return scanner.next(); + } catch (IOException e) { + throw new HoodieException("Error while scanning for keys", e); + } + } + + @Override + public String next() { + return scanner.getKeyString(); + } + }; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcReader.java index 319f8d7da1add..045fe778200a4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcReader.java @@ -88,4 +88,9 @@ public void close() { public long getTotalRecords() { return orcUtils.getRowCount(conf, path); } + + @Override + public Iterator getRecordKeyIterator() throws IOException { + return orcUtils.readRowKeys(conf, path).iterator(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java index 9ead1ac87ba50..b06641690331e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java @@ -23,12 +23,15 @@ import java.util.Set; 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.Path; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.ParquetReaderIterator; import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroReadSupport; @@ -38,6 +41,7 @@ public class HoodieParquetReader implements HoodieFileR private Path path; private Configuration conf; private final BaseFileUtils parquetUtils; + private Schema schema; public HoodieParquetReader(Configuration configuration, Path path) { this.conf = configuration; @@ -45,6 +49,7 @@ public HoodieParquetReader(Configuration configuration, Path path) { this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); } + @Override public String[] readMinMaxRecordKeys() { return parquetUtils.readMinMaxRecordKeys(conf, path); } @@ -62,13 +67,18 @@ public Set filterRowKeys(Set candidateRowKeys) { @Override public Iterator getRecordIterator(Schema schema) throws IOException { AvroReadSupport.setAvroReadSchema(conf, schema); + AvroReadSupport.setRequestedProjection(conf, schema); ParquetReader reader = AvroParquetReader.builder(path).withConf(conf).build(); return new ParquetReaderIterator(reader); } @Override public Schema getSchema() { - return parquetUtils.readAvroSchema(conf, path); + if (schema == null) { + schema = parquetUtils.readAvroSchema(conf, path); + } + + return schema; } @Override @@ -79,4 +89,21 @@ public void close() { public long getTotalRecords() { return parquetUtils.getRowCount(conf, path); } + + @Override + public Iterator getRecordKeyIterator() throws IOException { + Iterator recordIterator = getRecordIterator(HoodieAvroUtils.getRecordKeySchema()); + return new Iterator() { + @Override + public boolean hasNext() { + return recordIterator.hasNext(); + } + + @Override + public String next() { + Object obj = recordIterator.next(); + return ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + } + }; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index e408ad939a890..a4446c2f35b94 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -26,13 +26,12 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; 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.exception.HoodieMetadataException; import org.apache.hadoop.fs.FileStatus; @@ -41,33 +40,24 @@ import org.apache.log4j.Logger; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; +import java.util.Set; public abstract class BaseTableMetadata implements HoodieTableMetadata { private static final Logger LOG = LogManager.getLogger(BaseTableMetadata.class); - static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024; - static final int BUFFER_SIZE = 10 * 1024 * 1024; - protected final transient HoodieEngineContext engineContext; protected final SerializableConfiguration hadoopConf; protected final String datasetBasePath; protected final HoodieTableMetaClient datasetMetaClient; protected final Option metrics; protected final HoodieMetadataConfig metadataConfig; - // Directory used for Spillable Map when merging records - protected final String spillableMapDirectory; - private String syncedInstantTime; protected boolean enabled; - private TimelineMergedTableMetadata timelineMergedMetadata; protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, String spillableMapDirectory) { @@ -75,7 +65,6 @@ protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataCon this.hadoopConf = new SerializableConfiguration(engineContext.getHadoopConf()); this.datasetBasePath = datasetBasePath; this.datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(datasetBasePath).build(); - this.spillableMapDirectory = spillableMapDirectory; this.metadataConfig = metadataConfig; this.enabled = metadataConfig.useFileListingMetadata(); @@ -84,9 +73,6 @@ protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataCon } else { this.metrics = Option.empty(); } - if (enabled) { - openTimelineScanner(); - } } /** @@ -161,7 +147,7 @@ public Map getAllFilesInPartitions(List partitionP */ protected List fetchAllPartitionPaths() throws IOException { HoodieTimer timer = new HoodieTimer().startTimer(); - Option> hoodieRecord = getMergedRecordByKey(RECORDKEY_PARTITION_LIST); + Option> hoodieRecord = getRecordByKeyFromMetadata(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer())); List partitions = Collections.emptyList(); @@ -179,28 +165,6 @@ protected List fetchAllPartitionPaths() throws IOException { } } - if (metadataConfig.validateFileListingMetadata()) { - // Validate the Metadata Table data by listing the partitions from the file system - timer.startTimer(); - FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(getEngineContext(), - hadoopConf, datasetBasePath, metadataConfig.shouldAssumeDatePartitioning()); - List actualPartitions = fileSystemBackedTableMetadata.getAllPartitionPaths(); - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_PARTITIONS_STR, timer.endTimer())); - - Collections.sort(actualPartitions); - Collections.sort(partitions); - if (!actualPartitions.equals(partitions)) { - LOG.error("Validation of metadata partition list failed. Lists do not match."); - LOG.error("Partitions from metadata: " + Arrays.toString(partitions.toArray())); - LOG.error("Partitions from file system: " + Arrays.toString(actualPartitions.toArray())); - - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_ERRORS_STR, 0)); - } - - // Return the direct listing as it should be correct - partitions = actualPartitions; - } - LOG.info("Listed partitions from metadata: #partitions=" + partitions.size()); return partitions; } @@ -217,7 +181,7 @@ FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { } HoodieTimer timer = new HoodieTimer().startTimer(); - Option> hoodieRecord = getMergedRecordByKey(partitionName); + Option> hoodieRecord = getRecordByKeyFromMetadata(partitionName, MetadataPartitionType.FILES.partitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer())); FileStatus[] statuses = {}; @@ -229,116 +193,75 @@ FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { statuses = hoodieRecord.get().getData().getFileStatuses(hadoopConf.get(), partitionPath); } - if (metadataConfig.validateFileListingMetadata()) { - // Validate the Metadata Table data by listing the partitions from the file system - timer.startTimer(); - - String partitionPathStr = FSUtils.getRelativePartitionPath(new Path(datasetMetaClient.getBasePath()), partitionPath); - String latestDataInstantTime = getLatestDatasetInstantTime(); - HoodieTableFileSystemView dataFsView = new HoodieTableFileSystemView(datasetMetaClient, datasetMetaClient.getActiveTimeline()); - List directStatuses = dataFsView.getAllFileSlices(partitionPathStr).flatMap(slice -> { - List paths = new ArrayList<>(); - slice.getBaseFile().ifPresent(baseFile -> { - if (HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, latestDataInstantTime)) { - paths.add(baseFile.getFileStatus()); - } - }); - //TODO(metadata): this will remain problematic; no way to know the commit time based on log file written - slice.getLogFiles().forEach(logFile -> paths.add(logFile.getFileStatus())); - return paths.stream(); - }).collect(Collectors.toList()); - - List directFilenames = directStatuses.stream() - .map(fileStatus -> fileStatus.getPath().getName()).sorted() - .collect(Collectors.toList()); - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_FILES_STR, timer.endTimer())); - - List metadataFilenames = Arrays.stream(statuses) - .map(s -> s.getPath().getName()).sorted() - .collect(Collectors.toList()); - - if (!metadataFilenames.equals(directFilenames)) { - LOG.error("Validation of metadata file listing for partition " + partitionName + " failed."); - LOG.error("File list from metadata: " + Arrays.toString(metadataFilenames.toArray())); - LOG.error("File list from direct listing: " + Arrays.toString(directFilenames.toArray())); - - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_ERRORS_STR, 0)); - } - - // Return the direct listing as it should be correct - statuses = directStatuses.toArray(new FileStatus[0]); - } - LOG.info("Listed file in partition from metadata: partition=" + partitionName + ", #files=" + statuses.length); return statuses; } /** - * Retrieve the merged {@code HoodieRecord} mapped to the given key. + * Reads record keys from record-level index. + * + * If the Metadata Table is not enabled, an exception is thrown to distinguish this from the absence of the key. * - * @param key The key of the record + * @param recordKeys The list of record keys to read */ - private Option> getMergedRecordByKey(String key) { - Option> mergedRecord; - Option> metadataHoodieRecord = getRecordByKeyFromMetadata(key); - // Retrieve record from unsynced timeline instants - Option> timelineHoodieRecord = timelineMergedMetadata.getRecordByKey(key); - if (timelineHoodieRecord.isPresent()) { - if (metadataHoodieRecord.isPresent()) { - HoodieRecordPayload mergedPayload = timelineHoodieRecord.get().getData().preCombine(metadataHoodieRecord.get().getData()); - mergedRecord = Option.of(new HoodieRecord(metadataHoodieRecord.get().getKey(), mergedPayload)); - } else { - mergedRecord = timelineHoodieRecord; + @Override + public Map readRecordLevelIndex(Set recordKeys) { + ValidationUtils.checkState(enabled, "Cannot access record-level index as Metadata Table is disabled"); + ValidationUtils.checkState(metadataConfig.isRecordLevelIndexEnabled(), + "Cannot access record-level index as it is disabled in the config"); + + HoodieTimer timer = new HoodieTimer().startTimer(); + + // Pre loading the keys helps in reducing the overhead of individual key lookup and merge + loadAndCacheKeys(recordKeys, MetadataPartitionType.RECORD_LEVEL_INDEX.partitionPath()); + + Map recordKeyToLocation = new HashMap<>(recordKeys.size()); + recordKeys.stream().sorted().forEach(recordKey -> { + Option> hoodieRecord = getRecordByKeyFromCache(recordKey); + if (hoodieRecord.isPresent()) { + recordKeyToLocation.put(recordKey, hoodieRecord.get().getData().getRecordLocation()); } - } else { - mergedRecord = metadataHoodieRecord; - } - return mergedRecord; - } + }); - protected abstract Option> getRecordByKeyFromMetadata(String key); + clearKeyCache(); - private void openTimelineScanner() { - if (timelineMergedMetadata == null) { - List unSyncedInstants = findInstantsToSyncForReader(); - timelineMergedMetadata = - new TimelineMergedTableMetadata(datasetMetaClient, unSyncedInstants, getSyncedInstantTime(), null); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_RECORDINDEX_STR, timer.endTimer())); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.LOOKUP_RECORDKEYS_COUNT_STR, recordKeys.size())); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.RECORDINDEX_HITS_STR, recordKeyToLocation.size())); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.RECORDINDEX_MISS_STR, recordKeys.size() - recordKeyToLocation.size())); - syncedInstantTime = unSyncedInstants.isEmpty() ? getLatestDatasetInstantTime() - : unSyncedInstants.get(unSyncedInstants.size() - 1).getTimestamp(); - } + return recordKeyToLocation; } /** - * Return the timestamp of the latest synced instant. + * Load the records for the given keys into a cache. + * + * @param recordKeys The keys of the record which are to be loaded. + * @param partitionPath The metadata partition. */ - @Override - public Option getSyncedInstantTime() { - if (!enabled) { - return Option.empty(); - } + protected abstract void loadAndCacheKeys(Set recordKeys, String partitionPath); - return Option.ofNullable(syncedInstantTime); - } + /** + * Clear the cache of records. + */ + protected abstract void clearKeyCache(); /** - * Return the instants which are not-synced to the {@code HoodieTableMetadata}. + * Reads and returns a record from metadata. * - * This is the list of all completed but un-synched instants. + * @param key Key of the record to read. + * @param partitionName The metadata partition. + * @return */ - protected abstract List findInstantsToSyncForReader(); + protected abstract Option> getRecordByKeyFromMetadata(String key, String partitionName); /** - * Return the instants which are not-synced to the {@code HoodieTableMetadataWriter}. + * Reads and returns a record from cache. * - * This is the list of all completed but un-synched instants which do not have any incomplete instants in between them. + * @param key Key of the record to read. + * @return */ - protected abstract List findInstantsToSyncForWriter(); - - @Override - public boolean isInSync() { - return enabled && findInstantsToSyncForWriter().isEmpty(); - } + protected abstract Option> getRecordByKeyFromCache(String key); protected HoodieEngineContext getEngineContext() { return engineContext != null ? engineContext : new HoodieLocalEngineContext(hadoopConf.get()); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java index ce1cf5502c80f..4e5dd4844990f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -37,6 +38,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; public class FileSystemBackedTableMetadata implements HoodieTableMetadata { @@ -131,12 +133,17 @@ public Option getSyncedInstantTime() { } @Override - public boolean isInSync() { - return true; + public Option getLatestCompactionTime() { + throw new UnsupportedOperationException(); } @Override public void close() throws Exception { // no-op } + + @Override + public Map readRecordLevelIndex(Set recordKeys) { + throw new UnsupportedOperationException(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 8995ab49184dd..a31a1a347081f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -31,16 +31,17 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.DefaultSizeEstimator; +import org.apache.hudi.common.util.HoodieRecordSizeEstimator; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; -import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; @@ -52,18 +53,15 @@ import org.apache.log4j.Logger; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; /** * Table metadata provided by an internal DFS backed Hudi metadata table. - * - * If the metadata table does not exist, RPC calls are used to retrieve file listings from the file system. - * No updates are applied to the table and it is not synced. */ public class HoodieBackedTableMetadata extends BaseTableMetadata { @@ -72,15 +70,13 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { private String metadataBasePath; // Metadata table's timeline and metaclient private HoodieTableMetaClient metaClient; - private HoodieTableConfig tableConfig; - private List latestFileSystemMetadataSlices; // should we reuse the open file handles, across calls private final boolean reuse; - - // Readers for the base and log file which store the metadata - private transient HoodieFileReader baseFileReader; - private transient HoodieMetadataMergedLogRecordScanner logRecordScanner; + // Readers for each shard identified of a partition + private Map> shardReaders = new ConcurrentHashMap<>(); + // Cache of records loaded from metadata + protected transient ExternalSpillableMap cachedRecords; public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, String spillableMapDirectory) { @@ -92,243 +88,280 @@ public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetada super(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory); this.reuse = reuse; initIfNeeded(); + + try { + this.cachedRecords = new ExternalSpillableMap<>(metadataConfig.getMaxRecordCacheMemory(), metadataConfig.getSplliableMapDir(), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(HoodieMetadataRecord.SCHEMA$)); + } catch (IOException e) { + throw new HoodieMetadataException("Could not allocate an ExternalSpillableMap at " + metadataConfig.getSplliableMapDir(), e); + } } private void initIfNeeded() { + this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath); if (!enabled) { - LOG.info("Metadata table is disabled for " + datasetBasePath); + if (!HoodieTableMetadata.isMetadataTable(metadataBasePath)) { + LOG.info("Metadata table is disabled."); + } } else if (this.metaClient == null) { - this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath); try { this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build(); - this.tableConfig = metaClient.getTableConfig(); - HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); - latestFileSystemMetadataSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); } catch (TableNotFoundException e) { LOG.warn("Metadata table was not found at path " + metadataBasePath); this.enabled = false; this.metaClient = null; - this.tableConfig = null; } catch (Exception e) { LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e); this.enabled = false; this.metaClient = null; - this.tableConfig = null; } } } @Override - protected Option> getRecordByKeyFromMetadata(String key) { + protected Option> getRecordByKeyFromMetadata(String key, String partitionName) { + openReadersIfNeeded(partitionName); - openReadersIfNeededOrThrow(); try { - List timings = new ArrayList<>(); - HoodieTimer timer = new HoodieTimer().startTimer(); - - // Retrieve record from base file - HoodieRecord hoodieRecord = null; - if (baseFileReader != null) { - HoodieTimer readTimer = new HoodieTimer().startTimer(); - Option baseRecord = baseFileReader.getRecordByKey(key); - if (baseRecord.isPresent()) { - hoodieRecord = tableConfig.populateMetaFields() ? SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), - tableConfig.getPayloadClass()) : SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), - tableConfig.getPayloadClass(), Pair.of(tableConfig.getRecordKeyFieldProp(), - tableConfig.getPartitionFieldProp())); - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer())); - } - } - timings.add(timer.endTimer()); - - // Retrieve record from log file - timer.startTimer(); - if (logRecordScanner != null) { - Option> logHoodieRecord = logRecordScanner.getRecordByKey(key); - if (logHoodieRecord.isPresent()) { - if (hoodieRecord != null) { - // Merge the payloads - HoodieRecordPayload mergedPayload = logHoodieRecord.get().getData().preCombine(hoodieRecord.getData()); - hoodieRecord = new HoodieRecord(hoodieRecord.getKey(), mergedPayload); - } else { - hoodieRecord = logHoodieRecord.get(); - } - } - } - timings.add(timer.endTimer()); - LOG.info(String.format("Metadata read for key %s took [baseFileRead, logMerge] %s ms", key, timings)); - return Option.ofNullable(hoodieRecord); - } catch (IOException ioe) { - throw new HoodieIOException("Error merging records from metadata table for key :" + key, ioe); + List readers = shardReaders.get(partitionName); + int shardIndex = HoodieTableMetadataUtil.keyToShard(key, readers.size()); + return readers.get(shardIndex).getRecordByKey(key); } finally { if (!reuse) { - closeOrThrow(); + close(partitionName); } } } - private void openReadersIfNeededOrThrow() { - try { - openReadersIfNeeded(); - } catch (IOException e) { - throw new HoodieIOException("Error opening readers to the Metadata Table: ", e); - } - } - /** - * Returns a new pair of readers to the base and log files. + * Opens readers to the shards of a partition. */ - private void openReadersIfNeeded() throws IOException { - if (reuse && (baseFileReader != null || logRecordScanner != null)) { - // quickly exit out without synchronizing if reusing and readers are already open - return; - } - - // we always force synchronization, if reuse=false, to handle concurrent close() calls as well. - synchronized (this) { - if (baseFileReader != null || logRecordScanner != null) { - return; - } + private void openReadersIfNeeded(String partitionName) { + shardReaders.computeIfAbsent(partitionName, k -> { + // Load the shards + List shards = HoodieTableMetadataUtil.loadPartitionShards(metaClient, partitionName); - final long baseFileOpenMs; - final long logScannerOpenMs; - - // Metadata is in sync till the latest completed instant on the dataset - HoodieTimer timer = new HoodieTimer().startTimer(); - String latestInstantTime = getLatestDatasetInstantTime(); - ValidationUtils.checkArgument(latestFileSystemMetadataSlices.size() == 1, "must be at-least one valid metadata file slice"); - - // If the base file is present then create a reader - Option basefile = latestFileSystemMetadataSlices.get(0).getBaseFile(); - if (basefile.isPresent()) { - String basefilePath = basefile.get().getPath(); - baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath)); - baseFileOpenMs = timer.endTimer(); - LOG.info(String.format("Opened metadata base file from %s at instant %s in %d ms", basefilePath, - basefile.get().getCommitTime(), baseFileOpenMs)); - } else { - baseFileOpenMs = 0; - timer.endTimer(); - } + // Only those log files which have a corresponding completed instant on the dataset should be read + // This is because the metadata table is updated before the dataset instants are committed. + Set validInstantTimestamps = datasetMetaClient.getActiveTimeline().filterCompletedInstants().getInstants() + .map(i -> i.getTimestamp()).collect(Collectors.toSet()); - // Open the log record scanner using the log files from the latest file slice - timer.startTimer(); - List logFilePaths = latestFileSystemMetadataSlices.get(0).getLogFiles() - .sorted(HoodieLogFile.getLogFileComparator()) - .map(o -> o.getPath().toString()) - .collect(Collectors.toList()); Option lastInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); String latestMetaInstantTimestamp = lastInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); - // Load the schema - Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); - HoodieCommonConfig commonConfig = HoodieCommonConfig.newBuilder().fromProperties(metadataConfig.getProps()).build(); - logRecordScanner = HoodieMetadataMergedLogRecordScanner.newBuilder() - .withFileSystem(metaClient.getFs()) - .withBasePath(metadataBasePath) - .withLogFilePaths(logFilePaths) - .withReaderSchema(schema) - .withLatestInstantTime(latestMetaInstantTimestamp) - .withMaxMemorySizeInBytes(MAX_MEMORY_SIZE_IN_BYTES) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableMapDirectory) - .withDiskMapType(commonConfig.getSpillableDiskMapType()) - .withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled()) - .build(); - - logScannerOpenMs = timer.endTimer(); - LOG.info(String.format("Opened metadata log files from %s at instant (dataset instant=%s, metadata instant=%s) in %d ms", - logFilePaths, latestInstantTime, latestMetaInstantTimestamp, logScannerOpenMs)); - - metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, baseFileOpenMs + logScannerOpenMs)); - } + // Create a ShardReader for each shard + return shards.stream().map(slice -> { + try { + HoodieFileReader baseFileReader = null; + HoodieMetadataMergedLogRecordScanner logRecordScanner = null; + final long baseFileOpenMs; + final long logScannerOpenMs; + + HoodieTimer timer = new HoodieTimer().startTimer(); + + // Metadata is in sync till the latest completed instant on the dataset + // If the base file is present then create a reader + Option basefile = slice.getBaseFile(); + if (basefile.isPresent()) { + String basefilePath = basefile.get().getPath(); + baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath)); + baseFileOpenMs = timer.endTimer(); + LOG.info(String.format("Opened metadata base file from %s at instant %s in %d ms", basefilePath, + basefile.get().getCommitTime(), baseFileOpenMs)); + } else { + baseFileOpenMs = 0; + timer.endTimer(); + } + + // Open the log record scanner using the log files from the latest file slice + timer.startTimer(); + List logFilePaths = slice.getLogFiles() + .sorted(HoodieLogFile.getLogFileComparator()) + .map(o -> o.getPath().toString()) + .collect(Collectors.toList()); + + Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); + HoodieCommonConfig commonConfig = HoodieCommonConfig.newBuilder().fromProperties(metadataConfig.getProps()).build(); + logRecordScanner = HoodieMetadataMergedLogRecordScanner.newBuilder() + .withFileSystem(metaClient.getFs()) + .withBasePath(metadataBasePath) + .withLogFilePaths(logFilePaths) + .withReaderSchema(schema) + .withLatestInstantTime(latestMetaInstantTimestamp) + .withMaxMemorySizeInBytes(metadataConfig.getMaxReaderMemory()) + .withBufferSize(metadataConfig.getMaxReaderBufferSize()) + .withSpillableMapBasePath(metadataConfig.getSplliableMapDir()) + .withDiskMapType(commonConfig.getSpillableDiskMapType()) + .withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled()) + .withLogBlockTimestamps(validInstantTimestamps) + .build(); + + logScannerOpenMs = timer.endTimer(); + + LOG.info(String.format("Opened %d metadata log files (dataset instant=%s, metadata instant=%s) in %d ms", + logFilePaths.size(), getLatestDatasetInstantTime(), latestMetaInstantTimestamp, logScannerOpenMs)); + + metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, baseFileOpenMs + logScannerOpenMs)); + return new ShardReader(baseFileReader, logRecordScanner, metaClient.getTableConfig(), metrics); + } catch (Exception e) { + throw new HoodieMetadataException("Error opening readers for metadata table partition " + partitionName, e); + } + }).collect(Collectors.toList()); + }); } - private void close(HoodieFileReader localFileReader, HoodieMetadataMergedLogRecordScanner localLogScanner) { - try { - if (localFileReader != null) { - localFileReader.close(); - } - if (localLogScanner != null) { - localLogScanner.close(); - } - } catch (Exception e) { - throw new HoodieException("Error closing resources during metadata table merge", e); + @Override + public void close() { + for (String partitionName : shardReaders.keySet()) { + close(partitionName); } + shardReaders.clear(); } - private void closeOrThrow() { - try { - close(); - } catch (Exception e) { - throw new HoodieException("Error closing metadata table readers", e); - } + private synchronized void close(String partitionName) { + shardReaders.remove(partitionName).forEach(sr -> sr.close()); } - @Override - public synchronized void close() throws Exception { - close(baseFileReader, logRecordScanner); - baseFileReader = null; - logRecordScanner = null; + public boolean enabled() { + return enabled; + } + + public SerializableConfiguration getHadoopConf() { + return hadoopConf; + } + + public HoodieTableMetaClient getMetaClient() { + return metaClient; + } + + public Map stats() { + return metrics.map(m -> m.getStats(true, metaClient, this)).orElse(new HashMap<>()); } - /** - * Return an ordered list of instants which have not been synced to the Metadata Table. - */ @Override - protected List findInstantsToSyncForReader() { - return findInstantsToSync(true); + public Option getSyncedInstantTime() { + if (metaClient != null) { + Option latestInstant = metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant(); + if (latestInstant.isPresent()) { + return Option.of(latestInstant.get().getTimestamp()); + } + } + + return Option.empty(); } - /** - * Return an ordered list of instants which have not been synced to the Metadata Table. - */ @Override - protected List findInstantsToSyncForWriter() { - return findInstantsToSync(false); + public Option getLatestCompactionTime() { + if (metaClient != null) { + Option latestCompaction = metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant(); + if (latestCompaction.isPresent()) { + return Option.of(latestCompaction.get().getTimestamp()); + } + } + + return Option.empty(); } - /** - * Return an ordered list of instants which have not been synced to the Metadata Table. - */ - private List findInstantsToSync(boolean ignoreIncompleteInstants) { - initIfNeeded(); + @Override + protected void loadAndCacheKeys(Set recordKeys, String partitionName) { + cachedRecords.clear(); - // if there are no instants yet, return empty list, since there is nothing to sync here. - if (!enabled || !metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().isPresent()) { - return Collections.EMPTY_LIST; + if (recordKeys.isEmpty()) { + return; } - // All instants on the data timeline, which are greater than the last instant on metadata timeline - // are candidates for sync. - String latestMetadataInstantTime = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get().getTimestamp(); - HoodieDefaultTimeline candidateTimeline = datasetMetaClient.getActiveTimeline().findInstantsAfter(latestMetadataInstantTime, Integer.MAX_VALUE); - Option earliestIncompleteInstant = ignoreIncompleteInstants ? Option.empty() - : candidateTimeline.filterInflightsAndRequested().firstInstant(); - - if (earliestIncompleteInstant.isPresent()) { - return candidateTimeline.filterCompletedInstants() - .findInstantsBefore(earliestIncompleteInstant.get().getTimestamp()) - .getInstants().collect(Collectors.toList()); - } else { - return candidateTimeline.filterCompletedInstants() - .getInstants().collect(Collectors.toList()); + // Open the readers + openReadersIfNeeded(partitionName); + + // Load keys from all shards involved + List readers = shardReaders.get(partitionName); + for (String key : recordKeys) { + ShardReader reader = readers.get(HoodieTableMetadataUtil.keyToShard(key, readers.size())); + Option> record = reader.getRecordByKey(key); + if (record.isPresent()) { + cachedRecords.put(key, record.get()); + } } } - public boolean enabled() { - return enabled; + @Override + protected void clearKeyCache() { + cachedRecords.clear(); } - public SerializableConfiguration getHadoopConf() { - return hadoopConf; + @Override + protected Option> getRecordByKeyFromCache(String key) { + return Option.ofNullable(cachedRecords.get(key)); } - public HoodieTableMetaClient getMetaClient() { - return metaClient; - } + /** + * A reader for one shard of the metadata table. + * + * A shard is made up of a base file and a set of log files. This reader merges the record from base and log files to + * return the latest record. + */ + private static class ShardReader { + private final HoodieFileReader baseFileReader; + private final HoodieMetadataMergedLogRecordScanner logFileReader; + private Option metrics; + private final HoodieTableConfig tableConfig; + + public ShardReader(HoodieFileReader baseFileReader, HoodieMetadataMergedLogRecordScanner logFileReader, + HoodieTableConfig tableConfig, Option metrics) { + this.baseFileReader = baseFileReader; + this.logFileReader = logFileReader; + this.tableConfig = tableConfig; + this.metrics = metrics; + } - public Map stats() { - return metrics.map(m -> m.getStats(true, metaClient, this)).orElse(new HashMap<>()); + public void close() { + try { + if (baseFileReader != null) { + baseFileReader.close(); + } + if (logFileReader != null) { + logFileReader.close(); + } + } catch (Exception e) { + throw new HoodieException("Error closing resources during metadata table merge", e); + } + } + + public Option> getRecordByKey(String key) { + try { + // Retrieve record from base file + HoodieRecord hoodieRecord = null; + if (baseFileReader != null) { + HoodieTimer timer = new HoodieTimer().startTimer(); + Option baseRecord = baseFileReader.getRecordByKey(key); + if (baseRecord.isPresent()) { + hoodieRecord = tableConfig.populateMetaFields() ? SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), + tableConfig.getPayloadClass()) : SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), + tableConfig.getPayloadClass(), Pair.of(tableConfig.getRecordKeyFieldProp(), + tableConfig.getPartitionFieldProp())); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, timer.endTimer())); + } + } + + // Retrieve record from log file + if (logFileReader != null) { + HoodieTimer timer = new HoodieTimer().startTimer(); + Option> logHoodieRecord = logFileReader.getRecordByKey(key); + if (logHoodieRecord.isPresent()) { + if (hoodieRecord != null) { + // Merge the payloads + HoodieRecordPayload mergedPayload = logHoodieRecord.get().getData().preCombine(hoodieRecord.getData()); + hoodieRecord = new HoodieRecord(hoodieRecord.getKey(), mergedPayload); + } else { + hoodieRecord = logHoodieRecord.get(); + } + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOGFILE_READ_STR, timer.endTimer())); + } + } + return Option.ofNullable(hoodieRecord); + } catch (IOException ioe) { + throw new HoodieIOException("Error merging records from metadata table for key :" + key, ioe); + } + } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java index 51b0315643371..2c0669dfaa097 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; @@ -37,15 +38,17 @@ * useful in limiting memory usage when only a small subset of updates records are to be read. */ public class HoodieMetadataMergedLogRecordScanner extends HoodieMergedLogRecordScanner { + // Set of all record keys that are to be read in memory private Set mergeKeyFilter; private HoodieMetadataMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, int bufferSize, String spillableMapBasePath, Set mergeKeyFilter, - ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { + ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled, + Option instantRange) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize, - spillableMapBasePath, Option.empty(), false, diskMapType, isBitCaskDiskMapCompressionEnabled); + spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled); this.mergeKeyFilter = mergeKeyFilter; performScan(); @@ -88,59 +91,71 @@ public Option> getRecordByKey(String key) { public static class Builder extends HoodieMergedLogRecordScanner.Builder { private Set mergeKeyFilter = Collections.emptySet(); + @Override public Builder withFileSystem(FileSystem fs) { this.fs = fs; return this; } + @Override public Builder withBasePath(String basePath) { this.basePath = basePath; return this; } + @Override public Builder withLogFilePaths(List logFilePaths) { this.logFilePaths = logFilePaths; return this; } + @Override public Builder withReaderSchema(Schema schema) { this.readerSchema = schema; return this; } + @Override public Builder withLatestInstantTime(String latestInstantTime) { this.latestInstantTime = latestInstantTime; return this; } + @Override public Builder withReadBlocksLazily(boolean readBlocksLazily) { throw new UnsupportedOperationException(); } + @Override public Builder withReverseReader(boolean reverseReader) { throw new UnsupportedOperationException(); } + @Override public Builder withBufferSize(int bufferSize) { this.bufferSize = bufferSize; return this; } + @Override public Builder withMaxMemorySizeInBytes(Long maxMemorySizeInBytes) { this.maxMemorySizeInBytes = maxMemorySizeInBytes; return this; } + @Override public Builder withSpillableMapBasePath(String spillableMapBasePath) { this.spillableMapBasePath = spillableMapBasePath; return this; } + @Override public Builder withDiskMapType(ExternalSpillableMap.DiskMapType diskMapType) { this.diskMapType = diskMapType; return this; } + @Override public Builder withBitCaskDiskMapCompressionEnabled(boolean isBitCaskDiskMapCompressionEnabled) { this.isBitCaskDiskMapCompressionEnabled = isBitCaskDiskMapCompressionEnabled; return this; @@ -151,11 +166,30 @@ public Builder withMergeKeyFilter(Set mergeKeyFilter) { return this; } + public Builder withLogBlockTimestamps(Set validLogBlockTimestamps) { + withInstantRange(Option.of(new ExplicitMatchRange(validLogBlockTimestamps))); + return this; + } + @Override public HoodieMetadataMergedLogRecordScanner build() { return new HoodieMetadataMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath, mergeKeyFilter, - diskMapType, isBitCaskDiskMapCompressionEnabled); + diskMapType, isBitCaskDiskMapCompressionEnabled, instantRange); + } + } + + private static class ExplicitMatchRange extends InstantRange { + Set instants; + + public ExplicitMatchRange(Set instants) { + super(Collections.min(instants), Collections.max(instants)); + this.instants = instants; + } + + @Override + public boolean isInRange(String instant) { + return this.instants.contains(instant); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java index 9f1b0a0799af0..fbe9a0bd331c4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java @@ -41,15 +41,21 @@ public class HoodieMetadataMetrics implements Serializable { // Metric names public static final String LOOKUP_PARTITIONS_STR = "lookup_partitions"; public static final String LOOKUP_FILES_STR = "lookup_files"; - public static final String VALIDATE_PARTITIONS_STR = "validate_partitions"; - public static final String VALIDATE_FILES_STR = "validate_files"; - public static final String VALIDATE_ERRORS_STR = "validate_errors"; + public static final String LOOKUP_RECORDINDEX_STR = "lookup_record_index"; + public static final String LOOKUP_RECORDKEYS_COUNT_STR = "lookup_recordkeys_count"; public static final String SCAN_STR = "scan"; public static final String BASEFILE_READ_STR = "basefile_read"; - public static final String INITIALIZE_STR = "initialize"; - public static final String SYNC_STR = "sync"; + public static final String LOGFILE_READ_STR = "logfile_read"; + public static final String INITIALIZE_FILE_LISTING_STR = "initialize_files"; + public static final String INITIALIZE_RECORD_INDEX_STR = "initialize_record_index"; public static final String REBOOTSTRAP_STR = "rebootstrap"; public static final String BOOTSTRAP_ERR_STR = "bootstrap_error"; + public static final String READ_RECORDKEYS_BOOTSTRAP_STR = "read_recordkeys_bootstrap"; + public static final String READ_FILES_BOOTSTRAP_STR = "read_filelisting_bootstrap"; + public static final String RECORDINDEX_SHARD_COUNT_STR = "record_index_shard_count"; + public static final String RECORDINDEX_BOOTSTRAP_COUNT_STR = "record_index_bootstrap_count"; + public static final String RECORDINDEX_HITS_STR = "record_index_hits"; + public static final String RECORDINDEX_MISS_STR = "record_index_misses"; // Stats names public static final String STAT_TOTAL_BASE_FILE_SIZE = "totalBaseFileSizeInBytes"; @@ -57,7 +63,6 @@ public class HoodieMetadataMetrics implements Serializable { public static final String STAT_COUNT_BASE_FILES = "baseFileCount"; public static final String STAT_COUNT_LOG_FILES = "logFileCount"; public static final String STAT_COUNT_PARTITION = "partitionCount"; - public static final String STAT_IN_SYNC = "isInSync"; public static final String STAT_LAST_COMPACTION_TIMESTAMP = "lastCompactionTimestamp"; private static final Logger LOG = LogManager.getLogger(HoodieMetadataMetrics.class); @@ -82,32 +87,35 @@ private Map getStats(HoodieTableFileSystemView fsView, boolean d Map stats = new HashMap<>(); // Total size of the metadata and count of base/log files - long totalBaseFileSizeInBytes = 0; - long totalLogFileSizeInBytes = 0; - int baseFileCount = 0; - int logFileCount = 0; - List latestSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); - - for (FileSlice slice : latestSlices) { - if (slice.getBaseFile().isPresent()) { - totalBaseFileSizeInBytes += slice.getBaseFile().get().getFileStatus().getLen(); - ++baseFileCount; + for (String metadataPartition : MetadataPartitionType.all()) { + List latestSlices = fsView.getLatestFileSlices(metadataPartition).collect(Collectors.toList()); + + // Total size of the metadata and count of base/log files + long totalBaseFileSizeInBytes = 0; + long totalLogFileSizeInBytes = 0; + int baseFileCount = 0; + int logFileCount = 0; + + for (FileSlice slice : latestSlices) { + if (slice.getBaseFile().isPresent()) { + totalBaseFileSizeInBytes += slice.getBaseFile().get().getFileStatus().getLen(); + ++baseFileCount; + } + Iterator it = slice.getLogFiles().iterator(); + while (it.hasNext()) { + totalLogFileSizeInBytes += it.next().getFileStatus().getLen(); + ++logFileCount; + } } - Iterator it = slice.getLogFiles().iterator(); - while (it.hasNext()) { - totalLogFileSizeInBytes += it.next().getFileStatus().getLen(); - ++logFileCount; - } - } - stats.put(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE, String.valueOf(totalBaseFileSizeInBytes)); - stats.put(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE, String.valueOf(totalLogFileSizeInBytes)); - stats.put(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES, String.valueOf(baseFileCount)); - stats.put(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES, String.valueOf(logFileCount)); + stats.put(metadataPartition + "." + STAT_TOTAL_BASE_FILE_SIZE, String.valueOf(totalBaseFileSizeInBytes)); + stats.put(metadataPartition + "." + STAT_TOTAL_LOG_FILE_SIZE, String.valueOf(totalLogFileSizeInBytes)); + stats.put(metadataPartition + "." + STAT_COUNT_BASE_FILES, String.valueOf(baseFileCount)); + stats.put(metadataPartition + "." + STAT_COUNT_LOG_FILES, String.valueOf(logFileCount)); + } if (detailed) { stats.put(HoodieMetadataMetrics.STAT_COUNT_PARTITION, String.valueOf(tableMetadata.getAllPartitionPaths().size())); - stats.put(HoodieMetadataMetrics.STAT_IN_SYNC, String.valueOf(tableMetadata.isInSync())); } return stats; @@ -121,26 +129,20 @@ protected void updateMetrics(String action, long durationInMs) { // Update sum of duration and total for count String countKey = action + ".count"; String durationKey = action + ".totalDuration"; - metricsRegistry.add(countKey, 1); - metricsRegistry.add(durationKey, durationInMs); - - LOG.info(String.format("Updating metadata metrics (%s=%dms, %s=1)", durationKey, durationInMs, countKey)); + incrementMetric(countKey, 1); + incrementMetric(durationKey, durationInMs); } - public void updateMetrics(long totalBaseFileSizeInBytes, long totalLogFileSizeInBytes, int baseFileCount, - int logFileCount) { - if (metricsRegistry == null) { - return; + public void updateSizeMetrics(HoodieTableMetaClient metaClient, HoodieBackedTableMetadata metadata) { + Map stats = getStats(false, metaClient, metadata); + for (Map.Entry e : stats.entrySet()) { + incrementMetric(e.getKey(), Long.parseLong(e.getValue())); } + } - // Update sizes and count for metadata table's data files - metricsRegistry.add("basefile.size", totalBaseFileSizeInBytes); - metricsRegistry.add("logfile.size", totalLogFileSizeInBytes); - metricsRegistry.add("basefile.count", baseFileCount); - metricsRegistry.add("logfile.count", logFileCount); - - LOG.info(String.format("Updating metadata size metrics (basefile.size=%d, logfile.size=%d, basefile.count=%d, " - + "logfile.count=%d)", totalBaseFileSizeInBytes, totalLogFileSizeInBytes, baseFileCount, logFileCount)); + protected void incrementMetric(String action, long value) { + LOG.info(String.format("Updating metadata metrics (%s=%d) in %s", action, value, metricsRegistry)); + metricsRegistry.add(action, value); } public Registry registry() { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 9c6eb89b909f8..ae34ad2345a47 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -20,9 +20,12 @@ import org.apache.hudi.avro.model.HoodieMetadataFileInfo; import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.avro.model.HoodieRecordIndexInfo; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieMetadataException; @@ -36,9 +39,11 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -65,10 +70,12 @@ public class HoodieMetadataPayload implements HoodieRecordPayload filesystemMetadata = null; + private Map filesystemMetadata; + private HoodieRecordIndexInfo recordIndexInfo; public HoodieMetadataPayload(Option record) { if (record.isPresent()) { @@ -76,12 +83,19 @@ public HoodieMetadataPayload(Option record) { // https://issues.apache.org/jira/browse/AVRO-1811 key = record.get().get("key").toString(); type = (int) record.get().get("type"); - if (record.get().get("filesystemMetadata") != null) { + if (type == PARTITION_LIST || type == FILE_LIST) { filesystemMetadata = (Map) record.get().get("filesystemMetadata"); filesystemMetadata.keySet().forEach(k -> { GenericRecord v = filesystemMetadata.get(k); filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long)v.get("size"), (Boolean)v.get("isDeleted"))); }); + } else if (type == RECORD_LEVEL_INDEX) { + GenericRecord recordLevelIndexMetadata = (GenericRecord) record.get().get("recordLevelIndexMetadata"); + recordIndexInfo = new HoodieRecordIndexInfo(recordLevelIndexMetadata.get("partition").toString(), + Long.parseLong(recordLevelIndexMetadata.get("fileIdHighBits").toString()), + Long.parseLong(recordLevelIndexMetadata.get("fileIdLowBits").toString()), + Integer.parseInt(recordLevelIndexMetadata.get("fileIndex").toString()), + Integer.parseInt(recordLevelIndexMetadata.get("instantTime").toString())); } } } @@ -92,6 +106,14 @@ private HoodieMetadataPayload(String key, int type, Map createPartitionFilesRecord(Str return new HoodieRecord<>(key, payload); } + /** + * Create and return a {@code HoodieMetadataPayload} to save an entry for the record level index. + * + * Each entry maps the key of a single record in HUDI to its location. + * + * @param recordKey Key of the record + * @param partition Name of the partition which contains the record + * @param fileId fileId which contains the record + * @param instantTime instantTime when the record was added + */ + public static HoodieRecord createRecordLevelIndexRecord(String recordKey, String partition, + String fileId, String instantTime) { + HoodieKey key = new HoodieKey(recordKey, MetadataPartitionType.RECORD_LEVEL_INDEX.partitionPath()); + // Data file names have a -D suffix to denote the index (D = integer) of the file written + final int index = fileId.lastIndexOf("-"); + // TODO: Some UUIDs are invalid + UUID uuid; + int fileIndex = 0; + try { + uuid = UUID.fromString(fileId.substring(0, index)); + fileIndex = Integer.parseInt(fileId.substring(index + 1)); + } catch (Exception e) { + // TODO: only for testing. + uuid = UUID.randomUUID(); + } + Date instantDate; + try { + instantDate = HoodieActiveTimeline.COMMIT_FORMATTER.parse(instantTime); + } catch (Exception e) { + throw new HoodieMetadataException("Invalid instantTime format: " + instantTime, e); + } + + HoodieMetadataPayload payload = new HoodieMetadataPayload(recordKey, new HoodieRecordIndexInfo(partition, + uuid.getMostSignificantBits(), uuid.getLeastSignificantBits(), fileIndex, (int)(instantDate.getTime() / 1000))); + return new HoodieRecord<>(key, payload); + } + + /** + * Create and return a {@code HoodieMetadataPayload} to delete a record in the Metadata Table's record level index. + * + * @param recordKey Key of the record to be deleted + */ + public static HoodieRecord createRecordLevelIndexDelete(String recordKey) { + HoodieKey key = new HoodieKey(recordKey, MetadataPartitionType.RECORD_LEVEL_INDEX.partitionPath()); + HoodieMetadataPayload payload = new HoodieMetadataPayload(); + return new HoodieRecord<>(key, payload); + } + @Override public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { - ValidationUtils.checkArgument(previousRecord.type == type, - "Cannot combine " + previousRecord.type + " with " + type); - - Map combinedFileInfo = null; + ValidationUtils.checkArgument(previousRecord.type == type, "Cannot combine " + previousRecord.type + " with " + type); switch (type) { case PARTITION_LIST: case FILE_LIST: - combinedFileInfo = combineFilesystemMetadata(previousRecord); - break; + Map combinedFileInfo = combineFilesystemMetadata(previousRecord); + return new HoodieMetadataPayload(key, type, combinedFileInfo); + case RECORD_LEVEL_INDEX: + ValidationUtils.checkArgument(previousRecord.recordIndexInfo.getInstantTime() == recordIndexInfo.getInstantTime(), + String.format("InstantTime should not change from %s to %s", previousRecord.recordIndexInfo.getInstantTime(), + recordIndexInfo.getInstantTime())); + return this; default: throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type); } - - return new HoodieMetadataPayload(key, type, combinedFileInfo); } @Override @@ -158,7 +228,7 @@ public Option getInsertValue(Schema schema) throws IOException { return Option.empty(); } - HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata); + HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata, recordIndexInfo); return Option.of(record); } @@ -188,6 +258,16 @@ null, null, null, new Path(partitionPath, e.getKey()))) .toArray(FileStatus[]::new); } + /** + * If this is a record-level index entry, returns the file to which this is mapped. + */ + public HoodieRecordLocation getRecordLocation() { + final UUID uuid = new UUID(recordIndexInfo.getFileIdHighBits(), recordIndexInfo.getFileIdLowBits()); + final String fileId = String.format("%s-%d", uuid.toString(), recordIndexInfo.getFileIndex()); + final Date instantDate = new Date(recordIndexInfo.getInstantTime() * 1000); + return new HoodieRecordLocation(HoodieActiveTimeline.COMMIT_FORMATTER.format(instantDate), fileId); + } + private Stream> filterFileInfoEntries(boolean isDeleted) { if (filesystemMetadata == null) { return Stream.empty(); @@ -229,8 +309,12 @@ public String toString() { final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {"); sb.append("key=").append(key).append(", "); sb.append("type=").append(type).append(", "); - sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); - sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); + if (type == PARTITION_LIST || type == FILE_LIST) { + sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); + sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); + } else if (type == RECORD_LEVEL_INDEX) { + sb.append(getRecordLocation().toString()); + } sb.append('}'); return sb.toString(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java index 506792125e5a2..831813d17e30d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -21,9 +21,9 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -31,6 +31,7 @@ import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Set; /** * Interface that supports querying various pieces of metadata about a hudi table. @@ -97,14 +98,26 @@ static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetad List getAllPartitionPaths() throws IOException; /** +<<<<<<< HEAD * Fetch all files for given partition paths. */ Map getAllFilesInPartitions(List partitionPaths) throws IOException; /** * Get the instant time to which the metadata is synced w.r.t data timeline. +======= + * Returns the location of recordKeys which are found in the record level index. + */ + public Map readRecordLevelIndex(Set recordKeys); + + /** + * Returns the timestamp of the latest synced instant. +>>>>>>> 47573155c (API to read from record level index.) */ Option getSyncedInstantTime(); - boolean isInSync(); + /** + * Returns the timestamp of the latest compaction. + */ + Option getLatestCompactionTime(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 594231225a6d7..50ff10c41a6a5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -22,24 +22,26 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; -import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.exception.HoodieException; - +import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; @@ -57,57 +59,6 @@ public class HoodieTableMetadataUtil { private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class); - /** - * Converts a timeline instant to metadata table records. - * - * @param datasetMetaClient The meta client associated with the timeline instant - * @param instant to fetch and convert to metadata table records - * @return a list of metadata table records - * @throws IOException - */ - public static Option> convertInstantToMetaRecords(HoodieTableMetaClient datasetMetaClient, HoodieInstant instant, Option lastSyncTs) throws IOException { - HoodieTimeline timeline = datasetMetaClient.getActiveTimeline(); - Option> records = Option.empty(); - ValidationUtils.checkArgument(instant.isCompleted(), "Only completed instants can be synced."); - - switch (instant.getAction()) { - case HoodieTimeline.CLEAN_ACTION: - HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(datasetMetaClient, instant); - records = Option.of(convertMetadataToRecords(cleanMetadata, instant.getTimestamp())); - break; - case HoodieTimeline.DELTA_COMMIT_ACTION: - case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.COMPACTION_ACTION: - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); - records = Option.of(convertMetadataToRecords(commitMetadata, instant.getTimestamp())); - break; - case HoodieTimeline.ROLLBACK_ACTION: - HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( - timeline.getInstantDetails(instant).get()); - records = Option.of(convertMetadataToRecords(rollbackMetadata, instant.getTimestamp(), lastSyncTs)); - break; - case HoodieTimeline.RESTORE_ACTION: - HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( - timeline.getInstantDetails(instant).get()); - records = Option.of(convertMetadataToRecords(restoreMetadata, instant.getTimestamp(), lastSyncTs)); - break; - case HoodieTimeline.SAVEPOINT_ACTION: - // Nothing to be done here - break; - case HoodieTimeline.REPLACE_COMMIT_ACTION: - HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes( - timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); - // Note: we only add new files created here. Replaced files are removed from metadata later by cleaner. - records = Option.of(convertMetadataToRecords(replaceMetadata, instant.getTimestamp())); - break; - default: - throw new HoodieException("Unknown type of action " + instant.getAction()); - } - - return records; - } - /** * Finds all new files/partitions created as part of commit and creates metadata table records for them. * @@ -118,6 +69,7 @@ public static Option> convertInstantToMetaRecords(HoodieTable public static List convertMetadataToRecords(HoodieCommitMetadata commitMetadata, String instantTime) { List records = new LinkedList<>(); List allPartitions = new LinkedList<>(); + int[] newFileCount = {0}; commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { final String partition = partitionStatName.equals("") ? NON_PARTITIONED_NAME : partitionStatName; allPartitions.add(partition); @@ -141,14 +93,15 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord( partition, Option.of(newFiles), Option.empty()); records.add(record); + newFileCount[0] += newFiles.size(); }); // New partitions created HoodieRecord record = HoodieMetadataPayload.createPartitionListRecord(new ArrayList<>(allPartitions)); records.add(record); - LOG.info("Updating at " + instantTime + " from Commit/" + commitMetadata.getOperationType() - + ". #partitions_updated=" + records.size()); + LOG.info(String.format("Updating at %s from Commit/%s. #partitions_updated=%d, #files_added=%d", instantTime, commitMetadata.getOperationType(), + records.size(), newFileCount[0])); return records; } @@ -223,11 +176,17 @@ public static List convertMetadataToRecords(HoodieRestoreMetadata return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore"); } - public static List convertMetadataToRecords(HoodieRollbackMetadata rollbackMetadata, String instantTime, Option lastSyncTs) { + public static List convertMetadataToRecords(HoodieRollbackMetadata rollbackMetadata, String instantTime, + Option lastSyncTs, boolean wasSynced) { Map> partitionToAppendedFiles = new HashMap<>(); Map> partitionToDeletedFiles = new HashMap<>(); processRollbackMetadata(rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs); + if (!wasSynced) { + // Since the instant-being-rolled-back was never committed to the metadata table, the files added there + // need not be deleted. For MOR Table, the rollback appends logBlocks so we need to keep the appended files. + partitionToDeletedFiles.clear(); + } return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Rollback"); } @@ -340,4 +299,77 @@ private static List convertFilesToRecords(Map return records; } + + /** + * Returns a list of commits which were rolled back as part of a Rollback or Restore operation. + * + * @param instant The Rollback operation to read + * @param timeline + */ + public static List getCommitsRolledback(HoodieInstant instant, HoodieActiveTimeline timeline) { + try { + if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( + timeline.getInstantDetails(instant).get()); + return rollbackMetadata.getCommitsRollback(); + } + + List commitsRolledback = new LinkedList<>(); + + if (instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)) { + // Restore is made up of several rollbacks + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( + timeline.getInstantDetails(instant).get()); + restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> { + rms.forEach(rm -> commitsRolledback.addAll(rm.getCommitsRollback())); + }); + } + + return commitsRolledback; + } catch (IOException e) { + throw new HoodieMetadataException("Error retrieving rollback commits for instant " + instant, e); + } + } + + /** + * Map a key to a shard. + * + * Note: For hashing, the algorithm is same as String.hashCode() but is being defined here as hashCode() + * implementation is not guaranteed by the JVM to be consistent across JVM versions and implementations. + * + * @param str + * @return An integer hash of the given string + */ + public static int keyToShard(String str, int numShards) { + int h = 0; + for (int i = 0; i < str.length(); ++i) { + h = 31 * h + str.charAt(i); + } + + return Math.abs(Math.abs(h) % numShards); + } + + /** + * Loads the list of shards for a partition of the Metadata Table. + * + * The list of shards is returned sorted in the correct order of shard index. + * @param metaClient + * @param partition The name of the partition whose shards are to be loaded. + * @return List of shards + */ + public static List loadPartitionShards(HoodieTableMetaClient metaClient, String partition) { + LOG.info("Loading shards for metadata table partition " + partition); + + // If there are no commits on the metadata table then the table's default FileSystemView will not return any file + // slices even though we may have initialized them. + HoodieTimeline timeline = metaClient.getActiveTimeline(); + if (timeline.empty()) { + final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.createNewInstantTime()); + timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails); + } + + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, timeline); + return fsView.getLatestFileSlices(partition).sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())) + .collect(Collectors.toList()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java index 0436de707d2fd..6e3fb3823f0c3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java @@ -18,8 +18,12 @@ package org.apache.hudi.metadata; +import java.util.Arrays; +import java.util.List; + public enum MetadataPartitionType { - FILES("files"); + FILES("files"), + RECORD_LEVEL_INDEX("record_index"); private final String partitionPath; @@ -30,4 +34,8 @@ public enum MetadataPartitionType { public String partitionPath() { return partitionPath; } + + public static List all() { + return Arrays.asList(MetadataPartitionType.FILES.partitionPath(), MetadataPartitionType.RECORD_LEVEL_INDEX.partitionPath()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java deleted file mode 100644 index 9ba3f26079d14..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - -/** - * Provides functionality to convert timeline instants to table metadata records and then merge by key. Specify - * a filter to limit keys that are merged and stored in memory. - */ -public class TimelineMergedTableMetadata implements Serializable { - - private static final Logger LOG = LogManager.getLogger(TimelineMergedTableMetadata.class); - - HoodieTableMetaClient metaClient; - private List instants; - private Option lastSyncTs; - private Set mergeKeyFilter; - - // keep it a simple hash map, so it can be easily passed onto the executors, once merged. - protected final Map> timelineMergedRecords; - - public TimelineMergedTableMetadata(HoodieTableMetaClient metaClient, List instants, - Option lastSyncTs, Set mergeKeyFilter) { - this.metaClient = metaClient; - this.instants = instants; - this.lastSyncTs = lastSyncTs; - this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet(); - this.timelineMergedRecords = new HashMap<>(); - - scan(); - } - - /** - * Converts instants in scanner to metadata table records and processes each record. - * - * @param - * @throws IOException - */ - private void scan() { - for (HoodieInstant instant : instants) { - try { - Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(metaClient, instant, lastSyncTs); - if (records.isPresent()) { - records.get().forEach(record -> processNextRecord(record)); - } - } catch (Exception e) { - LOG.error(String.format("Got exception when processing timeline instant %s", instant.getTimestamp()), e); - throw new HoodieException(String.format("Got exception when processing timeline instant %s", instant.getTimestamp()), e); - } - } - } - - /** - * Process metadata table record by merging with existing record if it is a part of the key filter. - * - * @param hoodieRecord - */ - private void processNextRecord(HoodieRecord hoodieRecord) { - String key = hoodieRecord.getRecordKey(); - if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(key)) { - if (timelineMergedRecords.containsKey(key)) { - // Merge and store the merged record - HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(timelineMergedRecords.get(key).getData(), new Properties()); - timelineMergedRecords.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue)); - } else { - // Put the record as is - timelineMergedRecords.put(key, hoodieRecord); - } - } - } - - /** - * Retrieve merged hoodie record for given key. - * - * @param key of the record to retrieve - * @return {@code HoodieRecord} if key was found else {@code Option.empty()} - */ - public Option> getRecordByKey(String key) { - return Option.ofNullable((HoodieRecord) timelineMergedRecords.get(key)); - } -} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 6bf2e9fabe1ff..e0f0198e963ee 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -123,6 +123,14 @@ private static void deleteMetaFile(String basePath, String instantTime, String s } } + private static void deleteMetaFile(String basePath, String instantTime, String suffix) throws IOException { + Path parentPath = Paths.get(basePath, HoodieTableMetaClient.METAFOLDER_NAME); + Path metaFilePath = parentPath.resolve(instantTime + suffix); + if (Files.exists(metaFilePath)) { + Files.delete(metaFilePath); + } + } + public static void createCommit(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION); } @@ -304,4 +312,16 @@ public static Map getBaseFileCountsForPaths(String basePath, FileS public static void deleteDeltaCommit(String basePath, String instantTime, FileSystem fs) throws IOException { deleteMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION, fs); } + + public static void deleteCommit(String basePath, String instantTime) throws IOException { + deleteMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION); + } + + public static void deleteRequestedCommit(String basePath, String instantTime) throws IOException { + deleteMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_COMMIT_EXTENSION); + } + + public static void deleteInflightCommit(String basePath, String instantTime) throws IOException { + deleteMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index 046fc9ed3d73f..aaf6e14180381 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -76,6 +76,9 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedReplaceCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRollbackFile; +import static org.apache.hudi.common.testutils.FileCreateUtils.deleteRequestedCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.deleteInflightCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.deleteCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; public class HoodieTestTable { @@ -433,4 +436,11 @@ public HoodieTestTableException(Throwable t) { super(t); } } + + public void removeCommit(String instantTime) throws Exception { + deleteRequestedCommit(basePath, instantTime); + deleteInflightCommit(basePath, instantTime); + deleteCommit(basePath, instantTime); + metaClient = HoodieTableMetaClient.reload(metaClient); + } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 53122c3eced1d..0e356da67e76c 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -214,8 +214,6 @@ public void notifyCheckpointComplete(long checkpointId) { startInstant(); // sync Hive if is enabled syncHiveIfEnabled(); - // sync metadata if is enabled - syncMetadataIfEnabled(); } }, "commits the instant %s", this.instant ); @@ -283,23 +281,6 @@ private void initMetadataSync() { this.metadataSyncExecutor = new NonThrownExecutor(LOG, true); } - /** - * Sync the write metadata to the metadata table. - */ - private void syncMetadataIfEnabled() { - if (tableState.syncMetadata) { - this.metadataSyncExecutor.execute(this::syncMetadata, - "sync metadata table for instant %s", this.instant); - } - } - - /** - * Sync the write metadata to the metadata table. - */ - private void syncMetadata() { - this.writeClient.syncTableMetadata(); - } - private void reset() { this.eventBuffer = new WriteMetadataEvent[this.parallelism]; } @@ -349,11 +330,6 @@ private void initInstant(String instant) { LOG.info("Recommit instant {}", instant); commitInstant(instant); } - if (tableState.syncMetadata) { - // initialize metadata table first if enabled - // condition: the data set timeline has committed instants - syncMetadata(); - } // starts a new instant startInstant(); }, "initialize instant %s", instant); @@ -374,8 +350,6 @@ private void handleEndInputEvent(WriteMetadataEvent event) { commitInstant(this.instant); // sync Hive if is enabled in batch mode. syncHiveIfEnabled(); - // sync metadata if is enabled in batch mode. - syncMetadataIfEnabled(); } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index 12eb039891c72..a370106f7ec13 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -79,7 +79,6 @@ private static HiveSyncConfig buildSyncConfig(Configuration conf) { hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS); hiveSyncConfig.useJdbc = conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC); hiveSyncConfig.useFileListingFromMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED); - hiveSyncConfig.verifyMetadataFileListing = false; hiveSyncConfig.ignoreExceptions = conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS); hiveSyncConfig.supportTimestamp = conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP); hiveSyncConfig.autoCreateDatabase = conf.getBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index f4bddbd392855..cd792b37c5ce1 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -72,7 +72,6 @@ import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP; -import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP; public class HoodieInputFormatUtils { @@ -419,7 +418,6 @@ public static Map> groupSnapshotPathsByMetaCli public static HoodieMetadataConfig buildMetadataConfig(Configuration conf) { return HoodieMetadataConfig.newBuilder() .enable(conf.getBoolean(METADATA_ENABLE_PROP.key(), DEFAULT_METADATA_ENABLE_FOR_READERS)) - .validate(conf.getBoolean(METADATA_VALIDATE_PROP.key(), METADATA_VALIDATE_PROP.defaultValue())) .build(); } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index bcdc5aaaea75f..794861d2537d7 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -88,7 +88,7 @@ protected List getPartitions(Option partitionsLimit) throws IOE // calls in metrics as they are not part of normal HUDI operation. HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); List partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metaClient.getBasePath(), - HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false); + HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false); // Sort partition so we can pick last N partitions by default Collections.sort(partitionPaths); if (!partitionPaths.isEmpty()) { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 5543ea1615687..ee352c905ccdb 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -122,9 +122,6 @@ case class HoodieFileIndex( properties.put(HoodieMetadataConfig.METADATA_ENABLE_PROP, sqlConf.getConfString(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(), HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString)) - properties.put(HoodieMetadataConfig.METADATA_VALIDATE_PROP, - sqlConf.getConfString(HoodieMetadataConfig.METADATA_VALIDATE_PROP.key(), - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue().toString)) properties.putAll(options.asJava) properties } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 6213ab859484a..5b96c8efcf68f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -454,7 +454,6 @@ object HoodieSparkSqlWriter { hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS) hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC) hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_ENABLE_PROP) - hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_VALIDATE_PROP) hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP).toBoolean hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).toBoolean diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 0fd299167582a..901fea4bb05cc 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -26,7 +26,6 @@ import org.apache.hudi.common.config.{HoodieConfig, TypedProperties} import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters.mapAsScalaMapConverter import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP -import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator} @@ -54,7 +53,6 @@ object HoodieWriterUtils { PARTITIONPATH_FIELD.key -> PARTITIONPATH_FIELD.defaultValue, KEYGENERATOR_CLASS.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL, METADATA_ENABLE_PROP.key -> METADATA_ENABLE_PROP.defaultValue.toString, - METADATA_VALIDATE_PROP.key -> METADATA_VALIDATE_PROP.defaultValue.toString, COMMIT_METADATA_KEYPREFIX.key -> COMMIT_METADATA_KEYPREFIX.defaultValue, INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue, STREAMING_RETRY_CNT.key -> STREAMING_RETRY_CNT.defaultValue, diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java index 9ae6a05c73bfe..7db795de1ef53 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java @@ -128,6 +128,7 @@ public class TestBootstrap extends HoodieClientTestBase { private JobConf rtJobConf; private SparkSession spark; + @Override @BeforeEach public void setUp() throws Exception { bootstrapBasePath = tmpFolder.toAbsolutePath().toString() + "/data"; @@ -139,6 +140,7 @@ public void setUp() throws Exception { reloadInputFormats(); } + @Override @AfterEach public void tearDown() throws IOException { cleanupSparkContexts(); @@ -373,8 +375,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -392,8 +393,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -409,8 +409,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -427,8 +426,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -443,8 +441,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -461,8 +458,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -549,6 +545,7 @@ public Map> select(List partitions = FSUtils.getAllPartitionPaths(context, baseDir, useFileListingFromMetadata, verifyMetadataFileListing, shouldAssumeDatePartitioning); + List partitions = FSUtils.getAllPartitionPaths(context, baseDir, useFileListingFromMetadata, shouldAssumeDatePartitioning); if (partitions.size() > 0) { LOG.info(String.format("The job needs to copy %d partitions.", partitions.size())); @@ -192,8 +188,7 @@ public static void main(String[] args) throws IOException { // Copy HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning, cfg.useFileListingFromMetadata, - cfg.verifyMetadataFileListing); + copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning, cfg.useFileListingFromMetadata); // Stop the job jsc.stop(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index 42ab6ca558b64..a310514d59650 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -155,7 +155,7 @@ private Option getLatestCommitTimestamp(FileSystem fs, Config cfg) { } private List getPartitions(HoodieEngineContext engineContext, Config cfg) { - return FSUtils.getAllPartitionPaths(engineContext, cfg.sourceBasePath, true, false, false); + return FSUtils.getAllPartitionPaths(engineContext, cfg.sourceBasePath, true, false); } private void createSuccessTag(FileSystem fs, Config cfg) throws IOException { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java index a2bc5f7f1ab60..2bf50b2e3653a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java @@ -87,8 +87,7 @@ private void setHostAddrFromSparkConf(SparkConf sparkConf) { public void run() throws IOException { JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-view-perf-" + cfg.basePath, cfg.sparkMaster); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - List allPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, cfg.basePath, - cfg.useFileListingFromMetadata, cfg.verifyMetadataFileListing, true); + List allPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, cfg.basePath, cfg.useFileListingFromMetadata, true); Collections.shuffle(allPartitionPaths); List selected = allPartitionPaths.stream().filter(p -> !p.contains("error")).limit(cfg.maxPartitions) .collect(Collectors.toList()); @@ -300,9 +299,6 @@ public static class Config implements Serializable { @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; - @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") - public Boolean verifyMetadataFileListing = HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(); - @Parameter(names = {"--help", "-h"}) public Boolean help = false; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java index 9aa03e2e46d22..6b5141507f1fd 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java @@ -69,8 +69,7 @@ public void testEmptySnapshotCopy() throws IOException { // Do the snapshot HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); copier.snapshot(jsc(), basePath, outputPath, true, - HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue()); + HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS); // Nothing changed; we just bail out assertEquals(fs.listStatus(new Path(basePath)).length, 1); @@ -123,8 +122,7 @@ public void testSnapshotCopy() throws Exception { // Do a snapshot copy HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc(), basePath, outputPath, false, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue()); + copier.snapshot(jsc(), basePath, outputPath, false, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS); // Check results assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName())));