diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java index 143a5123c54b4..64bbad3596fd9 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java @@ -25,12 +25,14 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; 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.config.HoodieWriteConfig; import org.apache.hudi.metadata.HoodieBackedTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hadoop.fs.FileStatus; @@ -129,42 +131,10 @@ public String create() throws IOException { } @CliCommand(value = "metadata delete", help = "Remove the Metadata Table") - public String delete() throws Exception { - HoodieCLI.getTableMetaClient(); - Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); - try { - FileStatus[] statuses = HoodieCLI.fs.listStatus(metadataPath); - if (statuses.length > 0) { - HoodieCLI.fs.delete(metadataPath, true); - } - } catch (FileNotFoundException e) { - // Metadata directory does not exist - } - - return String.format("Removed Metadata Table from %s", metadataPath); - } - - @CliCommand(value = "metadata init", help = "Update the metadata table from commits since the creation") - public String init(@CliOption(key = {"readonly"}, unspecifiedDefaultValue = "false", - help = "Open in read-only mode") final boolean readOnly) throws Exception { - HoodieCLI.getTableMetaClient(); - Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); - try { - HoodieCLI.fs.listStatus(metadataPath); - } catch (FileNotFoundException e) { - // Metadata directory does not exist - throw new RuntimeException("Metadata directory (" + metadataPath.toString() + ") does not exist."); - } - - HoodieTimer timer = new HoodieTimer().startTimer(); - if (!readOnly) { - HoodieWriteConfig writeConfig = getWriteConfig(); - initJavaSparkContext(); - SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); - } - - String action = readOnly ? "Opened" : "Initialized"; - return String.format(action + " Metadata Table in %s (duration=%.2fsec)", metadataPath, (timer.endTimer()) / 1000.0); + public String delete(@CliOption(key = {"backup"}, help = "Backup the metadata table before delete", mandatory = true) final boolean backup) throws Exception { + HoodieTableMetaClient dataMetaClient = HoodieCLI.getTableMetaClient(); + HoodieTableMetadataUtil.deleteMetadataTable(dataMetaClient, new HoodieSparkEngineContext(jsc), backup); + return "Metadata Table has been deleted from " + getMetadataTableBasePath(HoodieCLI.basePath); } @CliCommand(value = "metadata stats", help = "Print stats about the metadata") @@ -172,7 +142,7 @@ public String stats() throws IOException { HoodieCLI.getTableMetaClient(); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(HoodieCLI.conf), - config, HoodieCLI.basePath, "/tmp"); + config, HoodieCLI.basePath); Map stats = metadata.stats(); final List rows = new ArrayList<>(); @@ -196,7 +166,7 @@ public String listPartitions() throws IOException { initJavaSparkContext(); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), config, - HoodieCLI.basePath, "/tmp"); + HoodieCLI.basePath); if (!metadata.enabled()) { return "[ERROR] Metadata Table not enabled/initialized\n\n"; @@ -224,7 +194,7 @@ public String listFiles( HoodieCLI.getTableMetaClient(); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); HoodieBackedTableMetadata metaReader = new HoodieBackedTableMetadata( - new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp"); + new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath); if (!metaReader.enabled()) { return "[ERROR] Metadata Table not enabled/initialized\n\n"; @@ -252,7 +222,7 @@ public String validateFiles( HoodieCLI.getTableMetaClient(); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); HoodieBackedTableMetadata metadataReader = new HoodieBackedTableMetadata( - new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp"); + new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath); if (!metadataReader.enabled()) { return "[ERROR] Metadata Table not enabled/initialized\n\n"; @@ -260,7 +230,7 @@ public String validateFiles( HoodieMetadataConfig fsConfig = HoodieMetadataConfig.newBuilder().enable(false).build(); HoodieBackedTableMetadata fsMetaReader = new HoodieBackedTableMetadata( - new HoodieLocalEngineContext(HoodieCLI.conf), fsConfig, HoodieCLI.basePath, "/tmp"); + new HoodieLocalEngineContext(HoodieCLI.conf), fsConfig, HoodieCLI.basePath); HoodieTimer timer = new HoodieTimer().startTimer(); List metadataPartitions = metadataReader.getAllPartitionPaths(); 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 8da1db5b2224e..1b4e4d04366d3 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 @@ -33,6 +33,7 @@ import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.client.utils.TransactionUtils; import org.apache.hudi.common.HoodiePendingRollbackInfo; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -172,13 +173,14 @@ public boolean commit(String instantTime, O writeStatuses, Option> extraMetadata, String commitActionType, Map> partitionToReplacedFileIds); - public boolean commitStats(String instantTime, List stats, Option> extraMetadata, - String commitActionType) { - return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap()); + public boolean commitStats(String instantTime, HoodieData writeStatuses, List stats, + Option> extraMetadata, String commitActionType) { + return commitStats(instantTime, writeStatuses, stats, extraMetadata, commitActionType, Collections.emptyMap()); } - public boolean commitStats(String instantTime, List stats, Option> extraMetadata, - String commitActionType, Map> partitionToReplaceFileIds) { + public boolean commitStats(String instantTime, HoodieData writeStatuses, List stats, + Option> extraMetadata, String commitActionType, + Map> partitionToReplaceFileIds) { // Skip the empty commit if not allowed if (!config.allowEmptyCommit() && stats.isEmpty()) { return true; @@ -194,7 +196,7 @@ public boolean commitStats(String instantTime, List stats, Opti lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); try { preCommit(inflightInstant, metadata); - commit(table, commitActionType, instantTime, metadata, stats); + commit(table, commitActionType, instantTime, metadata, stats, writeStatuses); postCommit(table, metadata, instantTime, extraMetadata); LOG.info("Committed " + instantTime); releaseResources(); @@ -217,22 +219,18 @@ public boolean commitStats(String instantTime, List stats, Opti } protected void commit(HoodieTable table, String commitActionType, String instantTime, HoodieCommitMetadata metadata, - List stats) throws IOException { + List stats, HoodieData writeStatuses) throws IOException { LOG.info("Committing " + instantTime + " action " + commitActionType); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); // Finalize write finalizeWrite(table, instantTime, stats); // update Metadata table - writeTableMetadata(table, instantTime, commitActionType, metadata); + writeTableMetadata(table, instantTime, commitActionType, metadata, writeStatuses); activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } - protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { - return createTable(config, hadoopConf, false); - } - - protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, boolean refreshTimeline); + protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf); void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) { try { @@ -265,9 +263,10 @@ protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata met * @param actionType action type of the commit. * @param metadata instance of {@link HoodieCommitMetadata}. */ - protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) { + protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata, + HoodieData writeStatuses) { context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table"); - table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, instantTime, + table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, writeStatuses, instantTime, table.isTableServiceAction(actionType))); } @@ -297,7 +296,7 @@ public void bootstrap(Option> extraMetadata) { */ public void rollbackFailedBootstrap() { LOG.info("Rolling back pending bootstrap if present"); - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); Option instant = Option.fromJavaOptional( inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst()); @@ -465,7 +464,7 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option> extraMetadata) { if (config.inlineTableServices()) { - if (config.isMetadataTableEnabled()) { + if (table.getMetaClient().getTableConfig().isMetadataTableEnabled()) { table.getHoodieView().sync(); } // Do an inline compaction if enabled @@ -529,7 +528,7 @@ protected void autoCleanOnCommit() { * Run any pending compactions. */ public void runAnyPendingCompactions() { - runAnyPendingCompactions(createTable(config, hadoopConf, config.isMetadataTableEnabled())); + runAnyPendingCompactions(createTable(config, hadoopConf)); } /** @@ -539,7 +538,7 @@ public void runAnyPendingCompactions() { * @param comment - Comment for the savepoint */ public void savepoint(String user, String comment) { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); if (table.getCompletedCommitsTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } @@ -563,7 +562,7 @@ public void savepoint(String user, String comment) { * @param comment - Comment for the savepoint */ public void savepoint(String instantTime, String user, String comment) { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); table.savepoint(context, instantTime, user, comment); } @@ -575,7 +574,7 @@ public void savepoint(String instantTime, String user, String comment) { * @return true if the savepoint was deleted successfully */ public void deleteSavepoint(String savepointTime) { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); SavepointHelpers.deleteSavepoint(table, savepointTime); } @@ -590,7 +589,7 @@ public void deleteSavepoint(String savepointTime) { * @return true if the savepoint was restored to successfully */ public void restoreToSavepoint(String savepointTime) { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); SavepointHelpers.validateSavepointPresence(table, savepointTime); restoreToInstant(savepointTime); SavepointHelpers.validateSavepointRestore(table, savepointTime); @@ -673,7 +672,7 @@ public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws H final String restoreInstantTime = HoodieActiveTimeline.createNewInstantTime(); Timer.Context timerContext = metrics.getRollbackCtx(); try { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime); if (timerContext != null) { final long durationInMs = metrics.getDurationInMs(timerContext.stop()); @@ -1091,17 +1090,17 @@ private Option scheduleTableServiceInternal(String instantTime, Option clusteringPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) + Option clusteringPlan = createTable(config, hadoopConf) .scheduleClustering(context, instantTime, extraMetadata); return clusteringPlan.isPresent() ? Option.of(instantTime) : Option.empty(); case COMPACT: LOG.info("Scheduling compaction at instant time :" + instantTime); - Option compactionPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) + Option compactionPlan = createTable(config, hadoopConf) .scheduleCompaction(context, instantTime, extraMetadata); return compactionPlan.isPresent() ? Option.of(instantTime) : Option.empty(); case CLEAN: LOG.info("Scheduling cleaning at instant time :" + instantTime); - Option cleanerPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) + Option cleanerPlan = createTable(config, hadoopConf) .scheduleCleaning(context, instantTime, extraMetadata); return cleanerPlan.isPresent() ? Option.of(instantTime) : Option.empty(); default: diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java index 8f74858669278..6ac9021a111fc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java @@ -193,6 +193,10 @@ public long getTotalErrorRecords() { return totalErrorRecords; } + public boolean isTrackingSuccessRecords() { + return trackSuccessRecords; + } + public void setTotalErrorRecords(long totalErrorRecords) { this.totalErrorRecords = totalErrorRecords; } 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 3e9ac06fe78db..43f164408384f 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 @@ -434,6 +434,15 @@ public class HoodieWriteConfig extends HoodieConfig { .sinceVersion("0.10.0") .withDocumentation("File Id Prefix provider class, that implements `org.apache.hudi.fileid.FileIdPrefixProvider`"); + /** + * If a valid location is specified, a copy of the write config is saved before each operation. + */ + public static final ConfigProperty CONFIG_EXPORT_DIR = ConfigProperty + .key("hoodie.write.config.save.dir") + .defaultValue("/user/hudi/runtime_configs/0.10") + .sinceVersion("0.10.0") + .withDocumentation("The directory where write configs are saved before each operation."); + private ConsistencyGuardConfig consistencyGuardConfig; // Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled @@ -1634,9 +1643,13 @@ public boolean getPushGatewayRandomJobNameSuffix() { } public String getMetricReporterMetricsNamePrefix() { - return getStringOrDefault(HoodieMetricsConfig.METRICS_REPORTER_PREFIX); + String prefix = getStringOrDefault(HoodieMetricsConfig.METRICS_REPORTER_PREFIX); + if (prefix.isEmpty()) { + prefix = getTableName(); + } + return prefix; } - + /** * memory configs. */ @@ -1839,6 +1852,36 @@ public String getFileIdPrefixProviderClassName() { return getString(FILEID_PREFIX_PROVIDER_CLASS); } + /** + * Record index configs. + */ + public boolean createRecordIndex() { + return metadataConfig.createRecordIndex(); + } + + public int getRecordIndexMinFileGroupCount() { + return metadataConfig.getRecordIndexMinFileGroupCount(); + } + + public int getRecordIndexMaxFileGroupCount() { + return metadataConfig.getRecordIndexMaxFileGroupCount(); + } + + public float getRecordIndexGrowthFactor() { + return metadataConfig.getRecordIndexGrowthFactor(); + } + + public long getMaxMetadataFileGroupSizeBytes() { + return metadataConfig.getMaxFileGroupSizeBytes(); + } + + /** + * Directory where write config should be exported before each operation. + */ + public String getConfigExportDir() { + return getString(CONFIG_EXPORT_DIR); + } + public static class Builder { protected final HoodieWriteConfig writeConfig = new HoodieWriteConfig(); @@ -2195,6 +2238,11 @@ public Builder withProperties(Properties properties) { return this; } + public Builder withConfigExportDir(String dir) { + writeConfig.setValue(CONFIG_EXPORT_DIR, dir); + return this; + } + protected void setDefaults() { writeConfig.setDefaultValue(MARKERS_TYPE, getDefaultMarkersType(engineType)); // Check for mandatory properties 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 974f4d546b100..54fae4fb04fa1 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 @@ -24,10 +24,14 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; 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.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.exception.HoodieNotSupportedException; @@ -45,11 +49,30 @@ */ @PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) public abstract class HoodieIndex implements Serializable { + // Metrics for updateLocation + public static final String UPDATE_LOC_DURATION = "update_loc_duration"; + public static final String UPDATE_LOC_NUM_PARTITIONS = "update_loc_num_partitions"; + public static final String TOTAL_UPDATE = "total_updates"; + public static final String TOTAL_INSERT = "total_inserts"; + public static final String TOTAL_DELETE = "total_deletes"; + public static final String TOTAL_WRITES = "total_writes"; - protected final HoodieWriteConfig config; + // Metrics for tagLocation + public static final String TAG_LOC_DURATION = "tag_loc_duration"; + public static final String TAG_LOC_NUM_PARTITIONS = "tag_num_partitions"; + public static final String TAG_LOC_RECORD_COUNT = "tag_count"; + public static final String TAG_LOC_HITS = "tag_hits"; + + // Metric registry + public Option registry = Option.empty(); + + public final HoodieWriteConfig config; protected HoodieIndex(HoodieWriteConfig config) { this.config = config; + if ((config.getTableName() != null) && !config.getTableName().isEmpty()) { + this.registry = Option.of(Registry.getRegistry(config.getTableName() + "." + this.getClass().getSimpleName())); + } } /** @@ -128,7 +151,22 @@ public abstract HoodieData updateLocation( public void close() { } + /** + * Check if the given commit timestamp is valid. + * + * Commit timestamp is considered valid if either it is present in the timeline or is less than the first + * commit ts in the timeline. + * + * @param metaClient + * @param commitTs the commit timestamp to check + * @returns true if the commit timestamp is valid, false otherwise + */ + protected boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) { + HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); + return !commitTimeline.empty() && commitTimeline.containsOrBeforeTimelineStarts(commitTs); + } + public enum IndexType { - HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE + HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE, RECORD_INDEX } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java index a223d695cc03a..07465e05bff59 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -28,6 +28,7 @@ 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.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; @@ -70,6 +71,10 @@ public HoodieBloomIndex(HoodieWriteConfig config, BaseHoodieBloomIndexHelper blo public HoodieData> tagLocation( HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) { + registry.ifPresent(r -> r.add(TAG_LOC_NUM_PARTITIONS, records.getNumPartitions())); + + HoodieTimer timer = new HoodieTimer().startTimer(); + // Step 0: cache the input records if needed if (config.getBloomIndexUseCaching()) { records.persist(new HoodieConfig(config.getProps()) @@ -101,6 +106,10 @@ record -> new ImmutablePair<>(record.getPartitionPath(), record.getRecordKey())) keyFilenamePairs.unpersist(); } + registry.ifPresent(r -> r.add(TAG_LOC_DURATION, timer.endTimer())); + registry.ifPresent(r -> r.add(TAG_LOC_RECORD_COUNT, records.count())); + registry.ifPresent(r -> r.add(TAG_LOC_HITS, keyFilenamePairs.count())); + return taggedRecords; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index d8faa95a44e75..f2b3b9202e025 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -32,6 +32,7 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.marker.WriteMarkersFactory; @@ -116,8 +117,15 @@ protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String this.writeSchema = overriddenSchema.orElseGet(() -> getWriteSchema(config)); this.writeSchemaWithMetaFields = HoodieAvroUtils.addMetadataFields(writeSchema, config.allowOperationMetadataField()); this.timer = new HoodieTimer().startTimer(); - this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), - !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction()); + + // We need to track written records within WriteStatus in two cases: + // 1. When the HoodieIndex being used is not implicit with storage + // 2. If any of the metadata table partitions (record index, etc) which require written record tracking are enabled + final boolean trackSuccessRecords = !hoodieTable.getIndex().isImplicitWithStorage() + || HoodieTableMetadataUtil.needsWriteStatusTracking(config.getMetadataConfig(), hoodieTable.getMetaClient()); + + this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), trackSuccessRecords, + config.getWriteStatusFailureFraction()); this.taskContextSupplier = taskContextSupplier; this.writeToken = makeWriteToken(); } 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 a379b75ea3a55..2a94439984904 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 @@ -25,12 +25,15 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.AbstractHoodieWriteClient; +import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.ConsistencyGuardConfig; 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.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -39,6 +42,7 @@ 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.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableConfig; @@ -53,14 +57,19 @@ import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.metrics.HoodieMetricsConfig; +import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -72,8 +81,8 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -96,7 +105,12 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta // Virtual keys support for metadata table. This Field is // from the metadata payload schema. - private static final String RECORD_KEY_FIELD = HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY; + private static final String RECORD_KEY_FIELD = HoodieMetadataPayload.KEY_FIELD_NAME; + + // Average size of a record saved within the record index. + // Record index has a fixed size schema. This has been calculated based on experiments with default settings + // for block size (4MB), compression (GZ) and disabling the hudi metadata fields. + private static final int RECORD_INDEX_AVERAGE_RECORD_SIZE = 48; protected HoodieWriteConfig metadataWriteConfig; protected HoodieWriteConfig dataWriteConfig; @@ -128,8 +142,10 @@ protected HoodieBackedTableMetadataWriter(Configu this.dataWriteConfig = writeConfig; this.engineContext = engineContext; this.hadoopConf = new SerializableConfiguration(hadoopConf); + this.dataMetaClient = + HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build(); - if (writeConfig.isMetadataTableEnabled()) { + if (dataMetaClient.getTableConfig().isMetadataTableEnabled() || writeConfig.isMetadataTableEnabled()) { this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX; this.metadataWriteConfig = createMetadataWriteConfig(writeConfig); enabled = true; @@ -143,13 +159,10 @@ protected HoodieBackedTableMetadataWriter(Configu ValidationUtils.checkArgument(this.metadataWriteConfig.shouldAutoCommit(), "Auto commit is required for Metadata Table"); ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(), - "File listing cannot be used for Metadata Table"); + "Metadata table cannot be enabled for Metadata Table"); initRegistry(); - this.dataMetaClient = - HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build(); initialize(engineContext, actionMetadata, inflightInstantTimestamp); - initTableMetadata(); } else { enabled = false; this.metrics = Option.empty(); @@ -161,8 +174,6 @@ public HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConf this(hadoopConf, writeConfig, engineContext, Option.empty(), Option.empty()); } - protected abstract void initRegistry(); - /** * Create a {@code HoodieWriteConfig} to use for the Metadata Table. * @@ -174,6 +185,16 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi int minCommitsToKeep = Math.max(writeConfig.getMetadataMinCommitsToKeep(), writeConfig.getMinCommitsToKeep()); int maxCommitsToKeep = Math.max(writeConfig.getMetadataMaxCommitsToKeep(), writeConfig.getMaxCommitsToKeep()); + // File groups in each partitions are fixed at creation time and we do not want them to be split into muliple files + // ever. Hence we use a very large basefile size in metadata table. The actual size of the HFiles created will + // eventually depend on the number of file groups seleted for each partition (See estimateFileGroupCount function) + final long maxHFileSizeBytes = 10 * 1024 * 1024 * 1024L; // 10GB + + // Keeping the log blocks as large as the log files themselves reduces the number of HFile blocks to be checked for + // presence of keys. + final long maxLogFileSizeBytes = writeConfig.getMetadataConfig().getMaxLogFileSize(); + final int maxLogBlockSizeBytes = (int)maxLogFileSizeBytes; + // Create the write config for the metadata table by borrowing options from the main write config. HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) @@ -213,7 +234,9 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withFinalizeWriteParallelism(parallelism) .withAllowMultiWriteOnSameInstant(true) .withKeyGenerator(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) - .withPopulateMetaFields(dataWriteConfig.getMetadataConfig().populateMetaFields()); + .withPopulateMetaFields(dataWriteConfig.getMetadataConfig().populateMetaFields()) + .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(maxHFileSizeBytes) + .logFileMaxSize(maxLogFileSizeBytes).logFileDataBlockMaxSize(maxLogBlockSizeBytes).build()); // RecordKey properties are needed for the metadata table records final Properties properties = new Properties(); @@ -281,7 +304,7 @@ public void initTableMetadata() { this.metadata.close(); } this.metadata = new HoodieBackedTableMetadata(engineContext, dataWriteConfig.getMetadataConfig(), - dataWriteConfig.getBasePath(), dataWriteConfig.getSpillableMapBasePath()); + dataWriteConfig.getBasePath()); this.metadataMetaClient = metadata.getMetadataMetaClient(); } catch (Exception e) { throw new HoodieException("Error initializing metadata table for reads", e); @@ -302,34 +325,63 @@ protected void bootstrapIfNeeded(HoodieEngineCont HoodieTableMetaClient dataMetaClient, Option actionMetadata, Option inflightInstantTimestamp) throws IOException { - HoodieTimer timer = new HoodieTimer().startTimer(); - - boolean exists = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), - HoodieTableMetaClient.METAFOLDER_NAME)); + boolean exists = dataMetaClient.getTableConfig().isMetadataTableEnabled(); boolean rebootstrap = false; - // If the un-synced instants have been archived, then - // the metadata table will need to be bootstrapped again. + // Check if the metadata table need to be re-bootstrapped if (exists) { - final HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()) + metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()) .setBasePath(metadataWriteConfig.getBasePath()).build(); final Option latestMetadataInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); - rebootstrap = isBootstrapNeeded(latestMetadataInstant, actionMetadata); } if (rebootstrap) { - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.REBOOTSTRAP_STR, 1)); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.REBOOTSTRAP_STR, 1)); LOG.info("Deleting Metadata Table directory so that it can be re-bootstrapped"); - dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath()), true); + HoodieTableMetadataUtil.deleteMetadataTable(dataMetaClient, engineContext, false); + dataMetaClient = HoodieTableMetadataUtil.setMetadataPartitionState(dataMetaClient, MetadataPartitionType.FILES, false); 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 timestamp of the last completed instant. + String createInstantTime = dataMetaClient.getActiveTimeline().filterCompletedInstants() + .getReverseOrderedInstants().findFirst().map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + + // 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, dataMetaClient, inflightInstantTimestamp)) { - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); + if (bootstrapFileListings(engineContext, dataMetaClient, createInstantTime, inflightInstantTimestamp)) { + // Set the table config to enable metadata table + dataMetaClient = HoodieTableMetadataUtil.setMetadataPartitionState(dataMetaClient, MetadataPartitionType.FILES, true); + } else if (dataWriteConfig.createRecordIndex()) { + // Since record index is enabled, we need the file listing bootstrap to succeed. + throw new HoodieMetadataException("Failed to initialize file listing for metadata table"); + } else { + // initialization will be attempted next time. + return; + } + } else { + // Open existing metadata table + initTableMetadata(); + } + + // If the metadata table already existed, createInstantTime is already a valid deltacommit on the + // metadata table. Since commits are immutable, we should not use the same timestamp to bootstrap additional + // indexes to prevent any error from wiping out the data from already completed deltacommit at createInstantTime. + if (exists) { + createInstantTime = HoodieTableMetadataUtil.createIndexInitTimestamp(createInstantTime); + } + + // Bootstrap record index if required + if (dataWriteConfig.createRecordIndex() && !dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.RECORD_INDEX)) { + if (bootstrapRecordLevelIndex(engineContext, dataMetaClient, createInstantTime, inflightInstantTimestamp)) { + // Success case + dataMetaClient = HoodieTableMetadataUtil.setMetadataPartitionState(dataMetaClient, MetadataPartitionType.RECORD_INDEX, true); + } else { + // Fatal error as we wont be able to tag records for commit and it may lead to duplicates. + throw new HoodieMetadataException("Failed to initialize record index for metadata table"); } } } @@ -425,9 +477,11 @@ private boolean isCommitRevertedByInFlightAction( * @param dataMetaClient {@code HoodieTableMetaClient} for the dataset. * @param inflightInstantTimestamp */ - private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTableMetaClient dataMetaClient, + private boolean bootstrapFileListings(HoodieEngineContext engineContext, + HoodieTableMetaClient dataMetaClient, String createInstantTime, Option inflightInstantTimestamp) throws IOException { ValidationUtils.checkState(enabled, "Metadata table cannot be initialized as it is not enabled"); + HoodieTimer timer = new HoodieTimer().startTimer(); // We can only bootstrap if there are no pending operations on the dataset List pendingDataInstant = dataMetaClient.getActiveTimeline() @@ -436,16 +490,12 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi .collect(Collectors.toList()); if (!pendingDataInstant.isEmpty()) { - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BOOTSTRAP_ERR_STR, 1)); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.BOOTSTRAP_ERR_STR, 1)); LOG.warn("Cannot bootstrap metadata table as operation(s) are in progress on the dataset: " + Arrays.toString(pendingDataInstant.toArray())); return 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 timestamp of the latest completed action. - String createInstantTime = dataMetaClient.getActiveTimeline().filterCompletedInstants() - .getReverseOrderedInstants().findFirst().map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime); HoodieTableMetaClient.withPropertyBuilder() @@ -460,7 +510,7 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); initTableMetadata(); - initializeFileGroups(dataMetaClient, MetadataPartitionType.FILES, createInstantTime, 1); + initializeFileGroups(MetadataPartitionType.FILES, createInstantTime, 1); // List all partitions in the basePath of the containing dataset LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); @@ -470,10 +520,116 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi // During bootstrap, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out of these // large number of files and calling the existing update(HoodieCommitMetadata) function does not scale well. // Hence, we have a special commit just for the bootstrap scenario. - bootstrapCommit(dirInfoList, createInstantTime); + bootstrapCommit(dirInfoList, createInstantTime, 1); + + final long totalFiles = dirInfoList.stream().mapToLong(d -> d.getTotalFiles()).sum(); + + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.INITIALIZE_FILE_LISTING_TIME_STR, timer.endTimer())); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.READ_FILES_COUNT_STR, totalFiles)); + return true; + } + + /** + * Bootstrap the record index. + * + * @param dataMetaClient {@code HoodieTableMetaClient} for the dataset + * @param createInstantTime InstantTime to use for the commit + * @param inflightInstantTimestamp + * @param partitions List of partitions from which the data files are to be read + */ + private boolean bootstrapRecordLevelIndex(HoodieEngineContext engineContext, HoodieTableMetaClient dataMetaClient, + String createInstantTime, Option inflightInstantTimestamp) throws IOException { + ValidationUtils.checkState(enabled, "Record level index cannot be initialized as Metadata Table is not enabled"); + ValidationUtils.checkState(dataWriteConfig.createRecordIndex(), + "Record level index cannot be initialized as it is not enabled"); + + // Starting two timers to time reading of keys and total time to bootstrap + HoodieTimer timer = new HoodieTimer().startTimer().startTimer(); + + // Collect the list of base files present + final List partitions = metadata.getAllPartitionPaths(); + final HoodieMetadataFileSystemView fsView = new HoodieMetadataFileSystemView(dataMetaClient, + dataMetaClient.getActiveTimeline(), metadata); + final List> partitionBaseFilePairs = new ArrayList<>(); + for (String partition : partitions) { + partitionBaseFilePairs.addAll(fsView.getLatestBaseFiles(partition) + // ignore base files being created due to the inflight operation + .filter(baseFile -> !inflightInstantTimestamp.isPresent() + || !baseFile.getCommitTime().equals(inflightInstantTimestamp.get())) + .map(basefile -> Pair.of(partition, basefile.getFileName())).collect(Collectors.toList())); + } + + LOG.info("Initializing record index from " + partitionBaseFilePairs.size() + " base files in " + + partitions.size() + " partitions"); + + // Collect record keys from the files in parallel + HoodieData records = readRecordKeysFromBaseFiles(engineContext, partitionBaseFilePairs); + records.persist("MEMORY_AND_DISK_SER"); + final long recordCount = records.count(); + + // Initialize the file groups + final int fileGroupCount = estimateFileGroupCount(MetadataPartitionType.RECORD_INDEX.partitionPath(), recordCount, + RECORD_INDEX_AVERAGE_RECORD_SIZE, dataWriteConfig.getRecordIndexMinFileGroupCount(), + dataWriteConfig.getRecordIndexMaxFileGroupCount(), dataWriteConfig.getRecordIndexGrowthFactor()); + initializeFileGroups(MetadataPartitionType.RECORD_INDEX, createInstantTime, fileGroupCount); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.STAT_COUNT_FILE_GROUP, fileGroupCount)); + + if (recordCount > 0) { + LOG.info("Initializing record index with " + recordCount + " mappings"); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.READ_RECORDKEYS_TIME_STR, timer.endTimer())); + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.READ_RECORDKEYS_COUNT_STR, recordCount)); + + // tag and commit records + records = tagRecordsWithLocation(records, MetadataPartitionType.RECORD_INDEX.partitionPath()); + commit(records, createInstantTime, MetadataPartitionType.RECORD_INDEX.partitionPath(), fileGroupCount); + } + + metrics.ifPresent(m -> m.incrementMetric(HoodieMetadataMetrics.INITIALIZE_RECORD_INDEX_TIME_STR, timer.endTimer())); return true; } + /** + * Estimates the file group count to use for a partition. + * + * @param partitionName Name of the partition for which the file group count is to be estimated. + * @param recordCount The number of records expected to be written. + * @param averageRecordSize Average size of each record to be writen. + * @param minFileGroupCount Minimum number of file groups to use. + * @param maxFileGroupCount Maximum number of file groups to use. + * @param growthFactor By what factor are the records (recordCount) expected to grow? + * @param maxFileGroupSizeBytes Maximum size of the file group. + * @return The estimated number of file groups. + */ + private int estimateFileGroupCount(String partitionName, long recordCount, int averageRecordSize, int minFileGroupCount, + int maxFileGroupCount, float growthFactor) { + long maxRequestedFileGroupSizeBytes = dataWriteConfig.getMetadataConfig().getMaxFileGroupSizeBytes(); + int fileGroupCount = 1; + // If a fixed number of file groups are desired + if ((minFileGroupCount == maxFileGroupCount) && (minFileGroupCount != 0)) { + fileGroupCount = minFileGroupCount; + } else { + // Number of records to estimate for + final long expectedNumRecords = (long)Math.ceil((float)recordCount * growthFactor); + // Maximum records that should be written to each file group so it does not go over the size limit required + final long maxRecordsPerFileGroup = maxRequestedFileGroupSizeBytes / Math.max(averageRecordSize, 1L); + final long estimatedFileGroupCount = expectedNumRecords / maxRecordsPerFileGroup; + + if (estimatedFileGroupCount >= maxFileGroupCount) { + fileGroupCount = maxFileGroupCount; + } else if (estimatedFileGroupCount <= minFileGroupCount) { + fileGroupCount = minFileGroupCount; + } else { + fileGroupCount = Math.max(1, (int) estimatedFileGroupCount); + } + } + + LOG.info(String.format("Estimated file group count for metadata partition %s is %d " + + "[recordCount=%d, avgRecordSize=%d, minFileGroupCount=%d, maxFileGroupCount=%d, growthFactor=%f, " + + "maxFileGroupSizeBytes=%d]", partitionName, fileGroupCount, recordCount, averageRecordSize, minFileGroupCount, + maxFileGroupCount, growthFactor, maxRequestedFileGroupSizeBytes)); + return fileGroupCount; + } + /** * Function to find hoodie partitions and list files in them in parallel. * @@ -494,7 +650,7 @@ private List listAllPartitions(HoodieTableMetaClient datasetMetaC // In each round we will list a section of directories int numDirsToList = Math.min(fileListingParallelism, pathsToList.size()); // List all directories in parallel - List processedDirectories = engineContext.map(pathsToList.subList(0, numDirsToList), path -> { + List processedDirectories = engineContext.map(pathsToList.subList(0, numDirsToList), path -> { FileSystem fs = path.getFileSystem(conf.get()); String relativeDirPath = FSUtils.getRelativePartitionPath(new Path(datasetBasePath), path); return new DirectoryInfo(relativeDirPath, fs.listStatus(path)); @@ -535,13 +691,11 @@ private List listAllPartitions(HoodieTableMetaClient datasetMetaC * All FileGroups for a given metadata partition has a fixed prefix as per the {@link MetadataPartitionType#getFileIdPrefix()}. * Each file group is suffixed with 4 digits with increments of 1 starting with 0000. * - * Lets say we configure 10 file groups for record level index partittion, and prefix as "record-index-bucket-" + * Lets say we configure 10 file groups for record index partittion, and prefix as "record-index-bucket-" * File groups will be named as : * record-index-bucket-0000, .... -> ..., record-index-bucket-0009 */ - private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, - int fileGroupCount) throws IOException { - + private void initializeFileGroups(MetadataPartitionType metadataPartition, String instantTime, int fileGroupCount) throws IOException { final HashMap blockHeader = new HashMap<>(); blockHeader.put(HeaderMetadataType.INSTANT_TIME, instantTime); // Archival of data table has a dependency on compaction(base files) in metadata table. @@ -552,7 +706,8 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata LOG.info(String.format("Creating %d file groups for partition %s with base fileId %s at instant time %s", fileGroupCount, metadataPartition.partitionPath(), metadataPartition.getFileIdPrefix(), instantTime)); for (int i = 0; i < fileGroupCount; ++i) { - final String fileGroupFileId = String.format("%s%04d", metadataPartition.getFileIdPrefix(), i); + // Suffix of -0 is used as the file-index (see HoodieCreateHandle) as we want a single file per fileGroup + final String fileGroupFileId = String.format("%s%04d-0", metadataPartition.getFileIdPrefix(), i); try { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), metadataPartition.partitionPath())) @@ -560,7 +715,7 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata .withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION) .withFileSize(0L) .withSizeThreshold(metadataWriteConfig.getLogFileMaxSize()) - .withFs(dataMetaClient.getFs()) + .withFs(metadataMetaClient.getFs()) .withRolloverLogWriteToken(HoodieLogFormat.DEFAULT_WRITE_TOKEN) .withLogWriteToken(HoodieLogFormat.DEFAULT_WRITE_TOKEN) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); @@ -573,25 +728,16 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata } /** - * Interface to assist in converting commit metadata to List of HoodieRecords to be written to metadata table. - * Updates of different commit metadata uses the same method to convert to HoodieRecords and hence. - */ - private interface ConvertMetadataFunction { - List convertMetadata(); - } - - /** - * Processes commit metadata from data table and commits to metadata table. - * @param instantTime instant time of interest. - * @param convertMetadataFunction converter function to convert the respective metadata to List of HoodieRecords to be written to metadata table. - * @param type of commit metadata. - * @param canTriggerTableService true if table services can be triggered. false otherwise. + * Commits updates to the file listing partition. + * + * @param instantTime instant time to use for the commit + * @param recordList List of HoodieRecords to be written to file listing partition. + * @param canTriggerTableService true if table service can be triggered, false otherwise. */ - private void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction, boolean canTriggerTableService) { - if (enabled && metadata != null) { - List records = convertMetadataFunction.convertMetadata(); - commit(engineContext.parallelize(records, 1), MetadataPartitionType.FILES.partitionPath(), instantTime, canTriggerTableService); - } + private void commitFileListingUpdate(String instantTime, List recordList, boolean canTriggerTableService) { + HoodieData records = engineContext.parallelize(recordList, 1); + records = tagRecordsWithLocation(records, MetadataPartitionType.FILES.partitionPath()); + commit(records, instantTime, canTriggerTableService); } /** @@ -601,8 +747,17 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co * @param isTableServiceAction {@code true} if commit metadata is pertaining to a table service. {@code false} otherwise. */ @Override - public void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean isTableServiceAction) { - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime), !isTableServiceAction); + public void update(HoodieCommitMetadata commitMetadata, HoodieData writeStatus, String instantTime, + boolean isTableServiceAction) { + // Update file listing (always updated and enabled if metadata table is enabled) + HoodieData records = engineContext.parallelize(HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime), 1); + records = tagRecordsWithLocation(records, MetadataPartitionType.FILES.partitionPath()); + + if (dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.RECORD_INDEX)) { + records = records.union(getRecordIndexUpdates(writeStatus)); + } + + commit(records, instantTime, !isTableServiceAction); } /** @@ -613,8 +768,7 @@ public void update(HoodieCommitMetadata commitMetadata, String instantTime, bool */ @Override public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime), - false); + commitFileListingUpdate(instantTime, HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime), false); } /** @@ -625,7 +779,7 @@ public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { */ @Override public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), + commitFileListingUpdate(instantTime, HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), restoreMetadata, instantTime, metadata.getSyncedInstantTime()), false); } @@ -637,23 +791,25 @@ public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { */ @Override public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) { - if (enabled && metadata != null) { - // Is this rollback of an instant that has been synced to the metadata table? - String rollbackInstant = rollbackMetadata.getCommitsRollback().get(0); - boolean wasSynced = metadataMetaClient.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. - // Revisit this logic to relax the compaction fencing : https://issues.apache.org/jira/browse/HUDI-2458 - Option latestCompaction = metadata.getLatestCompactionTime(); - if (latestCompaction.isPresent()) { - wasSynced = HoodieTimeline.compareTimestamps(rollbackInstant, HoodieTimeline.LESSER_THAN_OR_EQUALS, latestCompaction.get()); - } - } + if (!enabled || metadata == null) { + // Possible case of bootstrap could not happen + return; + } - List records = HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), rollbackMetadata, instantTime, - metadata.getSyncedInstantTime(), wasSynced); - commit(engineContext.parallelize(records, 1), MetadataPartitionType.FILES.partitionPath(), instantTime, false); + // Is this rollback of an instant that has been synced to the metadata table? + String rollbackInstant = rollbackMetadata.getCommitsRollback().get(0); + boolean wasSynced = metadataMetaClient.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. + // Revisit this logic to relax the compaction fencing : https://issues.apache.org/jira/browse/HUDI-2458 + Option latestCompaction = metadata.getLatestCompactionTime(); + if (latestCompaction.isPresent()) { + wasSynced = HoodieTimeline.compareTimestamps(rollbackInstant, HoodieTimeline.LESSER_THAN_OR_EQUALS, latestCompaction.get()); + } } + + commitFileListingUpdate(instantTime, HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), + rollbackMetadata, instantTime, metadata.getSyncedInstantTime(), wasSynced), false); } @Override @@ -663,14 +819,54 @@ public void close() throws Exception { } } + protected abstract void initRegistry(); + /** * Commit the {@code HoodieRecord}s to Metadata Table as a new delta-commit. - * @param records The HoodieData of records to be written. - * @param partitionName The partition to which the records are to be written. - * @param instantTime The timestamp to use for the deltacommit. + * + * @param records The HoodieData of records to be written. + * @param instantTime The timestamp to use for the commit. * @param canTriggerTableService true if table services can be scheduled and executed. false otherwise. */ - protected abstract void commit(HoodieData records, String partitionName, String instantTime, boolean canTriggerTableService); + protected abstract void commit(HoodieData records, String instantTime, boolean canTriggerTableService); + + /** + * Commit the {@code HoodieRecord}s to Metadata Table. + * + * This function is called during the initialization of a partition within metadata table and hence a large number + * of records may be written. Therefore, the implementation should try to optimize the write (e.g bulkInsertPrepped + * rather than upsertPrepped) if possible. + * + * @param records The HoodieData of records to be written. + * @param instantTime The timestamp to use for the commit. + * @param partitionName The partition to which the records will be written. + * @param fileGroupCount The maximum number of file groups to which the records will be written. + */ + protected void commit(HoodieData records, String instantTime, String partitionName, int fileGroupCount) { + commit(records, instantTime, false); + } + + /** + * Tag each record with the location in the given partition. + * + * The record is tagged with respective file slice's location based on its record key. + */ + private HoodieData tagRecordsWithLocation(HoodieData records, String partitionName) { + List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, + partitionName); + if (fileSlices.isEmpty()) { + // TODO: This can be optimized to create the fileSlices when required and to write to the base files first + // so that we can avoid the compaction post bootstrap. + fileSlices.addAll(HoodieTableMetadataUtil.getBootstrappedFileSlices(metadataMetaClient, partitionName)); + } + ValidationUtils.checkArgument(!fileSlices.isEmpty(), "FileSlices should be present for partition " + partitionName); + + return records.map(r -> { + FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), fileSlices.size())); + r.setCurrentLocation(new HoodieRecordLocation(slice.getBaseInstantTime(), slice.getFileId())); + return r; + }); + } /** * Perform a compaction on the Metadata Table. @@ -700,7 +896,7 @@ protected void compactIfNecessary(AbstractHoodieWriteClient writeClient, String // 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"; + final String compactionInstantTime = HoodieTableMetadataUtil.createCompactionTimestamp(latestDeltacommitTime); if (writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty())) { writeClient.compact(compactionInstantTime); } @@ -723,7 +919,7 @@ protected void cleanIfNecessary(AbstractHoodieWriteClient writeClient, String in // 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"); + writeClient.clean(HoodieTableMetadataUtil.createCleanTimestamp(instantTime)); } /** @@ -731,20 +927,17 @@ protected void cleanIfNecessary(AbstractHoodieWriteClient writeClient, String in * other regular commits. * */ - protected void bootstrapCommit(List partitionInfoList, String createInstantTime) { + protected void bootstrapCommit(List partitionInfoList, String createInstantTime, int fileGroupCount) { List partitions = partitionInfoList.stream().map(p -> p.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : p.getRelativePath()).collect(Collectors.toList()); final int totalFiles = partitionInfoList.stream().mapToInt(p -> p.getTotalFiles()).sum(); // Record which saves the list of all partitions HoodieRecord allPartitionRecord = HoodieMetadataPayload.createPartitionListRecord(partitions); - if (partitions.isEmpty()) { - // in case of boostrapping of a fresh table, there won't be any partitions, but we need to make a boostrap commit - commit(engineContext.parallelize(Collections.singletonList(allPartitionRecord), 1), MetadataPartitionType.FILES.partitionPath(), createInstantTime, false); - return; - } HoodieData partitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); + if (!partitionInfoList.isEmpty()) { + engineContext.setJobStatus(this.getClass().getSimpleName(), "Creating records for metadata table"); HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { // Record which saves files within a partition return HoodieMetadataPayload.createPartitionFilesRecord( @@ -755,7 +948,8 @@ protected void bootstrapCommit(List partitionInfoList, String cre LOG.info("Committing " + partitions.size() + " partitions and " + totalFiles + " files to metadata"); ValidationUtils.checkState(partitionRecords.count() == (partitions.size() + 1)); - commit(partitionRecords, MetadataPartitionType.FILES.partitionPath(), createInstantTime, false); + partitionRecords = tagRecordsWithLocation(partitionRecords, MetadataPartitionType.FILES.partitionPath()); + commit(partitionRecords, createInstantTime, MetadataPartitionType.FILES.partitionPath(), fileGroupCount); } /** @@ -818,4 +1012,97 @@ Map getFileNameToSizeMap() { return filenameToSizeMap; } } + + /** + * Read the record keys from base files in partitions and return records. + * + * @param engineContext + * @param partitionBaseFilePairs Map of partitionName to base file name. + */ + private HoodieData readRecordKeysFromBaseFiles(HoodieEngineContext engineContext, + List> partitionBaseFilePairs) { + if (partitionBaseFilePairs.isEmpty()) { + return engineContext.emptyHoodieData(); + } + + // TODO: follow the same model as bootstrap of file listing - limit the parallelism to fileListingParallelism + engineContext.setJobStatus(this.getClass().getSimpleName(), "Record Index: reading record keys from base files"); + return engineContext.parallelize(partitionBaseFilePairs, partitionBaseFilePairs.size()).flatMap(p -> { + final String partition = p.getKey(); + final String filename = p.getValue(); + Path dataFilePath = new Path(dataWriteConfig.getBasePath(), partition + Path.SEPARATOR + filename); + + final String fileId = FSUtils.getFileId(filename); + final String instantTime = FSUtils.getCommitTime(filename); + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), dataFilePath); + Iterator recordKeyIterator = reader.getRecordKeyIterator(); + + return new Iterator() { + @Override + public boolean hasNext() { + return recordKeyIterator.hasNext(); + } + + @Override + public HoodieRecord next() { + return HoodieMetadataPayload.createRecordIndexUpdate(recordKeyIterator.next(), partition, fileId, + instantTime); + } + }; + }); + } + + /** + * Return records that represent update to the record index due to a write to the dataset. + * + * @param writeStatues (@code WriteStatus} from the write + */ + private HoodieData getRecordIndexUpdates(HoodieData writeStatuses) { + HoodieTimer timer = new HoodieTimer().startTimer(); + Registry registry = Registry.getRegistry(dataWriteConfig.getTableName() + ".SparkMetadataTableRecordIndex"); + + HoodieData records = writeStatuses.flatMap(writeStatus -> { + long numUpdates = 0; + long numInserts = 0; + long numDeletes = 0; + + List recordList = new LinkedList<>(); + for (HoodieRecord writtenRecord : writeStatus.getWrittenRecords()) { + if (!writeStatus.isErrored(writtenRecord.getKey())) { + HoodieRecord hoodieRecord; + HoodieKey key = writtenRecord.getKey(); + Option newLocation = writtenRecord.getNewLocation(); + if (newLocation.isPresent()) { + if (writtenRecord.getCurrentLocation() != null) { + // This is an update, no need to update index + // TODO: This may be required for clustering usecases + ++numUpdates; + continue; + } + + hoodieRecord = HoodieMetadataPayload.createRecordIndexUpdate(key.getRecordKey(), key.getPartitionPath(), + newLocation.get().getFileId(), newLocation.get().getInstantTime()); + ++numInserts; + } else { + // Delete existing index for a deleted record + hoodieRecord = HoodieMetadataPayload.createRecordIndexDelete(key.getRecordKey()); + ++numDeletes; + } + + recordList.add(hoodieRecord); + } + } + + registry.add(HoodieIndex.TOTAL_UPDATE, numUpdates); + registry.add(HoodieIndex.TOTAL_INSERT, numInserts); + registry.add(HoodieIndex.TOTAL_DELETE, numDeletes); + registry.add(HoodieIndex.TOTAL_WRITES, numUpdates + numInserts + numDeletes); + return recordList.iterator(); + }); + + registry.add(HoodieIndex.UPDATE_LOC_DURATION, timer.endTimer()); + registry.add(HoodieIndex.UPDATE_LOC_NUM_PARTITIONS, writeStatuses.getNumPartitions()); + + return tagRecordsWithLocation(records, MetadataPartitionType.RECORD_INDEX.partitionPath()); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java index 4ec143bf06789..332be73b14f57 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java @@ -42,7 +42,7 @@ public HoodieTableMetadataKeyGenerator(TypedProperties config) { @Override public String getRecordKey(GenericRecord record) { - return KeyGenUtils.getRecordKey(record, HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY, isConsistentLogicalTimestampEnabled()); + return KeyGenUtils.getRecordKey(record, HoodieMetadataPayload.KEY_FIELD_NAME, isConsistentLogicalTimestampEnabled()); } @Override 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 4f5ac027c91eb..c4f6dd1dcecc6 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 @@ -21,6 +21,8 @@ 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.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieCommitMetadata; import java.io.Serializable; @@ -33,11 +35,13 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { /** * Update the metadata table due to a COMMIT operation. * @param commitMetadata commit metadata of the operation of interest. + * @param writeStatus write status associated with this commit operation. * @param instantTime instant time of the commit. - * @param isTableServiceAction true if caller is a table service. false otherwise. Only regular write operations can trigger metadata table services and this argument - * will assist in this. + * @param isTableServiceAction true if caller is a table service. false otherwise. Only regular write operations can + * trigger metadata table services and this argument will assist in this. */ - void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean isTableServiceAction); + void update(HoodieCommitMetadata commitMetadata, HoodieData writeStatus, String instantTime, + boolean isTableServiceAction); /** * Update the metadata table due to a CLEAN operation. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java index b570f512f371d..272b2da8ea5ab 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java @@ -58,6 +58,7 @@ private Metrics(HoodieWriteConfig metricConfig) { private void reportAndCloseReporter() { try { + LOG.info("Reporting and closing metrics"); registerHoodieCommonMetrics(); reporter.report(); if (getReporter() != null) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java index fd1558a8232bb..b910b9fc182c5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java @@ -18,6 +18,12 @@ package org.apache.hudi.table; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.apache.hudi.common.fs.FSUtils; + /** * Repartition input records into at least expected number of output spark partitions. It should give below guarantees - * Output spark partition will have records from only one hoodie partition. - Average records per output spark @@ -38,4 +44,14 @@ public interface BulkInsertPartitioner { * @return {@code true} if the records within a partition are sorted; {@code false} otherwise. */ boolean arePartitionRecordsSorted(); + + /** + * Returns the fileID prefixes to use when writing records for partitions. + * + * @param numPartitions Number of partitions for which the fileID prefix are required + * @returns A list of numPartitions fileID prefixes. + */ + default List generateFileIDPfxs(int numPartitions) { + return IntStream.range(0, numPartitions).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList()); + } } 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 ca34f8cc1d221..e1503aec90afc 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 @@ -49,7 +49,6 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewManager; -import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; @@ -105,7 +104,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; @@ -117,10 +116,9 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(config.getMetadataConfig().getProps()) .build(); - this.metadata = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath(), - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue()); + this.metadataReader = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath()); - 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(); @@ -130,7 +128,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; } @@ -453,7 +451,7 @@ public abstract Option scheduleRollback(HoodieEngineContext String instantTime, HoodieInstant instantToRollback, boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers); - + /** * Rollback the (inflight/committed) record changes with the given commit time. *
@@ -744,6 +742,10 @@ public final Option getMetadataWriter(String triggeri
     return getMetadataWriter(triggeringInstantTimestamp, Option.empty());
   }
 
+  public HoodieTableMetadata getMetadataReader() {
+    return metadataReader;
+  }
+
   /**
    * Check if action type is a table service.
    * @param actionType action type of interest.
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 58e03f5f39fc4..1b63244b463df 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
@@ -36,7 +36,6 @@
 import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
 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.util.CollectionUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
@@ -208,9 +207,9 @@ private Stream getInstantsToArchive() {
 
     // If metadata table is enabled, do not archive instants which are more recent that the last compaction on the
     // metadata table.
-    if (config.isMetadataTableEnabled()) {
+    if (table.getMetaClient().getTableConfig().isMetadataTableEnabled()) {
       try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(),
-          config.getBasePath(), FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) {
+          config.getBasePath())) {
         Option latestCompactionTime = tableMetadata.getLatestCompactionTime();
         if (!latestCompactionTime.isPresent()) {
           LOG.info("Not archiving as there is no compaction yet on the metadata table");
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 221f970cb5132..a71c95193a2b8 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
@@ -24,6 +24,8 @@
 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.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieRecordPayload;
@@ -56,9 +58,8 @@ public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config,
    * Writes commits metadata to table metadata.
    * @param metadata commit metadata of interest.
    */
-  protected final void writeTableMetadata(HoodieCommitMetadata metadata, String actionType) {
-    table.getMetadataWriter(instantTime).ifPresent(w -> w.update(
-        metadata, instantTime, table.isTableServiceAction(actionType)));
+  protected final void writeTableMetadata(HoodieCommitMetadata metadata, HoodieData writeStatus, String actionType) {
+    table.getMetadataWriter(instantTime).ifPresent(w -> w.update(metadata, writeStatus, instantTime, table.isTableServiceAction(actionType)));
   }
 
   /**
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java
index 9813b2b659677..0df40b53cc1f8 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java
@@ -240,8 +240,11 @@ public HoodieCleanMetadata execute() {
             LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e);
           }
         }
+        table.getMetaClient().reloadActiveTimeline();
+        if (table.getMetaClient().getTableConfig().isMetadataTableEnabled()) {
+          table.getHoodieView().sync();
+        }
       });
-      table.getMetaClient().reloadActiveTimeline();
     }
     // return the last clean metadata for now
     // TODO (NA) : Clean only the earliest pending clean just like how we do for other table services
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java
index 964859c0ae07d..4920d8664c0a7 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java
@@ -21,10 +21,11 @@
 
 import org.apache.hudi.common.config.ConfigProperty;
 import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.metadata.HoodieTableMetadataUtil;
 
-import java.util.Collections;
+import java.util.Hashtable;
 import java.util.Map;
 
 /**
@@ -34,12 +35,14 @@ public class ThreeToTwoDowngradeHandler implements DowngradeHandler {
 
   @Override
   public Map downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime, BaseUpgradeDowngradeHelper upgradeDowngradeHelper) {
-    if (config.isMetadataTableEnabled()) {
-      // Metadata Table in version 3 is synchronous and in version 2 is asynchronous. Downgrading to asynchronous
-      // 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.
-      HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context);
-    }
-    return Collections.emptyMap();
+    // Metadata Table in version 3 is synchronous and in version 2 is asynchronous. Downgrading to asynchronous
+    // 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.
+    HoodieTableMetadataUtil.deleteMetadataTable(upgradeDowngradeHelper.getTable(config, context).getMetaClient(), context, false);
+    Map tablePropsToAdd = new Hashtable<>();
+    // Clear all the metadata table partitions
+    tablePropsToAdd.put(HoodieTableConfig.METADATA_TABLE_PARTITIONS, "");
+
+    return tablePropsToAdd;
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java
index bff3788d56cfe..5466b739a611f 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java
@@ -36,13 +36,8 @@
 public class TwoToThreeUpgradeHandler implements UpgradeHandler {
   @Override
   public Map upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime, BaseUpgradeDowngradeHelper upgradeDowngradeHelper) {
-    if (config.isMetadataTableEnabled()) {
-      // Metadata Table in version 2 is asynchronous and in version 3 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.
-      HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context);
-    }
     Map tablePropsToAdd = new Hashtable<>();
+
     tablePropsToAdd.put(HoodieTableConfig.URL_ENCODE_PARTITIONING, config.getStringOrDefault(HoodieTableConfig.URL_ENCODE_PARTITIONING));
     tablePropsToAdd.put(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE, config.getStringOrDefault(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE));
     String keyGenClassName = Option.ofNullable(config.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME))
@@ -50,6 +45,17 @@ public Map upgrade(HoodieWriteConfig config, HoodieEngin
     ValidationUtils.checkState(keyGenClassName != null, String.format("Missing config: %s or %s",
             HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, HoodieWriteConfig.KEYGENERATOR_CLASS_NAME));
     tablePropsToAdd.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, keyGenClassName);
+
+    // Metadata Table in version 2 is asynchronous and in version 3 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.
+    if (upgradeDowngradeHelper != null) {
+      HoodieTableMetadataUtil.deleteMetadataTable(upgradeDowngradeHelper.getTable(config, context).getMetaClient(),
+          context, false);
+    }
+    // Clear all the metadata table partitions
+    tablePropsToAdd.put(HoodieTableConfig.METADATA_TABLE_PARTITIONS, "");
+
     return tablePropsToAdd;
   }
 }
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java
index fa0f5df61b183..5a481ad60af9f 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java
@@ -22,6 +22,7 @@
 import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
 import org.apache.hudi.avro.model.HoodieRestoreMetadata;
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.common.data.HoodieList;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -33,6 +34,7 @@
 import org.apache.hadoop.fs.FileSystem;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -77,7 +79,7 @@ public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationTy
     HoodieCommitMetadata commitMetadata = super.doWriteOperation(commitTime, operationType, newPartitionsToAdd,
         partitionToFilesNameLengthMap, bootstrap, createInflightCommit);
     if (writer != null && !createInflightCommit) {
-      writer.update(commitMetadata, commitTime, false);
+      writer.update(commitMetadata, HoodieList.of(Collections.EMPTY_LIST), commitTime, false);
     }
     return commitMetadata;
   }
@@ -86,7 +88,7 @@ public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationTy
   public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException {
     super.moveInflightCommitToComplete(instantTime, metadata);
     if (writer != null) {
-      writer.update(metadata, instantTime, false);
+      writer.update(metadata, HoodieList.of(Collections.EMPTY_LIST), instantTime, false);
     }
     return this;
   }
@@ -94,7 +96,7 @@ public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCo
   public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata, boolean ignoreWriter) throws IOException {
     super.moveInflightCommitToComplete(instantTime, metadata);
     if (!ignoreWriter && writer != null) {
-      writer.update(metadata, instantTime, false);
+      writer.update(metadata, HoodieList.of(Collections.EMPTY_LIST), instantTime, false);
     }
     return this;
   }
@@ -103,7 +105,7 @@ public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCo
   public HoodieTestTable moveInflightCompactionToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException {
     super.moveInflightCompactionToComplete(instantTime, metadata);
     if (writer != null) {
-      writer.update(metadata, instantTime, true);
+      writer.update(metadata, HoodieList.of(Collections.EMPTY_LIST), instantTime, true);
     }
     return this;
   }
@@ -120,7 +122,7 @@ public HoodieCleanMetadata doClean(String commitTime, Map parti
   public HoodieTestTable addCompaction(String instantTime, HoodieCommitMetadata commitMetadata) throws Exception {
     super.addCompaction(instantTime, commitMetadata);
     if (writer != null) {
-      writer.update(commitMetadata, instantTime, true);
+      writer.update(commitMetadata, HoodieList.of(Collections.EMPTY_LIST), instantTime, true);
     }
     return this;
   }
@@ -151,7 +153,7 @@ public HoodieTestTable addReplaceCommit(
       HoodieReplaceCommitMetadata completeReplaceMetadata) throws Exception {
     super.addReplaceCommit(instantTime, requestedReplaceMetadata, inflightReplaceMetadata, completeReplaceMetadata);
     if (writer != null) {
-      writer.update(completeReplaceMetadata, instantTime, true);
+      writer.update(completeReplaceMetadata, HoodieList.of(Collections.EMPTY_LIST), instantTime, true);
     }
     return this;
   }
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java
index 35928dc7cf319..0f2a6e8d915ee 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java
@@ -19,10 +19,13 @@
 
 package org.apache.hudi.table.upgrade;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.common.config.ConfigProperty;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
 import org.apache.hudi.keygen.KeyGenerator;
 
 import org.junit.jupiter.api.BeforeEach;
@@ -39,6 +42,7 @@
 class TestTwoToThreeUpgradeHandler {
 
   HoodieWriteConfig config;
+  HoodieEngineContext context;
 
   @BeforeEach
   void setUp() {
@@ -47,6 +51,7 @@ void setUp() {
         .withPath("/foo")
         .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
         .build();
+    context = new HoodieLocalEngineContext(new Configuration());
   }
 
   @ParameterizedTest
@@ -54,7 +59,7 @@ void setUp() {
   void upgradeHandlerShouldRetrieveKeyGeneratorConfig(String keyGenConfigKey) {
     config.setValue(keyGenConfigKey, KeyGenerator.class.getName());
     TwoToThreeUpgradeHandler handler = new TwoToThreeUpgradeHandler();
-    Map kv = handler.upgrade(config, null, null, null);
+    Map kv = handler.upgrade(config, context, null, null);
     assertEquals(KeyGenerator.class.getName(), kv.get(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME));
   }
 
@@ -62,7 +67,7 @@ void upgradeHandlerShouldRetrieveKeyGeneratorConfig(String keyGenConfigKey) {
   void upgradeHandlerShouldThrowWhenKeyGeneratorNotSet() {
     TwoToThreeUpgradeHandler handler = new TwoToThreeUpgradeHandler();
     Throwable t = assertThrows(IllegalStateException.class, () -> handler
-        .upgrade(config, null, null, null));
+        .upgrade(config, context, null, null));
     assertTrue(t.getMessage().startsWith("Missing config:"));
   }
 }
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 4108ba425e8ca..64b8942803368 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
@@ -19,6 +19,7 @@
 package org.apache.hudi.client;
 
 import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.data.HoodieList;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
@@ -109,12 +110,11 @@ protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) {
   @Override
   public boolean commit(String instantTime, List writeStatuses, Option> extraMetadata, String commitActionType, Map> partitionToReplacedFileIds) {
     List writeStats = writeStatuses.parallelStream().map(WriteStatus::getStat).collect(Collectors.toList());
-    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+    return commitStats(instantTime, HoodieList.of(writeStatuses), writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
   }
 
   @Override
-  protected HoodieTable>, List, List> createTable(HoodieWriteConfig config, Configuration hadoopConf,
-                                                                                                  boolean refreshTimeline) {
+  protected HoodieTable>, List, List> createTable(HoodieWriteConfig config, Configuration hadoopConf) {
     return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
   }
 
@@ -256,10 +256,11 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp
   }
 
   @Override
-  protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) {
+  protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata,
+      HoodieData writeStatuses) {
     this.metadataWriterOption.ifPresent(w -> {
       w.initTableMetadata(); // refresh the timeline
-      w.update(metadata, instantTime, getHoodieTable().isTableServiceAction(actionType));
+      w.update(metadata, writeStatuses, instantTime, getHoodieTable().isTableServiceAction(actionType));
     });
   }
 
@@ -371,7 +372,7 @@ public void completeCompaction(
       // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a
       // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition.
       table.getMetadataWriter(compactionInstant.getTimestamp()).ifPresent(
-          w -> w.update(metadata, compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction())));
+          w -> w.update(metadata, HoodieList.of(null), compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction())));
       LOG.info("Committing Compaction {} finished with result {}.", compactionCommitTime, metadata);
       CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata);
     } finally {
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java
index af9fee0688049..290365a08f51d 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java
@@ -24,9 +24,7 @@
 import org.apache.hudi.common.data.HoodieData;
 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.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.Option;
@@ -41,7 +39,6 @@
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
-import java.util.stream.Collectors;
 
 public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
 
@@ -79,7 +76,7 @@  FlinkHoodieBackedTableMetadataWriter(Configuratio
   protected void initRegistry() {
     if (metadataWriteConfig.isMetricsOn()) {
       // should support executor metrics
-      Registry registry = Registry.getRegistry("HoodieMetadata");
+      Registry registry = Registry.getRegistry(HoodieTableMetadata.METRIC_REGISTRY_NAME);
       this.metrics = Option.of(new HoodieMetadataMetrics(registry));
     } else {
       this.metrics = Option.empty();
@@ -101,10 +98,9 @@ protected  void initialize(HoodieEngineContext eng
   }
 
   @Override
-  protected void commit(HoodieData hoodieDataRecords, String partitionName, String instantTime, boolean canTriggerTableService) {
+  protected void commit(HoodieData hoodieDataRecords, String instantTime, boolean canTriggerTableService) {
     ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
-    List records = (List) hoodieDataRecords.get();
-    List recordList = prepRecords(records, partitionName, 1);
+    List recordList = (List) hoodieDataRecords.get();
 
     try (HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(engineContext, metadataWriteConfig)) {
       if (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime)) {
@@ -124,7 +120,7 @@ protected void commit(HoodieData hoodieDataRecords, String partiti
         metadataMetaClient.reloadActiveTimeline();
       }
 
-      List statuses = records.size() > 0
+      List statuses = recordList.size() > 0
           ? writeClient.upsertPreppedRecords(recordList, instantTime)
           : Collections.emptyList();
       statuses.forEach(writeStatus -> {
@@ -147,21 +143,4 @@ protected void commit(HoodieData hoodieDataRecords, String partiti
     // Update total size of the metadata and count of base/log files
     metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata));
   }
-
-  /**
-   * Tag each record with the location in the given partition.
-   *
-   * The record is tagged with respective file slice's location based on its record key.
-   */
-  private List prepRecords(List records, String partitionName, int numFileGroups) {
-    List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, partitionName);
-    ValidationUtils.checkArgument(fileSlices.size() == numFileGroups, String.format("Invalid number of file groups: found=%d, required=%d", fileSlices.size(), numFileGroups));
-
-    return records.stream().map(r -> {
-      FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), numFileGroups));
-      final String instantTime = slice.isEmpty() ? "I" : "U";
-      r.setCurrentLocation(new HoodieRecordLocation(instantTime, slice.getFileId()));
-      return r;
-    }).collect(Collectors.toList());
-  }
 }
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java
index 164b00e2d6ce4..9a92bde7e66e7 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java
@@ -60,13 +60,6 @@ public static  HoodieFlinkTable create(HoodieW
   public static  HoodieFlinkTable create(HoodieWriteConfig config,
                                                                            HoodieFlinkEngineContext context,
                                                                            HoodieTableMetaClient metaClient) {
-    return HoodieFlinkTable.create(config, context, metaClient, config.isMetadataTableEnabled());
-  }
-
-  public static  HoodieFlinkTable create(HoodieWriteConfig config,
-                                                                           HoodieFlinkEngineContext context,
-                                                                           HoodieTableMetaClient metaClient,
-                                                                           boolean refreshTimeline) {
     final HoodieFlinkTable hoodieFlinkTable;
     switch (metaClient.getTableType()) {
       case COPY_ON_WRITE:
@@ -78,7 +71,7 @@ public static  HoodieFlinkTable create(HoodieW
       default:
         throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
     }
-    if (refreshTimeline) {
+    if (metaClient.getTableConfig().isMetadataTableEnabled()) {
       hoodieFlinkTable.getHoodieView().sync();
     }
     return hoodieFlinkTable;
@@ -102,7 +95,7 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con
   @Override
   public  Option getMetadataWriter(String triggeringInstantTimestamp,
                                                                                             Option actionMetadata) {
-    if (config.isMetadataTableEnabled()) {
+    if (getMetaClient().getTableConfig().isMetadataTableEnabled() || config.isMetadataTableEnabled()) {
       return Option.of(FlinkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config,
           context, actionMetadata, Option.of(triggeringInstantTimestamp)));
     } else {
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 5dfa511a8823f..e35b189f6f8eb 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
@@ -19,6 +19,7 @@
 package org.apache.hudi.table.action.commit;
 
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieList;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieKey;
@@ -147,7 +148,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta
       HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
           extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
 
-      writeTableMetadata(metadata, actionType);
+      writeTableMetadata(metadata, HoodieList.of(result.getWriteStatuses()), actionType);
 
       activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
           Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java
index 7b4e3b675ea05..54d970519b46a 100644
--- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java
+++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java
@@ -103,7 +103,7 @@ private HoodieWriteConfig makeConfig(boolean rangePruning, boolean treeFiltering
   public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
     HoodieBloomIndex index = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance());
-    HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient, false);
+    HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient);
     HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA);
 
     // Create some partitions, and put some files
diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java
index 212187b2d7552..42e4b18c9491c 100644
--- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java
+++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java
@@ -86,13 +86,12 @@ public boolean commit(String instantTime,
                         String commitActionType,
                         Map> partitionToReplacedFileIds) {
     List writeStats = writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList());
-    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+    return commitStats(instantTime, HoodieList.of(writeStatuses), writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
   }
 
   @Override
   protected HoodieTable>, List, List> createTable(HoodieWriteConfig config,
-                                                                                                  Configuration hadoopConf,
-                                                                                                  boolean refreshTimeline) {
+                                                                                                  Configuration hadoopConf) {
     return HoodieJavaTable.create(config, context);
   }
 
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 2a93c5012ce1e..c4d3453be27ce 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
@@ -209,7 +209,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta
       HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
           extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
 
-      writeTableMetadata(metadata, actionType);
+      writeTableMetadata(metadata, HoodieList.of(result.getWriteStatuses()), actionType);
 
       activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
           Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
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 a52ab6e0f3d0c..301a6627e89fb 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,7 +52,8 @@ protected List getAllExistingFileIds(String partitionPath) {
   protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeResult) {
     Map> partitionToExistingFileIds = new HashMap<>();
     List partitionPaths = FSUtils.getAllPartitionPaths(context,
-        table.getMetaClient().getBasePath(), config.isMetadataTableEnabled(), config.shouldAssumeDatePartitioning());
+        table.getMetaClient().getBasePath(), table.getMetaClient().getTableConfig().isMetadataTableEnabled(),
+        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/HoodieReadClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java
index 84040f906ce32..c60369ba4c26a 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java
@@ -36,9 +36,9 @@
 import org.apache.hudi.exception.HoodieIndexException;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.SparkHoodieIndexFactory;
+import org.apache.hudi.index.SparkMetadataTableRecordIndex;
 import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
-
 import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
@@ -71,6 +71,7 @@ public class HoodieReadClient> implements Seria
   private transient Option sqlContextOpt;
   private final transient HoodieSparkEngineContext context;
   private final transient Configuration hadoopConf;
+  private final transient HoodieWriteConfig clientConfig;
 
   /**
    * @param basePath path to Hoodie table
@@ -96,11 +97,14 @@ public HoodieReadClient(HoodieSparkEngineContext context, String basePath, SQLCo
    */
   public HoodieReadClient(HoodieSparkEngineContext context, HoodieWriteConfig clientConfig) {
     this.context = context;
+    this.clientConfig = clientConfig;
     this.hadoopConf = context.getHadoopConf().get();
     final String basePath = clientConfig.getBasePath();
+
     // Create a Hoodie table which encapsulated the commits and files visible
     HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
     this.hoodieTable = HoodieSparkTable.create(clientConfig, context, metaClient);
+
     this.index = SparkHoodieIndexFactory.createIndex(clientConfig);
     this.sqlContextOpt = Option.empty();
   }
@@ -199,8 +203,18 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco
    * @return Tagged RDD of Hoodie records
    */
   public JavaRDD> tagLocation(JavaRDD> hoodieRecords) throws HoodieIndexException {
-    return HoodieJavaRDD.getJavaRDD(
-        index.tagLocation(HoodieJavaRDD.of(hoodieRecords), context, hoodieTable));
+    return HoodieJavaRDD.getJavaRDD(index.tagLocation(HoodieJavaRDD.of(hoodieRecords), context, hoodieTable));
+  }
+
+  /**
+   * Tag the records with location using metadata table record index (if exists).
+   *
+   * @param hoodieRecords Input RDD of Hoodie records
+   * @return Tagged RDD of Hoodie records
+   */
+  public JavaRDD> tagLocationUsingMetadataRecordIndex(JavaRDD> hoodieRecords) {
+    HoodieIndex recordIndex = new SparkMetadataTableRecordIndex<>(clientConfig);
+    return HoodieJavaRDD.getJavaRDD(recordIndex.tagLocation(HoodieJavaRDD.of(hoodieRecords), context, hoodieTable));
   }
 
   /**
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 9b2aad3ebafa1..62bbbca35e991 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
@@ -122,14 +122,13 @@ public boolean commit(String instantTime, JavaRDD writeStatuses, Op
                         String commitActionType, Map> partitionToReplacedFileIds) {
     context.setJobStatus(this.getClass().getSimpleName(), "Committing stats");
     List writeStats = writeStatuses.map(WriteStatus::getStat).collect();
-    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+    return commitStats(instantTime, HoodieJavaRDD.of(writeStatuses), writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
   }
 
   @Override
   protected HoodieTable>, JavaRDD, JavaRDD> createTable(HoodieWriteConfig config,
-                                                                                                           Configuration hadoopConf,
-                                                                                                           boolean refreshTimeline) {
-    return HoodieSparkTable.create(config, context, refreshTimeline);
+                                                                                                           Configuration hadoopConf) {
+    return HoodieSparkTable.create(config, context);
   }
 
   @Override
@@ -148,7 +147,9 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco
    */
   @Override
   public void bootstrap(Option> extraMetadata) {
-    getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata);
+    HoodieTable>, JavaRDD, JavaRDD> table =
+        getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS);
+    table.bootstrap(context, extraMetadata);
   }
 
   @Override
@@ -305,7 +306,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD compact(String compactionInstantTime, boolean shouldComplete) {
-    HoodieSparkTable table = HoodieSparkTable.create(config, context, true);
+    HoodieSparkTable table = HoodieSparkTable.create(config, context);
     preWrite(compactionInstantTime, WriteOperationType.COMPACT, table.getMetaClient());
     HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
     HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
@@ -348,7 +349,7 @@ protected JavaRDD compact(String compactionInstantTime, boolean sho
 
   @Override
   public HoodieWriteMetadata> cluster(String clusteringInstant, boolean shouldComplete) {
-    HoodieSparkTable table = HoodieSparkTable.create(config, context, config.isMetadataTableEnabled());
+    HoodieSparkTable table = HoodieSparkTable.create(config, context);
     preWrite(clusteringInstant, WriteOperationType.CLUSTER, table.getMetaClient());
     HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceTimeline();
     HoodieInstant inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant);
@@ -379,10 +380,12 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD s.getTotalWriteErrors() > 0L).map(s -> s.getFileId()).collect(Collectors.joining(",")));
     }
     final HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime);
+
     try {
       this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty());
       finalizeWrite(table, clusteringCommitTime, writeStats);
-      writeTableMetadataForTableServices(table, metadata,clusteringInstant);
+      preCommit(clusteringInstant, metadata);
+      writeTableMetadataForTableServices(table, metadata, null, clusteringInstant);
       // Update outstanding metadata indexes
       if (config.isLayoutOptimizationEnabled()
           && !config.getClusteringSortColumns().isEmpty()) {
@@ -413,12 +416,12 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD>, JavaRDD, JavaRDD> table, HoodieCommitMetadata commitMetadata,
-                                  HoodieInstant hoodieInstant) {
+                                  JavaRDD writeStatuses, HoodieInstant hoodieInstant) {
     boolean isTableServiceAction = table.isTableServiceAction(hoodieInstant.getAction());
     // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a
     // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition.
     table.getMetadataWriter(hoodieInstant.getTimestamp()).ifPresent(
-        w -> w.update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction));
+        w -> w.update(commitMetadata, HoodieJavaRDD.of(writeStatuses), hoodieInstant.getTimestamp(), isTableServiceAction));
   }
 
   @Override
@@ -484,7 +487,7 @@ private HoodieTable>, JavaRDD, JavaRDD table = HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, config.isMetadataTableEnabled());
+    HoodieSparkTable table = HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient);
     if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
       writeTimer = metrics.getCommitCtx();
     } else {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java
index d4eb25963e5be..13144f4dd7d0f 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java
@@ -139,4 +139,9 @@ public HoodieData union(HoodieData other) {
   public List collectAsList() {
     return rddData.collect();
   }
+
+  @Override
+  public int getNumPartitions() {
+    return rddData.getNumPartitions();
+  }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java
index 5e686463bc98f..63c4f48a7f6da 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java
@@ -63,6 +63,8 @@ public static HoodieIndex createIndex(HoodieWriteConfig config) {
         return new HoodieSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config));
       case GLOBAL_SIMPLE:
         return new HoodieGlobalSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config));
+      case RECORD_INDEX:
+        return new SparkMetadataTableRecordIndex<>(config);
       default:
         throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
     }
@@ -87,6 +89,8 @@ public static boolean isGlobalIndex(HoodieWriteConfig config) {
         return false;
       case GLOBAL_SIMPLE:
         return true;
+      case RECORD_INDEX:
+        return true;
       default:
         return createIndex(config).isGlobal();
     }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java
new file mode 100644
index 0000000000000..c81afc9ec9877
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java
@@ -0,0 +1,158 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+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.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.metadata.HoodieTableMetadataUtil;
+import org.apache.hudi.metadata.MetadataPartitionType;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.sql.execution.PartitionIdPassthrough;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * Hoodie Index implementation backed by the record index present in the Metadata Table.
+ */
+public class SparkMetadataTableRecordIndex
+    extends HoodieIndex>, JavaRDD, JavaRDD> {
+
+  public SparkMetadataTableRecordIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  @Override
+  public HoodieData> tagLocation(HoodieData> records, HoodieEngineContext context,
+      HoodieTable hoodieTable) {
+    final int numFileGroups;
+    try {
+      HoodieTableMetaClient metaClient = HoodieTableMetadataUtil.getMetadataTableMetaClient(hoodieTable.getMetaClient());
+      numFileGroups = HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metaClient,
+          MetadataPartitionType.RECORD_INDEX.partitionPath()).size();
+    } catch (TableNotFoundException e) {
+      // implies that metadata table has not been initialized yet (probably the first write on a new table)
+      return records;
+    }
+
+    JavaRDD> y = HoodieJavaRDD.getJavaRDD(records).keyBy(r -> HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), numFileGroups))
+        .partitionBy(new PartitionIdPassthrough(numFileGroups))
+        .map(t -> t._2);
+    ValidationUtils.checkState(y.getNumPartitions() <= numFileGroups);
+
+    registry.ifPresent(r -> r.add(TAG_LOC_NUM_PARTITIONS, records.getNumPartitions()));
+    return HoodieJavaRDD.of(y.mapPartitions(new LocationTagFunction(hoodieTable, registry)));
+  }
+
+  @Override
+  public HoodieData updateLocation(HoodieData writeStatuses, HoodieEngineContext context,
+                                                HoodieTable hoodieTable) {
+    // This is a no-op as metadata record index updates are automatically maintained within the metadata table.
+    return writeStatuses;
+  }
+
+  @Override
+  public boolean rollbackCommit(String instantTime) {
+    return true;
+  }
+
+  @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> {
+    private HoodieTable hoodieTable;
+    private Option registry;
+
+    public LocationTagFunction(HoodieTable hoodieTable, Option registry) {
+      this.hoodieTable = hoodieTable;
+      this.registry = registry;
+    }
+
+    @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();
+        keyToIndexMap.put(rec.getRecordKey(), taggedRecords.size());
+        taggedRecords.add(rec);
+      }
+
+      List recordKeys = keyToIndexMap.keySet().stream().sorted().collect(Collectors.toList());
+      try {
+        Map recordIndexInfo = hoodieTable.getMetadataReader().readRecordIndex(recordKeys);
+        HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
+
+        for (Entry e : recordIndexInfo.entrySet()) {
+          if (checkIfValidCommit(metaClient, e.getValue().getInstantTime())) {
+            HoodieRecord rec = taggedRecords.get(keyToIndexMap.get(e.getKey()));
+            rec.unseal();
+            rec.setCurrentLocation(e.getValue());
+            rec.seal();
+          }
+        }
+
+        registry.ifPresent(r -> r.add(TAG_LOC_DURATION, timer.endTimer()));
+        registry.ifPresent(r -> r.add(TAG_LOC_RECORD_COUNT, recordKeys.size()));
+        registry.ifPresent(r -> r.add(TAG_LOC_HITS, recordIndexInfo.size()));
+      } catch (UnsupportedOperationException e) {
+        // This means that record index is not created yet
+      }
+
+      return taggedRecords.iterator();
+    }
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
index e940c0b8211c1..0b05917ca9298 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
@@ -23,6 +23,7 @@
 import org.apache.hudi.client.utils.SparkMemoryUtils;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -30,7 +31,7 @@
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.HoodieTimer;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.RateLimiter;
 import org.apache.hudi.common.util.ReflectionUtils;
@@ -98,6 +99,10 @@ public class SparkHoodieHBaseIndex>
   private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
   private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
 
+  // Metric names
+  public static final String QPS_CALC_DURATION = "qps_calc.duration";
+  public static final String QPS_ACQUIRE_DURATION = "qps_acquire.duration";
+
   private static final Logger LOG = LogManager.getLogger(SparkHoodieHBaseIndex.class);
   private static Connection hbaseConnection = null;
   private HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = null;
@@ -192,19 +197,12 @@ private Get generateStatement(String key, long startTime, long endTime) throws I
     return generateStatement(key).setTimeRange(startTime, endTime);
   }
 
-  private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) {
-    HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants();
-    // Check if the last commit ts for this row is 1) present in the timeline or
-    // 2) is less than the first commit ts in the timeline
-    return !commitTimeline.empty()
-        && commitTimeline.containsOrBeforeTimelineStarts(commitTs);
-  }
-
   /**
    * Function that tags each HoodieRecord with an existing location, if known.
+   * @param registry
    */
   private Function2>, Iterator>> locationTagFunction(
-      HoodieTableMetaClient metaClient) {
+      HoodieTableMetaClient metaClient, Option registry) {
 
     // `multiGetBatchSize` is intended to be a batch per 100ms. To create a rate limiter that measures
     // operations per second, we need to multiply `multiGetBatchSize` by 10.
@@ -212,6 +210,7 @@ private Function2>, Iterator>>
     return (Function2>, Iterator>>) (partitionNum,
         hoodieRecordIterator) -> {
 
+      HoodieTimer timer = new HoodieTimer().startTimer();
       boolean updatePartitionPath = config.getHbaseIndexUpdatePartitionPath();
       RateLimiter limiter = RateLimiter.create(multiGetBatchSize * 10, TimeUnit.SECONDS);
       // Grab the global HBase connection
@@ -220,6 +219,8 @@ private Function2>, Iterator>>
           hbaseConnection = getHBaseConnection();
         }
       }
+
+      long[] stats = {0, 0}; // total, misses;
       List> taggedRecords = new ArrayList<>();
       try (HTable hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName))) {
         List statements = new ArrayList<>();
@@ -238,10 +239,12 @@ private Function2>, Iterator>>
           // clear statements to be GC'd
           statements.clear();
           for (Result result : results) {
+            ++stats[0];
             // first, attempt to grab location from HBase
             HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
             if (result.getRow() == null) {
               taggedRecords.add(currentRecord);
+              ++stats[1];
               continue;
             }
             String keyFromResult = Bytes.toString(result.getRow());
@@ -281,6 +284,10 @@ private Function2>, Iterator>>
       } catch (IOException e) {
         throw new HoodieIndexException("Failed to Tag indexed locations because of exception with HBase Client", e);
       }
+
+      registry.ifPresent(r -> r.add(TAG_LOC_DURATION, timer.endTimer()));
+      registry.ifPresent(r -> r.add(TAG_LOC_RECORD_COUNT, stats[0]));
+      registry.ifPresent(r -> r.add(TAG_LOC_HITS, stats[0] - stats[1]));
       return taggedRecords.iterator();
     };
   }
@@ -297,8 +304,9 @@ private Result[] doGet(HTable hTable, List keys, RateLimiter limiter) throw
   public HoodieData> tagLocation(
       HoodieData> records, HoodieEngineContext context,
       HoodieTable hoodieTable) {
+    registry.ifPresent(r -> r.add(TAG_LOC_NUM_PARTITIONS, records.getNumPartitions()));
     return HoodieJavaRDD.of(HoodieJavaRDD.getJavaRDD(records)
-        .mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true));
+        .mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient(), registry), true));
   }
 
   private Function2, Iterator> updateLocationFunction() {
@@ -315,6 +323,8 @@ private Function2, Iterator> updateL
       final long startTimeForPutsTask = DateTime.now().getMillis();
       LOG.info("startTimeForPutsTask for this task: " + startTimeForPutsTask);
 
+      long[] stats = {0, 0, 0}; // inserts, updates, deletes
+
       try (BufferedMutator mutator = hbaseConnection.getBufferedMutator(TableName.valueOf(tableName))) {
         final RateLimiter limiter = RateLimiter.create(multiPutBatchSize, TimeUnit.SECONDS);
         while (statusIterator.hasNext()) {
@@ -333,6 +343,7 @@ private Function2, Iterator> updateL
                 if (loc.isPresent()) {
                   if (rec.getCurrentLocation() != null) {
                     // This is an update, no need to update index
+                    ++stats[1];
                     continue;
                   }
                   Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
@@ -340,10 +351,12 @@ private Function2, Iterator> updateL
                   put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN, Bytes.toBytes(loc.get().getFileId()));
                   put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN, Bytes.toBytes(rec.getPartitionPath()));
                   mutations.add(put);
+                  ++stats[0];
                 } else {
                   // Delete existing index for a deleted record
                   Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
                   mutations.add(delete);
+                  ++stats[2];
                 }
               }
               if (mutations.size() < multiPutBatchSize) {
@@ -362,6 +375,11 @@ private Function2, Iterator> updateL
         }
         final long endPutsTime = DateTime.now().getMillis();
         LOG.info("hbase puts task time for this task: " + (endPutsTime - startTimeForPutsTask));
+
+        registry.ifPresent(r -> r.add(HoodieIndex.TOTAL_INSERT, stats[0]));
+        registry.ifPresent(r -> r.add(HoodieIndex.TOTAL_UPDATE, stats[1]));
+        registry.ifPresent(r -> r.add(HoodieIndex.TOTAL_DELETE, stats[2]));
+        registry.ifPresent(r -> r.add(HoodieIndex.TOTAL_WRITES, stats[0] + stats[1] + stats[2]));
       } catch (IOException e) {
         throw new HoodieIndexException("Failed to Update Index locations because of exception with HBase Client", e);
       }
@@ -402,6 +420,8 @@ public Map mapFileWithInsertsToUniquePartition(JavaRDD updateLocation(
       HoodieData writeStatus, HoodieEngineContext context,
       HoodieTable hoodieTable) {
+    HoodieTimer timer = new HoodieTimer().startTimer();
+
     JavaRDD writeStatusRDD = HoodieJavaRDD.getJavaRDD(writeStatus);
     final Option desiredQPSFraction = calculateQPSFraction(writeStatusRDD);
     final Map fileIdPartitionMap = mapFileWithInsertsToUniquePartition(writeStatusRDD);
@@ -419,32 +439,41 @@ public HoodieData updateLocation(
     // force trigger update location(hbase puts)
     writeStatusJavaRDD.count();
     this.hBaseIndexQPSResourceAllocator.releaseQPSResources();
+
+    registry.ifPresent(r -> r.add(UPDATE_LOC_DURATION, timer.endTimer()));
+    registry.ifPresent(r -> r.add(UPDATE_LOC_NUM_PARTITIONS, writeStatus.getNumPartitions()));
     return HoodieJavaRDD.of(writeStatusJavaRDD);
   }
 
   private Option calculateQPSFraction(JavaRDD writeStatusRDD) {
-    if (config.getHbaseIndexPutBatchSizeAutoCompute()) {
-      /*
-        Each writeStatus represents status information from a write done in one of the IOHandles.
-        If a writeStatus has any insert, it implies that the corresponding task contacts HBase for
-        doing puts, since we only do puts for inserts from HBaseIndex.
-       */
-      final Tuple2 numPutsParallelismTuple  = getHBasePutAccessParallelism(writeStatusRDD);
-      this.totalNumInserts = numPutsParallelismTuple._1;
-      this.numWriteStatusWithInserts = numPutsParallelismTuple._2;
-      this.numRegionServersForTable = getNumRegionServersAliveForTable();
-      final float desiredQPSFraction = this.hBaseIndexQPSResourceAllocator.calculateQPSFractionForPutsTime(
-          this.totalNumInserts, this.numRegionServersForTable);
-      LOG.info("Desired QPSFraction :" + desiredQPSFraction);
-      LOG.info("Number HBase puts :" + this.totalNumInserts);
-      LOG.info("Number of WriteStatus with inserts :" + numWriteStatusWithInserts);
-      return Option.of(desiredQPSFraction);
+    HoodieTimer timer = new HoodieTimer().startTimer();
+    try {
+      if (config.getHbaseIndexPutBatchSizeAutoCompute()) {
+        /*
+          Each writeStatus represents status information from a write done in one of the IOHandles.
+          If a writeStatus has any insert, it implies that the corresponding task contacts HBase for
+          doing puts, since we only do puts for inserts from HBaseIndex.
+        */
+        final Tuple2 numPutsParallelismTuple  = getHBasePutAccessParallelism(writeStatusRDD);
+        this.totalNumInserts = numPutsParallelismTuple._1;
+        this.numWriteStatusWithInserts = numPutsParallelismTuple._2;
+        this.numRegionServersForTable = getNumRegionServersAliveForTable();
+        final float desiredQPSFraction = this.hBaseIndexQPSResourceAllocator.calculateQPSFractionForPutsTime(
+            this.totalNumInserts, this.numRegionServersForTable);
+        LOG.info("Desired QPSFraction :" + desiredQPSFraction);
+        LOG.info("Number HBase puts :" + this.totalNumInserts);
+        LOG.info("Number of WriteStatus with inserts :" + numWriteStatusWithInserts);
+        return Option.of(desiredQPSFraction);
+      }
+      return Option.empty();
+    } finally {
+      registry.ifPresent(r -> r.add(QPS_CALC_DURATION, timer.endTimer()));
     }
-    return Option.empty();
   }
 
   private void acquireQPSResourcesAndSetBatchSize(final Option desiredQPSFraction,
                                                   final JavaSparkContext jsc) {
+    HoodieTimer timer = new HoodieTimer().startTimer();
     if (config.getHbaseIndexPutBatchSizeAutoCompute()) {
       SparkConf conf = jsc.getConf();
       int maxExecutors = conf.getInt(DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME, 1);
@@ -464,6 +493,8 @@ private void acquireQPSResourcesAndSetBatchSize(final Option desiredQPSFr
                               availableQpsFraction);
       LOG.info("multiPutBatchSize :" + multiPutBatchSize);
     }
+
+    registry.ifPresent(r -> r.add(QPS_ACQUIRE_DURATION, timer.endTimer()));
   }
 
   public Tuple2 getHBasePutAccessParallelism(final JavaRDD writeStatusRDD) {
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 ccb258a8cdc61..e9f656122b8b6 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
@@ -27,7 +27,6 @@
 import org.apache.hudi.common.metrics.Registry;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.Option;
@@ -35,7 +34,7 @@
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieMetadataException;
 import org.apache.hudi.metrics.DistributedRegistry;
-
+import org.apache.hudi.table.BulkInsertPartitioner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -90,9 +89,9 @@ protected void initRegistry() {
     if (metadataWriteConfig.isMetricsOn()) {
       Registry registry;
       if (metadataWriteConfig.isExecutorMetricsEnabled()) {
-        registry = Registry.getRegistry("HoodieMetadata", DistributedRegistry.class.getName());
+        registry = Registry.getRegistry(HoodieTableMetadata.METRIC_REGISTRY_NAME, DistributedRegistry.class.getName());
       } else {
-        registry = Registry.getRegistry("HoodieMetadata");
+        registry = Registry.getRegistry(HoodieTableMetadata.METRIC_REGISTRY_NAME);
       }
       this.metrics = Option.of(new HoodieMetadataMetrics(registry));
     } else {
@@ -121,11 +120,10 @@ protected  void initialize(HoodieEngineContext eng
     }
   }
 
-  protected void commit(HoodieData hoodieDataRecords, String partitionName, String instantTime, boolean canTriggerTableService) {
+  private void commitInternal(HoodieData hoodieDataRecords, String instantTime, boolean canTriggerTableService,
+      Option partitioner) {
     ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet.");
     ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
-    JavaRDD records = (JavaRDD) hoodieDataRecords.get();
-    JavaRDD recordRDD = prepRecords(records, partitionName, 1);
 
     try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {
       if (canTriggerTableService) {
@@ -150,7 +148,9 @@ protected void commit(HoodieData hoodieDataRecords, String partiti
         HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant);
         metadataMetaClient.reloadActiveTimeline();
       }
-      List statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect();
+      List statuses = partitioner.isPresent()
+          ? writeClient.bulkInsertPreppedRecords((JavaRDD) hoodieDataRecords.get(), instantTime, partitioner).collect()
+          : writeClient.upsertPreppedRecords((JavaRDD) hoodieDataRecords.get(), instantTime).collect();
       statuses.forEach(writeStatus -> {
         if (writeStatus.hasErrors()) {
           throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime);
@@ -169,19 +169,23 @@ protected void commit(HoodieData hoodieDataRecords, String partiti
     metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata));
   }
 
-  /**
-   * Tag each record with the location in the given partition.
-   *
-   * The record is tagged with respective file slice's location based on its record key.
-   */
-  private JavaRDD prepRecords(JavaRDD recordsRDD, String partitionName, int numFileGroups) {
-    List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, partitionName);
-    ValidationUtils.checkArgument(fileSlices.size() == numFileGroups, String.format("Invalid number of file groups: found=%d, required=%d", fileSlices.size(), numFileGroups));
-
-    return recordsRDD.map(r -> {
-      FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), numFileGroups));
-      r.setCurrentLocation(new HoodieRecordLocation(slice.getBaseInstantTime(), slice.getFileId()));
-      return r;
-    });
+  @Override
+  protected void commit(HoodieData records, String instantTime, boolean canTriggerTableService) {
+    commitInternal(records, instantTime, canTriggerTableService, Option.empty());
+  }
+
+  @Override
+  protected void commit(HoodieData records, String instantTime, String partitionName,
+      int fileGroupCount) {
+    LOG.info("Performing bulk insert for partition " + partitionName + " with " + fileGroupCount + " file groups");
+    SparkHoodieMetadataBulkInsertPartitioner partitioner = new SparkHoodieMetadataBulkInsertPartitioner(fileGroupCount);
+    commitInternal(records, instantTime, false, Option.of(partitioner));
+
+    // Ensure the expected number of file groups were created
+    List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient,
+        partitionName);
+    ValidationUtils.checkState(fileSlices.size() == fileGroupCount,
+        String.format("Wrong number of fileSlices created for partition %s: expected=%d, found=%d", partitionName,
+            fileGroupCount, fileSlices.size()));
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieMetadataBulkInsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieMetadataBulkInsertPartitioner.java
new file mode 100644
index 0000000000000..c917a5aa5d2c8
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieMetadataBulkInsertPartitioner.java
@@ -0,0 +1,111 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.spark.Partitioner;
+import org.apache.spark.api.java.JavaRDD;
+
+import scala.Tuple2;
+
+/**
+ * A {@code BulkInsertPartitioner} implementation for Metadata Table to improve performance of initialization of metadata
+ * table partition when a very large number of records are inserted.
+ *
+ * This partitioner requires the records to tbe already tagged with the appropriate file slice.
+ */
+public class SparkHoodieMetadataBulkInsertPartitioner implements BulkInsertPartitioner>, Serializable {
+
+  private class FileGroupPartitioner extends Partitioner {
+    private int numFileGroups;
+
+    public FileGroupPartitioner(int numFileGroups) {
+      this.numFileGroups = numFileGroups;
+    }
+
+    @Override
+    public int getPartition(Object key) {
+      return ((Tuple2)key)._1;
+    }
+
+    @Override
+    public int numPartitions() {
+      return numFileGroups;
+    }
+  }
+
+  // The file group count in the partition
+  private int fileGroupCount;
+  // FileIDs for the various partitions
+  private List fileIDPfxs;
+
+  public SparkHoodieMetadataBulkInsertPartitioner(int fileGroupCount) {
+    this.fileGroupCount = fileGroupCount;
+  }
+
+  @Override
+  public JavaRDD repartitionRecords(JavaRDD records, int outputSparkPartitions) {
+    Comparator> keyComparator = (Comparator> & Serializable)(t1, t2) -> {
+      return t1._2.compareTo(t2._2);
+    };
+
+    // Partition the records by their location
+    JavaRDD partitionedRDD = records
+        .keyBy(r -> new Tuple2(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), fileGroupCount), r.getRecordKey()))
+        .repartitionAndSortWithinPartitions(new FileGroupPartitioner(fileGroupCount), keyComparator)
+        .map(t -> t._2);
+    ValidationUtils.checkArgument(partitionedRDD.getNumPartitions() <= fileGroupCount,
+        String.format("Partitioned RDD has more partitions %d than the fileGroupCount %d", partitionedRDD.getNumPartitions(), fileGroupCount));
+
+    fileIDPfxs = partitionedRDD.mapPartitions(recordItr -> {
+      // Due to partitioning, all record in the partition should have same fileID
+      List fileIds = new ArrayList<>(1);
+      if (recordItr.hasNext()) {
+        HoodieRecord record = recordItr.next();
+        final String fileID = record.getCurrentLocation().getFileId();
+        // Remove the write-token from the fileID as we need to return only the prefix
+        int index = fileID.lastIndexOf("-");
+        fileIds.add(fileID.substring(0, index));
+      }
+      // Remove the file-index since we want to
+      return fileIds.iterator();
+    }, true).collect();
+    ValidationUtils.checkArgument(partitionedRDD.getNumPartitions() == fileIDPfxs.size(),
+        String.format("Generated fileIDPfxs (%d) are lesser in size than the partitions %d", fileIDPfxs.size(), partitionedRDD.getNumPartitions()));
+
+    return partitionedRDD;
+  }
+
+  @Override
+  public boolean arePartitionRecordsSorted() {
+    return true;
+  }
+
+  @Override
+  public List generateFileIDPfxs(int numPartitions) {
+    return fileIDPfxs;
+  }
+}
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 35c9ab3a0fe94..41420186504e5 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
@@ -55,28 +55,16 @@ protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context
   }
 
   public static  HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context) {
-    return create(config, context, false);
-  }
-
-  public static  HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context,
-                                                                           boolean refreshTimeline) {
     HoodieTableMetaClient metaClient =
         HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath())
             .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig())
             .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).build();
-    return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, refreshTimeline);
+    return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient);
   }
 
   public static  HoodieSparkTable create(HoodieWriteConfig config,
                                                                            HoodieSparkEngineContext context,
                                                                            HoodieTableMetaClient metaClient) {
-    return create(config, context, metaClient, false);
-  }
-
-  public static  HoodieSparkTable create(HoodieWriteConfig config,
-                                                                           HoodieSparkEngineContext context,
-                                                                           HoodieTableMetaClient metaClient,
-                                                                           boolean refreshTimeline) {
     HoodieSparkTable hoodieSparkTable;
     switch (metaClient.getTableType()) {
       case COPY_ON_WRITE:
@@ -88,7 +76,7 @@ public static  HoodieSparkTable create(HoodieW
       default:
         throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
     }
-    if (refreshTimeline) {
+    if (metaClient.getTableConfig().isMetadataTableEnabled()) {
       hoodieSparkTable.getHoodieView().sync();
     }
     return hoodieSparkTable;
@@ -112,15 +100,15 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con
   @Override
   public  Option getMetadataWriter(String triggeringInstantTimestamp,
                                                                                             Option actionMetadata) {
-    if (config.isMetadataTableEnabled()) {
+    if (metaClient.getTableConfig().isMetadataTableEnabled() || config.isMetadataTableEnabled()) {
       // Create the metadata table writer. First time after the upgrade this creation might trigger
       // metadata table bootstrapping. Bootstrapping process could fail and checking the table
       // existence after the creation is needed.
       final HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(
           context.getHadoopConf().get(), config, context, actionMetadata, Option.of(triggeringInstantTimestamp));
       try {
-        if (isMetadataTableExists || metaClient.getFs().exists(new Path(
-            HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath())))) {
+        if (isMetadataTableExists || metaClient.getTableConfig().isMetadataTableEnabled()
+            || metaClient.getFs().exists(new Path(HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath())))) {
           isMetadataTableExists = true;
           return Option.of(metadataWriter);
         }
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 3c876e75c28f5..d0c74888223d0 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
@@ -252,7 +252,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta
     metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit());
     metadata.setOperationType(operationType);
 
-    writeTableMetadata(metadata, actionType);
+    writeTableMetadata(metadata, HoodieJavaRDD.of(result.getWriteStatuses()), actionType);
 
     try {
       activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime),
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 a44172f379724..0b77bd9f97ee2 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
@@ -287,7 +287,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta
       HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
       HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
           extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
-      writeTableMetadata(metadata, actionType);
+      writeTableMetadata(metadata, HoodieJavaRDD.of(result.getWriteStatuses()), actionType);
       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-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 e3db3914ada77..8b09526d052e5 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
@@ -77,6 +77,7 @@
 import org.apache.hudi.metadata.HoodieMetadataPayload;
 import org.apache.hudi.metadata.HoodieTableMetadata;
 import org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator;
+import org.apache.hudi.metadata.HoodieTableMetadataUtil;
 import org.apache.hudi.metadata.MetadataPartitionType;
 import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
 import org.apache.hudi.table.HoodieSparkTable;
@@ -371,7 +372,7 @@ public void testMetadataTableServices() throws Exception {
     // this should have triggered compaction in metadata table
     tableMetadata = metadata(writeConfig, context);
     assertTrue(tableMetadata.getLatestCompactionTime().isPresent());
-    assertEquals(tableMetadata.getLatestCompactionTime().get(), "0000003001");
+    assertEquals(tableMetadata.getLatestCompactionTime().get(), HoodieTableMetadataUtil.createCompactionTimestamp("0000003"));
   }
 
 
@@ -402,7 +403,7 @@ public void testVirtualKeysInBaseFiles(boolean populateMetaFields) throws Except
 
     HoodieTableMetadata tableMetadata = metadata(writeConfig, context);
     assertTrue(tableMetadata.getLatestCompactionTime().isPresent());
-    assertEquals(tableMetadata.getLatestCompactionTime().get(), "0000003001");
+    assertEquals(tableMetadata.getLatestCompactionTime().get(), HoodieTableMetadataUtil.createCompactionTimestamp("0000003"));
 
     HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build();
     HoodieWriteConfig metadataTableWriteConfig = getMetadataWriteConfig(writeConfig);
@@ -450,7 +451,7 @@ public void testMetadataTableWithPendingCompaction(boolean simulateFailedCompact
     doWriteOperation(testTable, "0000003", INSERT);
 
     HoodieTableMetadata tableMetadata = metadata(writeConfig, context);
-    String metadataCompactionInstant = commitInstant + "001";
+    String metadataCompactionInstant = HoodieTableMetadataUtil.createCompactionTimestamp(commitInstant);
     assertTrue(tableMetadata.getLatestCompactionTime().isPresent());
     assertEquals(tableMetadata.getLatestCompactionTime().get(), metadataCompactionInstant);
 
@@ -483,7 +484,7 @@ public void testMetadataTableWithPendingCompaction(boolean simulateFailedCompact
 
     if (simulateFailedCompaction) {
       //trigger another compaction failure.
-      metadataCompactionInstant = "0000005001";
+      metadataCompactionInstant = HoodieTableMetadataUtil.createCompactionTimestamp("0000005");
       tableMetadata = metadata(writeConfig, context);
       assertTrue(tableMetadata.getLatestCompactionTime().isPresent());
       assertEquals(tableMetadata.getLatestCompactionTime().get(), metadataCompactionInstant);
@@ -805,11 +806,12 @@ public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exc
     init(tableType);
     HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
 
+    String newCommitTime;
     try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
         getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).build())) {
 
       // Write 1 (Bulk insert)
-      String newCommitTime = "0000001";
+      newCommitTime = "0000001";
       List records = dataGen.generateInserts(newCommitTime, 20);
       client.startCommitWithTime(newCommitTime);
       List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect();
@@ -880,6 +882,23 @@ public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exc
       client.restoreToInstant("0000006");
       validateMetadata(client);
     }
+
+    // Once metadata table is enabled, the write config enable flag should not have any effect
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    assertTrue(metaClient.getTableConfig().isMetadataTableEnabled());
+    try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
+      newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+      List records = dataGen.generateInserts(newCommitTime, 20);
+      client.startCommitWithTime(newCommitTime);
+      List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect();
+      assertNoWriteErrors(writeStatuses);
+      //validateMetadata(client);
+
+      // latest commit should have been written to metadata table
+      HoodieTableMetadata tableMetadata = metadata(client);
+      assertTrue(tableMetadata.getSyncedInstantTime().get().equals(newCommitTime));
+    }
+
   }
 
   /**
@@ -1233,7 +1252,9 @@ public void testUpgradeDowngrade() throws IOException {
     }
 
     // Metadata table should have been bootstrapped
+    metaClient = HoodieTableMetaClient.reload(metaClient);
     assertTrue(fs.exists(new Path(metadataTableBasePath)), "Metadata table should exist");
+    assertTrue(metaClient.getTableConfig().isMetadataTableEnabled(), "Metadata table should be enabled in Table Config");
     FileStatus oldStatus = fs.getFileStatus(new Path(metadataTableBasePath));
 
     // set hoodie.table.version to 2 in hoodie.properties file
@@ -1249,6 +1270,10 @@ public void testUpgradeDowngrade() throws IOException {
       writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamp).collect();
       assertNoWriteErrors(writeStatuses);
     }
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    assertTrue(metaClient.getTableConfig().isMetadataTableEnabled(), "Metadata table should be enabled in Table Config");
+    assertTrue(metaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.FILES),
+        "Metadata table files partition should be present in Table Config");
     assertTrue(fs.exists(new Path(metadataTableBasePath)), "Metadata table should exist");
     FileStatus currentStatus = fs.getFileStatus(new Path(metadataTableBasePath));
     assertTrue(currentStatus.getModificationTime() > prevStatus.getModificationTime());
@@ -1263,8 +1288,12 @@ public void testUpgradeDowngrade() throws IOException {
     new UpgradeDowngrade(metaClient, writeConfig, context, SparkUpgradeDowngradeHelper.getInstance())
         .run(HoodieTableVersion.TWO, null);
 
+    metaClient = HoodieTableMetaClient.reload(metaClient);
     assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.TWO.versionCode());
     assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not exist");
+    assertFalse(metaClient.getTableConfig().isMetadataTableEnabled(), "Metadata table should not be enabled in Table Config");
+    assertFalse(metaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.FILES),
+        "Metadata table files partition should not be present in Table Config");
   }
 
   /**
@@ -1479,10 +1508,10 @@ 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);
+      Registry metricsRegistry = Registry.getRegistry(client.getConfig().getTableName() + ".HoodieMetadata");
+      assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_FILE_LISTING_TIME_STR));
+      assertTrue(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_FILE_LISTING_TIME_STR) > 0L);
+
       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));
@@ -1544,8 +1573,8 @@ private HoodieWriteConfig getMetadataWriteConfig(HoodieWriteConfig writeConfig)
 
     // RecordKey properties are needed for the metadata table records
     final Properties properties = new Properties();
-    properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY);
-    properties.put("hoodie.datasource.write.recordkey.field", HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY);
+    properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), HoodieMetadataPayload.KEY_FIELD_NAME);
+    properties.put("hoodie.datasource.write.recordkey.field", HoodieMetadataPayload.KEY_FIELD_NAME);
     builder.withProperties(properties);
 
     if (writeConfig.isMetricsOn()) {
@@ -1648,7 +1677,7 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException
 
     HoodieTableMetadata tableMetadata = metadata(client);
     assertNotNull(tableMetadata, "MetadataReader should have been initialized");
-    if (!config.isMetadataTableEnabled()) {
+    if (!metaClient.getTableConfig().isMetadataTableEnabled()) {
       return;
     }
 
@@ -1760,12 +1789,15 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException
     assertEquals(metadataMetaClient.getTableConfig().getBaseFileFormat(), HoodieFileFormat.HFILE,
         "Metadata Table base file format should be HFile");
 
-    // Metadata table has a fixed number of partitions
+    // Metadata table has a fixed number of partitions. files partition is always present. The other partitions are
+    // created when their feature is enabled.
     // 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);
-    assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size());
+    assertTrue(metadataTablePartitions.size() >= 1);
+    assertTrue(metadataTablePartitions.contains(MetadataPartitionType.FILES.partitionPath()));
+    assertTrue(metadataTablePartitions.size() <= MetadataPartitionType.values().length);
 
     // Metadata table should automatically compact and clean
     // versions are +1 as autoclean / compaction happens end of commits
@@ -1806,8 +1838,7 @@ private HoodieBackedTableMetadataWriter metadataWriter(SparkRDDWriteClient clien
 
   private HoodieTableMetadata metadata(SparkRDDWriteClient client) {
     HoodieWriteConfig clientConfig = client.getConfig();
-    return HoodieTableMetadata.create(client.getEngineContext(), clientConfig.getMetadataConfig(), clientConfig.getBasePath(),
-        clientConfig.getSpillableMapBasePath());
+    return HoodieTableMetadata.create(client.getEngineContext(), clientConfig.getMetadataConfig(), clientConfig.getBasePath());
   }
 
   private void changeTableVersion(HoodieTableVersion version) throws IOException {
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java
index d6f151e34255a..fd4d3046edd6c 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java
@@ -28,8 +28,6 @@
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.EnumSource;
@@ -47,8 +45,6 @@
 
 public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
 
-  private static final Logger LOG = LogManager.getLogger(TestHoodieBackedTableMetadata.class);
-
   @Test
   public void testTableOperations() throws Exception {
     HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE;
@@ -65,7 +61,8 @@ private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception
   }
 
   private void verifyBaseMetadataTable() throws IOException {
-    HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false);
+    HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, writeConfig.getMetadataConfig(),
+        writeConfig.getBasePath());
     assertTrue(tableMetadata.enabled());
     List fsPartitionPaths = testTable.getAllPartitionPaths();
     List fsPartitions = new ArrayList<>();
@@ -79,7 +76,7 @@ private void verifyBaseMetadataTable() throws IOException {
     assertEquals(fsPartitions, metadataPartitions, "Partitions should match");
 
     // Files within each partition should match
-    HoodieTable table = HoodieSparkTable.create(writeConfig, context, true);
+    HoodieTable table = HoodieSparkTable.create(writeConfig, context);
     TableFileSystemView tableView = table.getHoodieView();
     List fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList());
     Map partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths);
@@ -103,11 +100,12 @@ private void verifyBaseMetadataTable() throws IOException {
   public void testMetadataTableKeyGenerator(final HoodieTableType tableType) throws Exception {
     init(tableType);
 
-    HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context,
-        writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false);
+    try (HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context,
+        writeConfig.getMetadataConfig(), writeConfig.getBasePath())) {
 
-    assertEquals(HoodieTableMetadataKeyGenerator.class.getCanonicalName(),
-        tableMetadata.getMetadataMetaClient().getTableConfig().getKeyGeneratorClassName());
+      assertEquals(HoodieTableMetadataKeyGenerator.class.getCanonicalName(),
+          tableMetadata.getMetadataMetaClient().getTableConfig().getKeyGeneratorClassName());
+    }
   }
 
   /**
@@ -117,10 +115,11 @@ public void testMetadataTableKeyGenerator(final HoodieTableType tableType) throw
   @EnumSource(HoodieTableType.class)
   public void testNotExistPartition(final HoodieTableType tableType) throws Exception {
     init(tableType);
-    HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context,
-        writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false);
-    FileStatus[] allFilesInPartition =
-        tableMetadata.getAllFilesInPartition(new Path(writeConfig.getBasePath() + "dummy"));
-    assertEquals(allFilesInPartition.length, 0);
+    try (HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context,
+        writeConfig.getMetadataConfig(), writeConfig.getBasePath())) {
+      FileStatus[] allFilesInPartition =
+          tableMetadata.getAllFilesInPartition(new Path(writeConfig.getBasePath() + "dummy"));
+      assertEquals(allFilesInPartition.length, 0);
+    }
   }
 }
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 01a3d967d7486..b95983de55dd4 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
@@ -27,6 +27,7 @@
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
 import org.apache.hudi.common.table.view.FileSystemViewStorageType;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
@@ -88,8 +89,10 @@ private static Stream indexTypeParams() {
         {IndexType.GLOBAL_BLOOM, true},
         {IndexType.SIMPLE, true},
         {IndexType.GLOBAL_SIMPLE, true},
+        {IndexType.RECORD_INDEX, true},
         {IndexType.SIMPLE, false},
-        {IndexType.GLOBAL_SIMPLE, false}
+        {IndexType.GLOBAL_SIMPLE, false},
+        {IndexType.RECORD_INDEX, false}
     };
     return Stream.of(data).map(Arguments::of);
   }
@@ -112,11 +115,15 @@ private void setUp(IndexType indexType, boolean populateMetaFields, boolean enab
     initFileSystem();
     metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE, populateMetaFields ? new Properties()
         : getPropertiesForKeyGen());
+    HoodieMetadataConfig.Builder metadataConfigBuilder = HoodieMetadataConfig.newBuilder().enable(enableMetadata);
+    if (indexType == IndexType.RECORD_INDEX) {
+      metadataConfigBuilder.createRecordIndex(true);
+    }
     config = getConfigBuilder()
         .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen())
         .withRollbackUsingMarkers(rollbackUsingMarkers)
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
-            .build()).withAutoCommit(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build()).build();
+            .build()).withAutoCommit(false).withMetadataConfig(metadataConfigBuilder.build()).build();
     writeClient = getHoodieWriteClient(config);
     this.index = writeClient.getIndex();
   }
@@ -130,7 +137,7 @@ public void tearDown() throws IOException {
   @MethodSource("indexTypeParams")
   public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populateMetaFields) throws Exception {
     setUp(indexType, populateMetaFields);
-    String newCommitTime = "001";
+    String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
     int totalRecords = 10 + random.nextInt(20);
     List records = dataGen.generateInserts(newCommitTime, totalRecords);
     JavaRDD writeRecords = jsc.parallelize(records, 1);
@@ -180,7 +187,7 @@ public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populate
   @MethodSource("indexTypeParams")
   public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean populateMetaFields) throws Exception {
     setUp(indexType, populateMetaFields);
-    String newCommitTime = "001";
+    String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
     int totalRecords = 10 + random.nextInt(20);
     List records = dataGen.generateInserts(newCommitTime, totalRecords);
     JavaRDD writeRecords = jsc.parallelize(records, 1);
@@ -195,7 +202,7 @@ public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean popul
     // We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
     // recomputed. This includes the state transitions. We need to delete the inflight instance so that subsequent
     // upsert will not run into conflicts.
-    metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight"));
+    metaClient.getFs().delete(new Path(metaClient.getMetaPath(), newCommitTime + ".inflight"));
 
     writeClient.upsert(writeRecords, newCommitTime);
     Assertions.assertNoWriteErrors(writeStatues.collect());
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java
index 2fb364187598b..34f32133f4bfd 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java
@@ -60,7 +60,7 @@ public void setUp(@TempDir Path tempDir) {
   }
 
   @ParameterizedTest
-  @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE"})
+  @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE", "RECORD_INDEX"})
   public void testCreateIndex(IndexType indexType) throws Exception {
     HoodieWriteConfig config;
     HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
@@ -93,6 +93,13 @@ public void testCreateIndex(IndexType indexType) throws Exception {
             .build();
         assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof SparkHoodieHBaseIndex);
         break;
+      case RECORD_INDEX:
+        config = clientConfigBuilder.withPath(basePath)
+            .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.RECORD_INDEX)
+                .withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build())
+            .build();
+        assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof SparkMetadataTableRecordIndex);
+        break;
       default:
         // no -op. just for checkstyle errors
     }
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
index 7674c3489072a..febd72832c388 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
@@ -236,7 +236,7 @@ public void testLogFileCountsAfterCompaction(boolean populateMetaFields) throws
       assertEquals(allPartitions.size(), testTable.listAllBaseFiles().length);
 
       // Verify that all data file has one log file
-      HoodieTable table = HoodieSparkTable.create(config, context(), metaClient, true);
+      HoodieTable table = HoodieSparkTable.create(config, context(), metaClient);
       for (String partitionPath : dataGen.getPartitionPaths()) {
         List groupedLogFiles =
             table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
@@ -550,4 +550,3 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception {
     }
   }
 }
-
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
index d312b644b660e..89e9d1bba3652 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
@@ -79,7 +79,6 @@
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.SQLContext;
 import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.TestInfo;
 
@@ -522,15 +521,19 @@ public static Pair, WorkloadStat> buildProfile(Jav
    */
   public void validateMetadata(HoodieTestTable testTable, List inflightCommits, HoodieWriteConfig writeConfig,
                                String metadataTableBasePath, boolean doFullValidation) throws IOException {
+    metaClient = HoodieTableMetaClient.reload(metaClient);
     HoodieTableMetadata tableMetadata = metadata(writeConfig, context);
     assertNotNull(tableMetadata, "MetadataReader should have been initialized");
-    if (!writeConfig.isMetadataTableEnabled()) {
+    if (tableMetadata instanceof FileSystemBackedTableMetadata) {
+      throw new IllegalStateException("TableMetadata should not be an instance of FileSystemBackedTableMetadata");
+    }
+
+    if (!metaClient.getTableConfig().isMetadataTableEnabled()) {
       return;
     }
 
-    if (!tableMetadata.getSyncedInstantTime().isPresent() || tableMetadata instanceof FileSystemBackedTableMetadata) {
-      throw new IllegalStateException("Metadata should have synced some commits or tableMetadata should not be an instance "
-          + "of FileSystemBackedTableMetadata");
+    if (!tableMetadata.getSyncedInstantTime().isPresent()) {
+      throw new IllegalStateException("Metadata should have synced some commits");
     }
     assertEquals(inflightCommits, testTable.inflightCommits());
 
@@ -553,8 +556,7 @@ public void validateMetadata(HoodieTestTable testTable, List inflightCom
     assertEquals(fsPartitions, metadataPartitions, "Partitions should match");
 
     // Files within each partition should match
-    metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable table = HoodieSparkTable.create(writeConfig, engineContext, true);
+    HoodieTable table = HoodieSparkTable.create(writeConfig, engineContext);
     TableFileSystemView tableView = table.getHoodieView();
     List fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList());
     Map partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths);
@@ -592,8 +594,7 @@ public HoodieBackedTableMetadataWriter metadataWriter(HoodieWriteConfig clientCo
   }
 
   public HoodieTableMetadata metadata(HoodieWriteConfig clientConfig, HoodieEngineContext hoodieEngineContext) {
-    return HoodieTableMetadata.create(hoodieEngineContext, clientConfig.getMetadataConfig(), clientConfig.getBasePath(),
-        clientConfig.getSpillableMapBasePath());
+    return HoodieTableMetadata.create(hoodieEngineContext, clientConfig.getMetadataConfig(), clientConfig.getBasePath());
   }
 
   protected void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableMetadata tableMetadata, TableFileSystemView tableView,
@@ -672,12 +673,15 @@ private void runFullValidation(HoodieWriteConfig writeConfig, String metadataTab
     assertEquals(metadataMetaClient.getTableConfig().getBaseFileFormat(), HoodieFileFormat.HFILE,
         "Metadata Table base file format should be HFile");
 
-    // Metadata table has a fixed number of partitions
+    // Metadata table has a fixed number of partitions. files partition is always present. The other partitions are
+    // created when their feature is enabled.
     // 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);
-    Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size());
+    assertTrue(metadataTablePartitions.size() >= 1);
+    assertTrue(metadataTablePartitions.contains(MetadataPartitionType.FILES.partitionPath()));
+    assertTrue(metadataTablePartitions.size() <= MetadataPartitionType.values().length);
 
     // Metadata table should automatically compact and clean
     // versions are +1 as autoClean / compaction happens end of commits
diff --git a/hudi-common/src/main/avro/HoodieMetadata.avsc b/hudi-common/src/main/avro/HoodieMetadata.avsc
index bf85587a3a7ac..b4ca41d09fb60 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,42 @@
                     ]
                 }
             }]
+        },
+        {
+            "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)"
+                    },
+                    {
+                        "name": "fileIdLowBits",
+                        "type": "long",
+                        "doc": "fileId which contains the record (low 64 bits)"
+                    },
+                    {
+                        "name": "fileIndex",
+                        "type": "int",
+                        "doc": "index of the file"
+                    },
+                    {
+                        "name": "instantTime",
+                        "type": "long",
+                        "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 6beb7d1b2742b..7a0b5707e4137 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
@@ -23,6 +23,8 @@
 
 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;
@@ -41,13 +43,16 @@ public final class HoodieMetadataConfig extends HoodieConfig {
 
   public static final String METADATA_PREFIX = "hoodie.metadata";
 
-  // Enable the internal Metadata Table which saves file listings
+  // Create the Metadata Table (MDT) which saves file listings
   public static final ConfigProperty ENABLE = ConfigProperty
       .key(METADATA_PREFIX + ".enable")
       .defaultValue(false)
       .sinceVersion("0.7.0")
-      .withDocumentation("Enable the internal metadata table which serves table metadata like level file listings");
+      .withDocumentation("Create and use the metadata table (MDT) which serves table metadata like file listings "
+          + "and indexes. If set to true, MDT will be created. Once created, this setting only controls if the MDT "
+          + "will be used on reader side. MDT cannot be disabled/removed by setting this to false.");
 
+  // Should readers use metadata table by default
   public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false;
 
   // Enable metrics for internal Metadata Table
@@ -137,6 +142,62 @@ public final class HoodieMetadataConfig extends HoodieConfig {
       .withDocumentation("There are cases when extra files are requested to be deleted from metadata table which was never added before. This config"
           + "determines how to handle such spurious deletes");
 
+  public static final ConfigProperty MAX_FILE_GROUP_SIZE_BYTES_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".max.filegroup.size")
+      .defaultValue(2 * 1024 * 1024 * 1024L)
+      .sinceVersion("0.11.0")
+      .withDocumentation("Maximum size in bytes of a single file group. Large file group takes longer to compact.");
+
+  public static final ConfigProperty MAX_LOG_FILE_SIZE_BYTES_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".max.logfile.size")
+      .defaultValue(2 * 1024 * 1024 * 1024L)
+      .sinceVersion("0.11.0")
+      .withDocumentation("Maximum size in bytes of a single log file. Larger log files can contain larger log blocks "
+          + "thereby reducing the number of blocks to search for keys");
+
+  public static final ConfigProperty RECORD_INDEX_CREATE_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".record.index.create")
+      .defaultValue(false)
+      .sinceVersion("0.11.0")
+      .withDocumentation("Create the HUDI Record Index within the Metadata Table");
+
+  public static final ConfigProperty RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".record.index.min.filegroup.count")
+      .defaultValue(2)
+      .sinceVersion("0.11.0")
+      .withDocumentation("Minimum number of file groups to use for Record Index.");
+
+  public static final ConfigProperty RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".record.index.max.filegroup.count")
+      .defaultValue(10000)
+      .sinceVersion("0.11.0")
+      .withDocumentation("Maximum number of file groups to use for Record Index.");
+
+  public static final ConfigProperty RECORD_INDEX_GROWTH_FACTOR_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".record.index.growth.factor")
+      .defaultValue(2.0f)
+      .sinceVersion("0.11.0")
+      .withDocumentation("The current number of records are multiplied by this number when estimating the number of "
+          + "file groups to create automatically. This helps account for growth in the number of records in the dataset.");
+
+  public static final ConfigProperty SPILLABLE_MAP_DIR_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".spillable.dir")
+      .defaultValue(FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())
+      .sinceVersion("0.10.0")
+      .withDocumentation("Directory where the spillable maps are saved");
+
+  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();
   }
@@ -177,6 +238,42 @@ public boolean ignoreSpuriousDeletes() {
     return getBoolean(IGNORE_SPURIOUS_DELETES);
   }
 
+  public boolean createRecordIndex() {
+    return enabled() && getBoolean(RECORD_INDEX_CREATE_PROP);
+  }
+
+  public int getRecordIndexMinFileGroupCount() {
+    return getInt(RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP);
+  }
+
+  public int getRecordIndexMaxFileGroupCount() {
+    return getInt(RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP);
+  }
+
+  public float getRecordIndexGrowthFactor() {
+    return getFloat(RECORD_INDEX_GROWTH_FACTOR_PROP);
+  }
+
+  public long getMaxFileGroupSizeBytes() {
+    return getLong(MAX_FILE_GROUP_SIZE_BYTES_PROP);
+  }
+
+  public String getSplliableMapDir() {
+    return getString(SPILLABLE_MAP_DIR_PROP);
+  }
+
+  public long getMaxReaderMemory() {
+    return getLong(MAX_READER_MEMORY_PROP);
+  }
+
+  public int getMaxReaderBufferSize() {
+    return getInt(MAX_READER_BUFFER_SIZE_PROP);
+  }
+
+  public long getMaxLogFileSize() {
+    return getLong(MAX_LOG_FILE_SIZE_BYTES_PROP);
+  }
+
   public static class Builder {
 
     private EngineType engineType = EngineType.SPARK;
@@ -265,6 +362,42 @@ public Builder withEngineType(EngineType engineType) {
       return this;
     }
 
+    public Builder createRecordIndex(boolean enabled) {
+      metadataConfig.setValue(RECORD_INDEX_CREATE_PROP, String.valueOf(enabled));
+      return this;
+    }
+
+    public Builder withRecordIndexFileGroupCount(int minCount, int maxCount) {
+      metadataConfig.setValue(RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP, String.valueOf(minCount));
+      metadataConfig.setValue(RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP, String.valueOf(maxCount));
+      return this;
+    }
+
+    public Builder withRecordIndexGrowthFactor(float factor) {
+      metadataConfig.setValue(RECORD_INDEX_GROWTH_FACTOR_PROP, String.valueOf(factor));
+      return this;
+    }
+
+    public Builder withMaxFileGroupSizeBytes(long sizeInBytes) {
+      metadataConfig.setValue(MAX_FILE_GROUP_SIZE_BYTES_PROP, String.valueOf(sizeInBytes));
+      return this;
+    }
+
+    public Builder withSpillableMapDir(String dir) {
+      metadataConfig.setValue(SPILLABLE_MAP_DIR_PROP, dir);
+      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 HoodieMetadataConfig build() {
       metadataConfig.setDefaultValue(ENABLE, getDefaultMetadataEnable(engineType));
       metadataConfig.setDefaults(HoodieMetadataConfig.class.getName());
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java
index 093fd439db09c..2f217f7983813 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java
@@ -108,4 +108,9 @@ public abstract  HoodieData mapPartitions(
    * @return collected results in {@link List}.
    */
   public abstract List collectAsList();
+
+  /**
+   * @return number of partitions of data.
+   */
+  public abstract int getNumPartitions();
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java
index 94416192abfb9..cfb879306e897 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java
@@ -144,4 +144,9 @@ public HoodieData union(HoodieData other) {
   public List collectAsList() {
     return listData;
   }
+
+  @Override
+  public int getNumPartitions() {
+    return 1;
+  }
 }
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 ceec282f18f1e..cf702a95a0010 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
@@ -26,7 +26,6 @@
 import org.apache.hudi.common.model.HoodiePartitionMetadata;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.ImmutablePair;
 import org.apache.hudi.common.util.collection.Pair;
@@ -276,8 +275,7 @@ public static List getAllPartitionPaths(HoodieEngineContext engineContex
         .enable(useFileListingFromMetadata)
         .withAssumeDatePartitioning(assumeDatePartitioning)
         .build();
-    try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
-        FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) {
+    try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr)) {
       return tableMetadata.getAllPartitionPaths();
     } catch (Exception e) {
       throw new HoodieException("Error fetching partition paths from metadata table", e);
@@ -286,8 +284,7 @@ public static List getAllPartitionPaths(HoodieEngineContext engineContex
 
   public static List getAllPartitionPaths(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
                                                   String basePathStr) {
-    try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
-        FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) {
+    try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr)) {
       return tableMetadata.getAllPartitionPaths();
     } catch (Exception e) {
       throw new HoodieException("Error fetching partition paths from metadata table", e);
@@ -299,8 +296,7 @@ public static Map getFilesInPartitions(HoodieEngineContext
                                                                String basePathStr,
                                                                String[] partitionPaths,
                                                                String spillableMapPath) {
-    try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
-        spillableMapPath, true)) {
+    try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr)) {
       return tableMetadata.getAllFilesInPartitions(Arrays.asList(partitionPaths));
     } catch (Exception ex) {
       throw new HoodieException("Error get files in partitions: " + String.join(",", partitionPaths), ex);
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java b/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java
index bf5ee7e7b71c2..de39ba917ad6b 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java
@@ -71,7 +71,10 @@ static Map getAllMetrics(boolean flush, boolean prefixWithRegistry
     synchronized (Registry.class) {
       HashMap allMetrics = new HashMap<>();
       REGISTRY_MAP.forEach((registryName, registry) -> {
-        allMetrics.putAll(registry.getAllCounts(prefixWithRegistryName));
+        // Merge the values. This is required as multiple registries of different implementations can have the same name
+        registry.getAllCounts(prefixWithRegistryName).forEach((key, value) -> {
+          allMetrics.merge(registryName + key, value, (value1, value2) -> value1 + value2);
+        });
         if (flush) {
           registry.clear();
         }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
index e4b60e2ea3854..fa51ea0842ef8 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
@@ -31,10 +31,11 @@
 import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
 import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
-
+import org.apache.hudi.metadata.MetadataPartitionType;
 import org.apache.avro.Schema;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -167,6 +168,12 @@ public class HoodieTableConfig extends HoodieConfig {
       .noDefaultValue()
       .withDocumentation("Key Generator class property for the hoodie table");
 
+  public static final ConfigProperty METADATA_TABLE_PARTITIONS = ConfigProperty
+      .key("hoodie.metadata.partitions")
+      .defaultValue("")
+      .withDocumentation("The comma-separated list of metadata table (MDT) partitions that have been initialized "
+          + "and are being used for this dataset.");
+
   public static final ConfigProperty URL_ENCODE_PARTITIONING = KeyGeneratorOptions.URL_ENCODE_PARTITIONING;
   public static final ConfigProperty HIVE_STYLE_PARTITIONING_ENABLE = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE;
 
@@ -470,6 +477,45 @@ public String getUrlEncodePartitioning() {
     return getString(URL_ENCODE_PARTITIONING);
   }
 
+  /**
+   * @returns true if metadata table has been created and is being used for this dataset, else returns false.
+   */
+  public boolean isMetadataTableEnabled() {
+    return !StringUtils.isNullOrEmpty(getStringOrDefault(METADATA_TABLE_PARTITIONS));
+  }
+
+  /**
+   * Checks if metadata table is enabled and the specified partition has been initialized.
+   *
+   * @param partition The partition to check
+   * @returns true if the specific partition has been initialized, else returns false.
+   */
+  public boolean isMetadataPartitionEnabled(MetadataPartitionType partition) {
+    String[] partitions = getStringOrDefault(METADATA_TABLE_PARTITIONS).split(",");
+    return Arrays.stream(partitions).anyMatch(p -> p.equals(partition.name()));
+  }
+
+  /**
+   * Enables or disables the specified metadata table partition.
+   *
+   * @param partition The partition to save
+   */
+  public void setMetadataPartitionState(MetadataPartitionType partition, boolean enabled) {
+    ValidationUtils.checkArgument(!partition.name().contains(","), "Metadata Table partition name cannot contain a comma: " + partition.name());
+    Set partitions = Arrays.stream(getStringOrDefault(METADATA_TABLE_PARTITIONS).split(","))
+        .filter(p -> !p.isEmpty()).collect(Collectors.toSet());
+    if (enabled) {
+      partitions.add(partition.name());
+    } else if (partition.name().equals(MetadataPartitionType.FILES.name())) {
+      // file listing partition is required for all other partitions to work
+      // Disabling file partition will also disable all partitions
+      partitions.clear();
+    } else {
+      partitions.remove(partition.name());
+    }
+    setValue(METADATA_TABLE_PARTITIONS, partitions.stream().sorted().collect(Collectors.joining(",")));
+  }
+
   public Map propsMap() {
     return props.entrySet().stream()
         .collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));
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 7f1fa2aa1d64a..d355cd3a963cf 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
@@ -24,13 +24,15 @@
 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.common.util.StringUtils;
+import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
 import org.apache.hudi.io.storage.HoodieHFileReader;
 
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -111,6 +113,7 @@ protected byte[] serializeRecords() throws IOException {
     int key = 0;
     int keySize = 0;
     Field keyField = records.get(0).getSchema().getField(this.keyField);
+
     if (keyField == null) {
       // Missing key metadata field so we should use an integer sequence key
       useIntegerKey = true;
@@ -124,9 +127,15 @@ protected byte[] serializeRecords() throws IOException {
       } else {
         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());
+
+      final byte[] recordBytes = serializeRecord(record, Option.ofNullable(keyField));
+      if (sortedRecordsMap.containsKey(recordKey)) {
+        LOG.error("Found duplicate record with recordKey: " + recordKey);
+        printRecord(sortedRecordsMap.get(recordKey), record.getSchema());
+        printRecord(recordBytes, record.getSchema());
+        throw new HoodieException(String.format("Writing multiple records with same key %s not supported for %s",
+            recordKey, this.getClass().getName()));
+      }
       sortedRecordsMap.put(recordKey, recordBytes);
     }
 
@@ -147,6 +156,12 @@ protected byte[] serializeRecords() throws IOException {
     return baos.toByteArray();
   }
 
+  private void printRecord(byte[] bs, Schema schema) throws IOException {
+    GenericRecord record = HoodieAvroUtils.bytesToAvro(bs, schema);
+    byte[] json = HoodieAvroUtils.avroToJson(record, true);
+    LOG.error("RECORD: " + new String(json));
+  }
+
   @Override
   protected void createRecordsFromContentBytes() throws IOException {
     if (enableInlineReading) {
@@ -162,6 +177,20 @@ public List getRecords(List keys) throws IOException {
     return records;
   }
 
+  /**
+   * Serialize the record to byte buffer.
+   *
+   * @param record         - Record to serialize
+   * @param keyField - Key field in the schema
+   * @return Serialized byte buffer for the record
+   */
+  private byte[] serializeRecord(final IndexedRecord record, final Option keyField) {
+    if (keyField.isPresent()) {
+      record.put(keyField.get().pos(), StringUtils.EMPTY_STRING);
+    }
+    return HoodieAvroUtils.indexedRecordToBytes(record);
+  }
+
   private void readWithInlineFS(List keys) throws IOException {
     boolean enableFullScan = keys.isEmpty();
     // Get schema from the header
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java
index 32c7125e309ea..b3cb2db7245f5 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java
@@ -77,7 +77,7 @@ private FileSystemViewManager(HoodieEngineContext context, FileSystemViewStorage
 
   /**
    * Drops reference to File-System Views. Future calls to view results in creating a new view
-   * 
+   *
    * @param basePath
    */
   public void clearFileSystemView(String basePath) {
@@ -89,7 +89,7 @@ public void clearFileSystemView(String basePath) {
 
   /**
    * Main API to get the file-system view for the base-path.
-   * 
+   *
    * @param basePath
    * @return
    */
@@ -125,7 +125,7 @@ public void close() {
 
   /**
    * Create RocksDB based file System view for a table.
-   * 
+   *
    * @param conf Hadoop Configuration
    * @param viewConf View Storage Configuration
    * @param metaClient HoodieTableMetaClient
@@ -139,7 +139,7 @@ private static RocksDbBasedFileSystemView createRocksDBBasedFileSystemView(Seria
 
   /**
    * Create a spillable Map based file System view for a table.
-   * 
+   *
    * @param conf Hadoop Configuration
    * @param viewConf View Storage Configuration
    * @param metaClient HoodieTableMetaClient
@@ -160,7 +160,7 @@ private static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieMeta
                                                                         HoodieTableMetaClient metaClient, SerializableSupplier metadataSupplier) {
     LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath());
     HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
-    if (metadataConfig.enabled()) {
+    if (metaClient.getTableConfig().isMetadataTableEnabled()) {
       ValidationUtils.checkArgument(metadataSupplier != null, "Metadata supplier is null. Cannot instantiate metadata file system view");
       return new HoodieMetadataFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(),
           metadataSupplier.get());
@@ -169,27 +169,27 @@ private static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieMeta
   }
 
   public static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient,
-                                                                       HoodieMetadataConfig metadataConfig) {
-    
-    return createInMemoryFileSystemViewWithTimeline(engineContext, metaClient, metadataConfig,
+                                                                       boolean useMetadataTable) {
+
+    return createInMemoryFileSystemViewWithTimeline(engineContext, metaClient, useMetadataTable,
         metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants());
-    
+
   }
-  
+
   public static HoodieTableFileSystemView createInMemoryFileSystemViewWithTimeline(HoodieEngineContext engineContext,
                                                                                    HoodieTableMetaClient metaClient,
-                                                                                   HoodieMetadataConfig metadataConfig,
+                                                                                   boolean useMetadataTable,
                                                                                    HoodieTimeline timeline) {
     LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath());
-    if (metadataConfig.enabled()) {
-      return new HoodieMetadataFileSystemView(engineContext, metaClient, timeline, metadataConfig);
+    if (useMetadataTable) {
+      return new HoodieMetadataFileSystemView(engineContext, metaClient, timeline);
     }
     return new HoodieTableFileSystemView(metaClient, timeline);
   }
 
   /**
    * Create a remote file System view for a table.
-   * 
+   *
    * @param conf Hadoop Configuration
    * @param viewConf View Storage Configuration
    * @param metaClient Hoodie Table MetaClient for the table.
@@ -217,7 +217,7 @@ public static FileSystemViewManager createViewManager(final HoodieEngineContext
                                                         final HoodieCommonConfig commonConfig,
                                                         final String basePath) {
     return createViewManager(context, metadataConfig, config, commonConfig,
-        () -> HoodieTableMetadata.create(context, metadataConfig, basePath, config.getSpillableDir(), true));
+        () -> HoodieTableMetadata.create(context, metadataConfig, basePath));
   }
 
   /**
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 e3e38eca86ca9..4414a306e5abc 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
@@ -344,4 +344,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 ccd421e677651..0596f41371e9b 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,11 +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.HoodieRecordGlobalLocation;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.HoodieTimer;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.exception.HoodieMetadataException;
 
 import org.apache.hadoop.fs.FileStatus;
@@ -51,32 +52,26 @@ 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 dataBasePath;
   protected final HoodieTableMetaClient dataMetaClient;
   protected final Option metrics;
   protected final HoodieMetadataConfig metadataConfig;
-  // Directory used for Spillable Map when merging records
-  protected final String spillableMapDirectory;
 
   protected boolean enabled;
 
-  protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
-                              String dataBasePath, String spillableMapDirectory) {
+  protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String dataBasePath) {
     this.engineContext = engineContext;
     this.hadoopConf = new SerializableConfiguration(engineContext.getHadoopConf());
     this.dataBasePath = dataBasePath;
     this.dataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(dataBasePath).build();
-    this.spillableMapDirectory = spillableMapDirectory;
     this.metadataConfig = metadataConfig;
 
-    this.enabled = metadataConfig.enabled();
+    this.enabled = dataMetaClient.getTableConfig().isMetadataTableEnabled();
     if (metadataConfig.enableMetrics()) {
-      this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata")));
+      final Registry registry = Registry.getRegistry(dataMetaClient.getTableConfig().getTableName(), METRIC_REGISTRY_NAME);
+      this.metrics = Option.of(new HoodieMetadataMetrics(registry));
     } else {
       this.metrics = Option.empty();
     }
@@ -152,11 +147,11 @@ public Map getAllFilesInPartitions(List partitions
   protected List fetchAllPartitionPaths() throws IOException {
     HoodieTimer timer = new HoodieTimer().startTimer();
     Option> hoodieRecord = getRecordByKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath());
-    metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer()));
+    metrics.ifPresent(m -> m.updateDurationMetric(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer()));
 
     List partitions = Collections.emptyList();
     if (hoodieRecord.isPresent()) {
-      mayBeHandleSpuriousDeletes(hoodieRecord, "\"all partitions\"");
+      mayBeHandleSpuriousDeletes(hoodieRecord.get(), "\"all partitions\"");
       partitions = hoodieRecord.get().getData().getFilenames();
       // Partition-less tables have a single empty partition
       if (partitions.contains(NON_PARTITIONED_NAME)) {
@@ -182,11 +177,11 @@ FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException {
 
     HoodieTimer timer = new HoodieTimer().startTimer();
     Option> hoodieRecord = getRecordByKey(partitionName, MetadataPartitionType.FILES.partitionPath());
-    metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer()));
+    metrics.ifPresent(m -> m.updateDurationMetric(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer()));
 
     FileStatus[] statuses = {};
     if (hoodieRecord.isPresent()) {
-      mayBeHandleSpuriousDeletes(hoodieRecord, partitionName);
+      mayBeHandleSpuriousDeletes(hoodieRecord.get(), partitionName);
       statuses = hoodieRecord.get().getData().getFileStatuses(hadoopConf.get(), partitionPath);
     }
 
@@ -214,16 +209,14 @@ Map fetchAllFilesInPartitionPaths(List partitionPath
     }
 
     HoodieTimer timer = new HoodieTimer().startTimer();
-    List>>> partitionsFileStatus =
+    Map> partitionsFileStatus =
         getRecordsByKeys(new ArrayList<>(partitionInfo.keySet()), MetadataPartitionType.FILES.partitionPath());
-    metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer()));
+    metrics.ifPresent(m -> m.updateDurationMetric(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer()));
     Map result = new HashMap<>();
 
-    for (Pair>> entry: partitionsFileStatus) {
-      if (entry.getValue().isPresent()) {
-        mayBeHandleSpuriousDeletes(entry.getValue(), entry.getKey());
-        result.put(partitionInfo.get(entry.getKey()).toString(), entry.getValue().get().getData().getFileStatuses(hadoopConf.get(), partitionInfo.get(entry.getKey())));
-      }
+    for (Map.Entry> entry: partitionsFileStatus.entrySet()) {
+      mayBeHandleSpuriousDeletes(entry.getValue(), entry.getKey());
+      result.put(partitionInfo.get(entry.getKey()).toString(), entry.getValue().getData().getFileStatuses(hadoopConf.get(), partitionInfo.get(entry.getKey())));
     }
 
     LOG.info("Listed files in partitions from metadata: partition list =" + Arrays.toString(partitionPaths.toArray()));
@@ -235,11 +228,11 @@ Map fetchAllFilesInPartitionPaths(List partitionPath
    * @param hoodieRecord instance of {@link HoodieRecord} of interest.
    * @param partitionName partition name of interest.
    */
-  private void mayBeHandleSpuriousDeletes(Option> hoodieRecord, String partitionName) {
-    if (!hoodieRecord.get().getData().getDeletions().isEmpty()) {
+  private void mayBeHandleSpuriousDeletes(HoodieRecord hoodieRecord, String partitionName) {
+    if (!hoodieRecord.getData().getDeletions().isEmpty()) {
       if (!metadataConfig.ignoreSpuriousDeletes()) {
         throw new HoodieMetadataException("Metadata record for " + partitionName + " is inconsistent: "
-            + hoodieRecord.get().getData());
+            + hoodieRecord.getData());
       } else {
         LOG.warn("Metadata record for " + partitionName + " encountered some files to be deleted which was not added before. "
             + "Ignoring the spurious deletes as the `" + HoodieMetadataConfig.IGNORE_SPURIOUS_DELETES.key() + "` config is set to false");
@@ -249,7 +242,36 @@ private void mayBeHandleSpuriousDeletes(Option> getRecordByKey(String key, String partitionName);
 
-  protected abstract List>>> getRecordsByKeys(List key, String partitionName);
+  protected abstract Map> getRecordsByKeys(List keys, String partitionName);
+
+  /**
+   * 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 recordKeys The list of record keys to read
+   */
+  @Override
+  public Map readRecordIndex(List recordKeys) {
+    ValidationUtils.checkState(dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.RECORD_INDEX),
+        "Cannot access record-level index as it is not available in the metadata table");
+
+    HoodieTimer timer = new HoodieTimer().startTimer();
+    Map> result = getRecordsByKeys(recordKeys,
+        MetadataPartitionType.RECORD_INDEX.partitionPath());
+
+    Map recordKeyToLocation = new HashMap<>(result.size());
+    result.entrySet().forEach(e -> {
+      recordKeyToLocation.put(e.getKey(), e.getValue().getData().getRecordGlobalLocation());
+    });
+
+    metrics.ifPresent(m -> m.updateDurationMetric(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()));
+
+    return recordKeyToLocation;
+  }
 
   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 759e0f1a3e434..a9b3eef860276 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.HoodieRecordGlobalLocation;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
@@ -78,6 +79,7 @@ public List getAllPartitionPaths() throws IOException {
       int listingParallelism = Math.min(DEFAULT_LISTING_PARALLELISM, pathsToList.size());
 
       // List all directories in parallel
+      engineContext.setJobStatus(this.getClass().getSimpleName(), "Listing all partitions");
       List> dirToFileListing = engineContext.map(pathsToList, path -> {
         FileSystem fileSystem = path.getFileSystem(hadoopConf.get());
         return Pair.of(path, fileSystem.listStatus(path));
@@ -116,6 +118,7 @@ public Map getAllFilesInPartitions(List partitionP
 
     int parallelism = Math.min(DEFAULT_LISTING_PARALLELISM, partitionPaths.size());
 
+    engineContext.setJobStatus(this.getClass().getSimpleName(), "Listing all files in multiple partitions");
     List> partitionToFiles = engineContext.map(partitionPaths, partitionPathStr -> {
       Path partitionPath = new Path(partitionPathStr);
       FileSystem fs = partitionPath.getFileSystem(hadoopConf.get());
@@ -144,4 +147,8 @@ public void close() throws Exception {
   public void reset() {
     // no-op
   }
+
+  public Map readRecordIndex(List 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 c9e538f72eaa0..cff1325988b42 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
@@ -40,8 +40,8 @@
 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.Pair;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieMetadataException;
@@ -77,31 +77,21 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
   // Metadata table's timeline and metaclient
   private HoodieTableMetaClient metadataMetaClient;
   private HoodieTableConfig metadataTableConfig;
-  // should we reuse the open file handles, across calls
-  private final boolean reuse;
 
   // Readers for latest file slice corresponding to file groups in the metadata partition of interest
-  private Map> partitionReaders = new ConcurrentHashMap<>();
+  private transient Map> fileSliceReaders = new ConcurrentHashMap<>();
+  // Latest file slices in the metadata partitions
+  private final Map> partitionFileSliceMap = new ConcurrentHashMap<>();
 
   public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
-                                   String datasetBasePath, String spillableMapDirectory) {
-    this(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory, false);
-  }
-
-  public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
-                                   String datasetBasePath, String spillableMapDirectory, boolean reuse) {
-    super(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory);
-    this.reuse = reuse;
+                                   String datasetBasePath) {
+    super(engineContext, metadataConfig, datasetBasePath);
     initIfNeeded();
   }
 
   private void initIfNeeded() {
     this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(dataBasePath);
-    if (!enabled) {
-      if (!HoodieTableMetadata.isMetadataTable(metadataBasePath)) {
-        LOG.info("Metadata table is disabled.");
-      }
-    } else if (this.metadataMetaClient == null) {
+    if (this.metadataMetaClient == null) {
       try {
         this.metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build();
         this.metadataTableConfig = metadataMetaClient.getTableConfig();
@@ -121,103 +111,132 @@ private void initIfNeeded() {
 
   @Override
   protected Option> getRecordByKey(String key, String partitionName) {
-    List>>> recordsByKeys = getRecordsByKeys(Collections.singletonList(key), partitionName);
-    return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue();
+    Map> recordsByKeys =
+        getRecordsByKeys(Collections.singletonList(key), partitionName);
+    return Option.ofNullable(recordsByKeys.getOrDefault(key, null));
+  }
+
+  @Override
+  protected Map> getRecordsByKeys(List keys, String partitionName) {
+    if (keys.isEmpty()) {
+      return Collections.emptyMap();
+    }
+
+    Map> result;
+
+    // Load the file slices for the partition. Each file slice is a shard which saves a portion of the keys.
+    List latestFileSlices = partitionFileSliceMap.computeIfAbsent(partitionName,
+        k -> HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName));
+    final int numFileSlices = latestFileSlices.size();
+
+    // Lookup keys from each shard
+    if (numFileSlices == 1) {
+      // Optimization for a single shard which is for smaller metadata table partitions
+      result = lookupKeysFromFileSlice(partitionName, keys, latestFileSlices.get(0));
+    } else {
+      // Parallel lookup for large sized partitions with multiple shards
+      // Partition the keys by the shard
+      ArrayList> partitionedKeys = new ArrayList<>(numFileSlices);
+      for (int i = 0; i < numFileSlices; ++i) {
+        partitionedKeys.add(new ArrayList<>());
+      }
+      keys.stream().forEach(key -> {
+        int shardIndex = HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key, numFileSlices);
+        partitionedKeys.get(shardIndex).add(key);
+      });
+
+      result = new HashMap<>();
+      HoodieEngineContext engineContext = getEngineContext();
+      engineContext.setJobStatus(this.getClass().getSimpleName(), "Reading keys from metadata table partition " + partitionName);
+      engineContext.map(partitionedKeys, keysList -> {
+        if (keysList.isEmpty()) {
+          return Collections.emptyMap();
+        }
+        int shardIndex = HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(keysList.get(0), numFileSlices);
+        return lookupKeysFromFileSlice(partitionName, keysList, latestFileSlices.get(shardIndex));
+      }, partitionedKeys.size())
+        .forEach(lookupResult -> result.putAll((Map>) lookupResult));
+    }
+
+    return result;
   }
 
-  protected List>>> getRecordsByKeys(List keys, String partitionName) {
-    Pair readers = openReadersIfNeeded(keys.get(0), partitionName);
+  /**
+   * Lookup list of keys from a single file slice.
+   *
+   * @param partitionName Name of the partition
+   * @param keys The list of keys to lookup
+   * @param fileSlice The file slice to read
+   * @return A {@code Map} of key name to {@code HoodieRecord} for the keys which were found in the file slice
+   */
+  private Map> lookupKeysFromFileSlice(String partitionName,
+      List keys, FileSlice fileSlice) {
+    Pair readers = openReadersIfNeeded(partitionName, fileSlice);
     try {
-      List timings = new ArrayList<>();
       HoodieFileReader baseFileReader = readers.getKey();
       HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
 
       if (baseFileReader == null && logRecordScanner == null) {
-        return Collections.emptyList();
+        return Collections.emptyMap();
       }
 
       // local map to assist in merging with base file records
-      Map>> logRecords = readLogRecords(logRecordScanner, keys, timings);
-      List>>> result = readFromBaseAndMergeWithLogRecords(
-          baseFileReader, keys, logRecords, timings, partitionName);
-      LOG.info(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", keys.size(), timings));
+      Map> logRecords = readLogRecords(logRecordScanner, keys);
+      Map> result = readFromBaseAndMergeWithLogRecords(
+          baseFileReader, keys, logRecords, partitionName);
       return result;
     } catch (IOException ioe) {
       throw new HoodieIOException("Error merging records from metadata table for  " + keys.size() + " key : ", ioe);
-    } finally {
-      if (!reuse) {
-        close(partitionName);
-      }
     }
   }
 
-  private Map>> readLogRecords(HoodieMetadataMergedLogRecordReader logRecordScanner,
-                                                                                  List keys, List timings) {
-    HoodieTimer timer = new HoodieTimer().startTimer();
-    Map>> logRecords = new HashMap<>();
+  private Map> readLogRecords(HoodieMetadataMergedLogRecordReader logRecordScanner,
+                                                                          List keys) {
     // Retrieve records from log file
-    timer.startTimer();
-    if (logRecordScanner != null) {
-      if (metadataConfig.enableFullScan()) {
-        // path which does full scan of log files
-        for (String key : keys) {
-          logRecords.put(key, logRecordScanner.getRecordByKey(key).get(0).getValue());
-        }
-      } else {
-        // this path will do seeks pertaining to the keys passed in
-        List>>> logRecordsList = logRecordScanner.getRecordsByKeys(keys);
-        for (Pair>> entry : logRecordsList) {
-          logRecords.put(entry.getKey(), entry.getValue());
-        }
-      }
-    } else {
-      for (String key : keys) {
-        logRecords.put(key, Option.empty());
-      }
-    }
-    timings.add(timer.endTimer());
+    HoodieTimer timer = new HoodieTimer().startTimer();
+    Map> logRecords = (logRecordScanner != null)
+        ? logRecordScanner.getRecordsByKeys(keys) : new HashMap<>();
+    final long avgReadTimePerKey = timer.endTimer() / keys.size();
+    metrics.ifPresent(m -> m.updateDurationMetric(HoodieMetadataMetrics.LOGFILE_READ_STR, avgReadTimePerKey));
     return logRecords;
   }
 
-  private List>>> readFromBaseAndMergeWithLogRecords(HoodieFileReader baseFileReader,
-                                                                                                             List keys, Map>> logRecords, List timings, String partitionName) throws IOException {
-    List>>> result = new ArrayList<>();
+  private Map> readFromBaseAndMergeWithLogRecords(HoodieFileReader baseFileReader,
+      List keys, Map> logRecords, String partitionName) throws IOException {
     // merge with base records
-    HoodieTimer timer = new HoodieTimer().startTimer();
-    timer.startTimer();
-    HoodieRecord hoodieRecord = null;
     // Retrieve record from base file
     if (baseFileReader != null) {
-      HoodieTimer readTimer = new HoodieTimer();
-      for (String key : keys) {
-        readTimer.startTimer();
-        Option baseRecord = baseFileReader.getRecordByKey(key);
-        if (baseRecord.isPresent()) {
-          hoodieRecord = getRecord(baseRecord, partitionName);
-          metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer()));
-          // merge base file record w/ log record if present
-          if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) {
-            HoodieRecordPayload mergedPayload = logRecords.get(key).get().getData().preCombine(hoodieRecord.getData());
-            result.add(Pair.of(key, Option.of(new HoodieRecord(hoodieRecord.getKey(), mergedPayload))));
-          } else {
-            // only base record
-            result.add(Pair.of(key, Option.of(hoodieRecord)));
+      HoodieTimer timer = new HoodieTimer().startTimer();
+      final Map> result = new HashMap<>();
+      keys.stream().sorted().forEach(key -> {
+        Option baseRecord = null;
+        try {
+          baseRecord = baseFileReader.getRecordByKey(key);
+          if (baseRecord.isPresent()) {
+            HoodieRecord hoodieRecord = getRecord(baseRecord, partitionName);
+            // merge base file record w/ log record if present
+            if (logRecords.containsKey(key)) {
+              HoodieRecordPayload mergedPayload = logRecords.get(key).getData().preCombine(hoodieRecord.getData());
+              result.put(key, new HoodieRecord(hoodieRecord.getKey(), mergedPayload));
+            } else {
+              // only base record
+              result.put(key, hoodieRecord);
+            }
+          } else if (logRecords.containsKey(key)) {
+            // only log record
+            result.put(key, logRecords.get(key));
           }
-        } else {
-          // only log record
-          result.add(Pair.of(key, logRecords.get(key)));
+        } catch (IOException e) {
+          throw new HoodieException("Could not read record from base file", e);
         }
-      }
-      timings.add(timer.endTimer());
+      });
+      final long avgReadTimePerKey = timer.endTimer() / keys.size();
+      metrics.ifPresent(m -> m.updateDurationMetric(HoodieMetadataMetrics.BASEFILE_READ_STR, avgReadTimePerKey));
+      return result;
     } else {
       // no base file at all
-      timings.add(timer.endTimer());
-      for (Map.Entry>> entry : logRecords.entrySet()) {
-        result.add(Pair.of(entry.getKey(), entry.getValue()));
-      }
+      return logRecords;
     }
-    return result;
   }
 
   private HoodieRecord getRecord(Option baseRecord, String partitionName) {
@@ -233,10 +252,19 @@ private HoodieRecord getRecord(Option base
   }
 
   /**
-   * Returns a new pair of readers to the base and log files.
+   * Opens and returns readers to one of the shards of a partition.
+   *
+   * @param partitionName the name of the partition
+   * @param fileSlice the latest file slice for the shard
    */
-  private Pair openReadersIfNeeded(String key, String partitionName) {
-    return partitionReaders.computeIfAbsent(partitionName, k -> {
+  private Pair openReadersIfNeeded(String partitionName,
+      FileSlice fileSlice) {
+    synchronized (this) {
+      if (fileSliceReaders == null) {
+        fileSliceReaders = new ConcurrentHashMap<>();
+      }
+    }
+    return fileSliceReaders.computeIfAbsent(fileSlice.getFileId(), k -> {
       try {
         final long baseFileOpenMs;
         final long logScannerOpenMs;
@@ -245,29 +273,23 @@ private Pair openReadersI
 
         // Metadata is in sync till the latest completed instant on the dataset
         HoodieTimer timer = new HoodieTimer().startTimer();
-        List latestFileSlices = HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName);
-        if (latestFileSlices.size() == 0) {
-          // empty partition
-          return Pair.of(null, null);
-        }
-        ValidationUtils.checkArgument(latestFileSlices.size() == 1, String.format("Invalid number of file slices: found=%d, required=%d", latestFileSlices.size(), 1));
-        final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key, latestFileSlices.size()));
 
         // Open base file reader
-        Pair baseFileReaderOpenTimePair = getBaseFileReader(slice, timer);
+        Pair baseFileReaderOpenTimePair = getBaseFileReader(fileSlice, timer);
         baseFileReader = baseFileReaderOpenTimePair.getKey();
         baseFileOpenMs = baseFileReaderOpenTimePair.getValue();
 
         // Open the log record scanner using the log files from the latest file slice
-        Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice,
+        Pair logRecordScannerOpenTimePair = getLogRecordScanner(fileSlice,
             partitionName);
         logRecordScanner = logRecordScannerOpenTimePair.getKey();
         logScannerOpenMs = logRecordScannerOpenTimePair.getValue();
 
-        metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, baseFileOpenMs + logScannerOpenMs));
+        metrics.ifPresent(metrics -> metrics.updateDurationMetric(HoodieMetadataMetrics.SCAN_STR, baseFileOpenMs + logScannerOpenMs));
         return Pair.of(baseFileReader, logRecordScanner);
       } catch (IOException e) {
-        throw new HoodieIOException("Error opening readers for metadata table partition " + partitionName, e);
+        throw new HoodieIOException(String.format("Error opening readers for file slice %s of metadata table partition %s",
+            fileSlice, partitionName), e);
       }
     });
   }
@@ -302,11 +324,18 @@ private Set getValidInstantTimestamps() {
     // instant which we have a log block for.
     final String earliestInstantTime = validInstantTimestamps.isEmpty() ? SOLO_COMMIT_TIMESTAMP : Collections.min(validInstantTimestamps);
     datasetTimeline.getRollbackAndRestoreTimeline().filterCompletedInstants().getInstants()
-        .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN, earliestInstantTime))
+        .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestInstantTime))
         .forEach(instant -> {
           validInstantTimestamps.addAll(getRollbackedCommits(instant, datasetTimeline));
         });
 
+    // When additional partitions are bootstrapped later (i.e after metadata table already exists), they use a separate
+    // deltacommit whose exact timestamp is not present on the dataset. These deltacommits should also be included.
+    metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().getInstants()
+        .filter(instant -> !validInstantTimestamps.contains(instant.getTimestamp()))
+        .filter(instant -> HoodieTableMetadataUtil.isIndexInitInstant(instant))
+        .forEach(instant -> validInstantTimestamps.add(instant.getTimestamp()));
+
     // SOLO_COMMIT_TIMESTAMP is used during bootstrap so it is a valid timestamp
     validInstantTimestamps.add(SOLO_COMMIT_TIMESTAMP);
     return validInstantTimestamps;
@@ -335,9 +364,9 @@ private Pair getLogRecordScanner(File
         .withLogFilePaths(logFilePaths)
         .withReaderSchema(schema)
         .withLatestInstantTime(latestMetadataInstantTime)
-        .withMaxMemorySizeInBytes(MAX_MEMORY_SIZE_IN_BYTES)
-        .withBufferSize(BUFFER_SIZE)
-        .withSpillableMapBasePath(spillableMapDirectory)
+        .withMaxMemorySizeInBytes(metadataConfig.getMaxReaderMemory())
+        .withBufferSize(metadataConfig.getMaxReaderBufferSize())
+        .withSpillableMapBasePath(metadataConfig.getSplliableMapDir())
         .withDiskMapType(commonConfig.getSpillableDiskMapType())
         .withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled())
         .withLogBlockTimestamps(validInstantTimestamps)
@@ -346,8 +375,15 @@ private Pair getLogRecordScanner(File
         .build();
 
     Long logScannerOpenMs = timer.endTimer();
-    LOG.info(String.format("Opened %d metadata log files (dataset instant=%s, metadata instant=%s) in %d ms",
+    LOG.info(String.format("Opened %d metadata log files (dataset instant=%s, metadata instant=%s,) in %d ms",
         logFilePaths.size(), getLatestDataInstantTime(), latestMetadataInstantTime, logScannerOpenMs));
+    if (metadataConfig.enableFullScan()) {
+      // Only on a full scan we read all the log blocks and these metrics can be published
+      metrics.ifPresent(m -> m.setMetric(partitionName + "." + HoodieMetadataMetrics.COUNT_LOG_BLOCKS,
+          logRecordScanner.getTotalLogBlocks()));
+      metrics.ifPresent(m -> m.setMetric(partitionName + "." + HoodieMetadataMetrics.COUNT_LOG_RECORDS,
+          logRecordScanner.getTotalLogRecords()));
+    }
     return Pair.of(logRecordScanner, logScannerOpenMs);
   }
 
@@ -382,26 +418,22 @@ private List getRollbackedCommits(HoodieInstant instant, HoodieActiveTim
 
   @Override
   public void close() {
-    for (String partitionName : partitionReaders.keySet()) {
-      close(partitionName);
-    }
-    partitionReaders.clear();
-  }
-
-  private synchronized void close(String partitionName) {
-    Pair readers = partitionReaders.remove(partitionName);
-    if (readers != null) {
-      try {
-        if (readers.getKey() != null) {
-          readers.getKey().close();
-        }
-        if (readers.getValue() != null) {
-          readers.getValue().close();
+    for (String key : fileSliceReaders.keySet()) {
+      Pair readers = fileSliceReaders.remove(key);
+      if (readers != null) {
+        try {
+          if (readers.getKey() != null) {
+            readers.getKey().close();
+          }
+          if (readers.getValue() != null) {
+            readers.getValue().close();
+          }
+        } catch (Exception e) {
+          throw new HoodieException("Error closing resources during metadata table merge", e);
         }
-      } catch (Exception e) {
-        throw new HoodieException("Error closing resources during metadata table merge", e);
       }
     }
+    fileSliceReaders.clear();
   }
 
   public boolean enabled() {
@@ -444,6 +476,10 @@ public Option getLatestCompactionTime() {
 
   @Override
   public void reset() {
+    metadataMetaClient = null;
+    fileSliceReaders.clear();
+    partitionFileSliceMap.clear();
+
     initIfNeeded();
     dataMetaClient.reloadActiveTimeline();
   }
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java
index d3b569ceb623a..25ed800ff9797 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java
@@ -22,7 +22,6 @@
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 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.exception.HoodieException;
 
@@ -47,11 +46,10 @@ public HoodieMetadataFileSystemView(HoodieTableMetaClient metaClient,
 
   public HoodieMetadataFileSystemView(HoodieEngineContext engineContext,
                                       HoodieTableMetaClient metaClient,
-                                      HoodieTimeline visibleActiveTimeline,
-                                      HoodieMetadataConfig metadataConfig) {
+                                      HoodieTimeline visibleActiveTimeline) {
     super(metaClient, visibleActiveTimeline);
-    this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(),
-        FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue());
+    this.tableMetadata = HoodieTableMetadata.create(engineContext,
+        HoodieMetadataConfig.newBuilder().enable(true).build(), metaClient.getBasePath());
   }
 
   /**
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java
index 01c8d05e9b220..e643fd1d07ce6 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java
@@ -19,9 +19,10 @@
 package org.apache.hudi.metadata;
 
 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 org.apache.avro.Schema;
@@ -32,8 +33,6 @@
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.util.SpillableMapUtils;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
 
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -50,8 +49,6 @@
  */
 public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordScanner {
 
-  private static final Logger LOG = LogManager.getLogger(HoodieMetadataMergedLogRecordReader.class);
-
   // Set of all record keys that are to be read in memory
   private Set mergeKeyFilter;
 
@@ -111,35 +108,38 @@ public static HoodieMetadataMergedLogRecordReader.Builder newBuilder() {
   }
 
   /**
-   * Retrieve a record given its key.
+   * Retrive records given their keys.
    *
-   * @param key Key of the record to retrieve
-   * @return {@code HoodieRecord} if key was found else {@code Option.empty()}
+   * This function has to be synchronized because we clear the records cache in case full scan is not enabled.
+   * @param keys Keys to retrieve
+   * @return Map of keys to their records. Only the found keys are returned.
    */
-  public List>>> getRecordByKey(String key) {
-    return Collections.singletonList(Pair.of(key, Option.ofNullable((HoodieRecord) records.get(key))));
-  }
+  public synchronized Map> getRecordsByKeys(List keys) {
+    if (!this.enableFullScan) {
+      // When full scan is disabled, we need to perform lookups of keys for each call.
+      records.clear();
+      scan(Option.of(keys));
+    }
 
-  public synchronized List>>> getRecordsByKeys(List keys) {
-    // Following operations have to be atomic, otherwise concurrent
-    // readers would race with each other and could crash when
-    // processing log block records as part of scan.
-    records.clear();
-    scan(Option.of(keys));
-    List>>> metadataRecords = new ArrayList<>();
-    keys.forEach(entry -> {
-      if (records.containsKey(entry)) {
-        metadataRecords.add(Pair.of(entry, Option.ofNullable((HoodieRecord) records.get(entry))));
-      } else {
-        metadataRecords.add(Pair.of(entry, Option.empty()));
+    Map> metadataRecords = new HashMap<>();
+    keys.forEach(key -> {
+      if (records.containsKey(key)) {
+        metadataRecords.put(key, (HoodieRecord) records.get(key));
       }
     });
+
+    if (!this.enableFullScan) {
+      // Clear record cache as we did not scan in full and we dont need to keep the cache of records
+      // for the next lookup
+      records.clear();
+    }
+
     return metadataRecords;
   }
 
   @Override
   protected String getKeyField() {
-    return HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY;
+    return HoodieMetadataPayload.KEY_FIELD_NAME;
   }
 
   /**
@@ -148,7 +148,6 @@ protected String getKeyField() {
   public static class Builder extends HoodieMergedLogRecordScanner.Builder {
     private Set mergeKeyFilter = Collections.emptySet();
     private boolean enableFullScan = HoodieMetadataConfig.ENABLE_FULL_SCAN_LOG_FILES.defaultValue();
-    private boolean enableInlineReading;
 
     @Override
     public Builder withFileSystem(FileSystem fs) {
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 2efc96c6f3dee..70f8d3638baf1 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
@@ -25,9 +25,6 @@
 import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
 import org.apache.hudi.exception.HoodieIOException;
 
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.HashMap;
@@ -41,22 +38,33 @@ 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 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 LOGFILE_READ_STR = "logfile_read";
+  public static final String INITIALIZE_FILE_LISTING_TIME_STR = "initialize_files_time";
+  public static final String INITIALIZE_RECORD_INDEX_TIME_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_TIME_STR = "read_recordkeys_time";
+  public static final String READ_RECORDKEYS_COUNT_STR = "read_recordkeys_count";
+  public static final String READ_FILES_COUNT_STR = "read_filelisting_count";
+  public static final String RECORDINDEX_HITS_STR = "record_index_hits";
+  public static final String RECORDINDEX_MISS_STR = "record_index_misses";
+  public static final String COUNT_LOG_BLOCKS = "logBlockCount";
+  public static final String COUNT_LOG_RECORDS = "logRecordsCount";
+
   // Stats names
   public static final String STAT_TOTAL_BASE_FILE_SIZE = "totalBaseFileSizeInBytes";
   public static final String STAT_TOTAL_LOG_FILE_SIZE = "totalLogFileSizeInBytes";
   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_COUNT_FILE_GROUP = "fileGroupCount";
   public static final String STAT_LAST_COMPACTION_TIMESTAMP = "lastCompactionTimestamp";
 
-  private static final Logger LOG = LogManager.getLogger(HoodieMetadataMetrics.class);
-
   private final Registry metricsRegistry;
 
   public HoodieMetadataMetrics(Registry metricsRegistry) {
@@ -77,8 +85,8 @@ private Map getStats(HoodieTableFileSystemView fsView, boolean d
     Map stats = new HashMap<>();
 
     // Total size of the metadata and count of base/log files
-    for (String metadataPartition : MetadataPartitionType.all()) {
-      List latestSlices = fsView.getLatestFileSlices(metadataPartition).collect(Collectors.toList());
+    for (MetadataPartitionType metadataPartition : MetadataPartitionType.all()) {
+      List latestSlices = fsView.getLatestFileSlices(metadataPartition.partitionPath()).collect(Collectors.toList());
 
       // Total size of the metadata and count of base/log files
       long totalBaseFileSizeInBytes = 0;
@@ -93,15 +101,17 @@ private Map getStats(HoodieTableFileSystemView fsView, boolean d
         }
         Iterator it = slice.getLogFiles().iterator();
         while (it.hasNext()) {
-          totalLogFileSizeInBytes += it.next().getFileSize();
+          HoodieLogFile logFile = it.next();
+          totalLogFileSizeInBytes += logFile.getFileSize();
           ++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));
+      stats.put(metadataPartition.partitionPath() + "." + STAT_TOTAL_BASE_FILE_SIZE, String.valueOf(totalBaseFileSizeInBytes));
+      stats.put(metadataPartition.partitionPath() + "." + STAT_TOTAL_LOG_FILE_SIZE, String.valueOf(totalLogFileSizeInBytes));
+      stats.put(metadataPartition.partitionPath() + "." + STAT_COUNT_BASE_FILES, String.valueOf(baseFileCount));
+      stats.put(metadataPartition.partitionPath() + "." + STAT_COUNT_LOG_FILES, String.valueOf(logFileCount));
+      stats.put(metadataPartition.partitionPath() + "." + STAT_COUNT_FILE_GROUP, String.valueOf(latestSlices.size()));
     }
 
     if (detailed) {
@@ -111,7 +121,7 @@ private Map getStats(HoodieTableFileSystemView fsView, boolean d
     return stats;
   }
 
-  protected void updateMetrics(String action, long durationInMs) {
+  protected void updateDurationMetric(String action, long durationInMs) {
     if (metricsRegistry == null) {
       return;
     }
@@ -126,15 +136,18 @@ protected void updateMetrics(String action, long durationInMs) {
   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()));
+      setMetric(e.getKey(), Long.parseLong(e.getValue()));
     }
   }
 
   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);
   }
 
+  protected void setMetric(String action, long value) {
+    metricsRegistry.set(action, value);
+  }
+
   public Registry registry() {
     return metricsRegistry;
   }
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 0b0d144a6e7e9..2c43111d6589a 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,11 +20,13 @@
 
 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.HoodieRecordGlobalLocation;
 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;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
@@ -36,9 +38,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;
 
@@ -63,18 +67,24 @@
 public class HoodieMetadataPayload implements HoodieRecordPayload {
 
   // HoodieMetadata schema field ids
-  public static final String SCHEMA_FIELD_ID_KEY = "key";
-  public static final String SCHEMA_FIELD_ID_TYPE = "type";
-  public static final String SCHEMA_FIELD_ID_METADATA = "filesystemMetadata";
+  public static final String KEY_FIELD_NAME = "key";
+  public static final String SCHEMA_FIELD_NAME_TYPE = "type";
+  public static final String SCHEMA_FIELD_NAME_METADATA = "filesystemMetadata";
+  public static final String SCHEMA_FIELD_NAME_RECORD_INDEX = "recordIndexMetadata";
 
   // Type of the record
   // This can be an enum in the schema but Avro 1.8 has a bug - https://issues.apache.org/jira/browse/AVRO-1810
   private static final int PARTITION_LIST = 1;
   private static final int FILE_LIST = 2;
+  private static final int RECORD_INDEX = 3;
+
+  // FileIndex value saved when the fileId has no index (old format)
+  private static final int MISSING_FILEINDEX = -1;
 
   private String key = null;
   private int type = 0;
-  private Map filesystemMetadata = null;
+  private Map filesystemMetadata;
+  private HoodieRecordIndexInfo recordIndexInfo;
 
   public HoodieMetadataPayload(GenericRecord record, Comparable orderingVal) {
     this(Option.of(record));
@@ -84,14 +94,21 @@ public HoodieMetadataPayload(Option record) {
     if (record.isPresent()) {
       // This can be simplified using SpecificData.deepcopy once this bug is fixed
       // https://issues.apache.org/jira/browse/AVRO-1811
-      key = record.get().get(SCHEMA_FIELD_ID_KEY).toString();
-      type = (int) record.get().get(SCHEMA_FIELD_ID_TYPE);
-      if (record.get().get(SCHEMA_FIELD_ID_METADATA) != null) {
-        filesystemMetadata = (Map) record.get().get("filesystemMetadata");
+      key = record.get().get(KEY_FIELD_NAME).toString();
+      type = (int) record.get().get(SCHEMA_FIELD_NAME_TYPE);
+      if (record.get().get(SCHEMA_FIELD_NAME_METADATA) != null) {
+        filesystemMetadata = (Map) record.get().get(SCHEMA_FIELD_NAME_METADATA);
         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 (record.get().get(SCHEMA_FIELD_NAME_RECORD_INDEX) != null) {
+        GenericRecord recordIndexMetadata = (GenericRecord) record.get().get(SCHEMA_FIELD_NAME_RECORD_INDEX);
+        recordIndexInfo = new HoodieRecordIndexInfo(recordIndexMetadata.get("partition").toString(),
+            Long.parseLong(recordIndexMetadata.get("fileIdHighBits").toString()),
+            Long.parseLong(recordIndexMetadata.get("fileIdLowBits").toString()),
+            Integer.parseInt(recordIndexMetadata.get("fileIndex").toString()),
+            Long.parseLong(recordIndexMetadata.get("instantTime").toString()));
       }
     }
   }
@@ -102,6 +119,14 @@ private HoodieMetadataPayload(String key, int type, Map createPartitionFilesRecord(Str
     return new HoodieRecord<>(key, payload);
   }
 
+  /**
+   * Create and return a {@code HoodieMetadataPayload} to insert or update an entry for the record 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 createRecordIndexUpdate(String recordKey, String partition,
+      String fileId, String instantTime) {
+    HoodieKey key = new HoodieKey(recordKey, MetadataPartitionType.RECORD_INDEX.partitionPath());
+    // Data file names have a -D suffix to denote the index (D = integer) of the file written
+    // In older HUID versions the file index was missing
+    final UUID uuid;
+    final int fileIndex;
+    try {
+      if (fileId.length() == 36) {
+        uuid = UUID.fromString(fileId);
+        fileIndex = MISSING_FILEINDEX;
+      } else {
+        final int index = fileId.lastIndexOf("-");
+        uuid = UUID.fromString(fileId.substring(0, index));
+        fileIndex = Integer.parseInt(fileId.substring(index + 1));
+      }
+    } catch (Exception e) {
+      throw new HoodieMetadataException(String.format("Invalid UUID or index: fileID=%s, partition=%s, instantTIme=%s",
+          fileId, partition, instantTime), e);
+    }
+
+    // Store instantTime as milliseconds sinch epoch. Using an int here allows dates till year 2038.
+    Date instantDate;
+    try {
+      instantDate = HoodieActiveTimeline.parseDateFromInstantTime(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,
+        instantDate.getTime()));
+    return new HoodieRecord<>(key, payload);
+  }
+
+  /**
+   * Create and return a {@code HoodieMetadataPayload} to delete a record in the Metadata Table's record index.
+   *
+   * @param recordKey Key of the record to be deleted
+   */
+  public static HoodieRecord createRecordIndexDelete(String recordKey) {
+    HoodieKey key = new HoodieKey(recordKey, MetadataPartitionType.RECORD_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;
+    if (previousRecord.type != type) {
+      throw new HoodieMetadataException("Cannot combine " + previousRecord.type  + " with " + type);
+    }
+    if (!previousRecord.key.equals(key)) {
+      throw new HoodieMetadataException("Cannot combine record with key " + previousRecord.key + " with " + key);
+    }
 
     switch (type) {
       case PARTITION_LIST:
       case FILE_LIST:
-        combinedFileInfo = combineFilesystemMetadata(previousRecord);
-        break;
+        Map combinedFileInfo = combineFilesystemMetadata(previousRecord);
+        return new HoodieMetadataPayload(key, type, combinedFileInfo);
+      case RECORD_INDEX:
+        // TODO: does not work with updates
+        if (previousRecord.recordIndexInfo.getInstantTime() != recordIndexInfo.getInstantTime()) {
+          throw new HoodieMetadataException(String.format("InstantTime for %s should not change from %s to %s", previousRecord.key,
+                previousRecord, toString()));
+        }
+        // TODO: This does not work with clustering
+        if (!previousRecord.getRecordGlobalLocation().equals(getRecordGlobalLocation())) {
+          throw new HoodieMetadataException(String.format("Location for %s should not change from %s to %s", previousRecord.key,
+                previousRecord, toString()));
+        }
+        return this;
       default:
         throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type);
     }
-
-    return new HoodieMetadataPayload(key, type, combinedFileInfo);
   }
 
   @Override
@@ -168,7 +261,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);
   }
 
@@ -198,6 +291,20 @@ 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 HoodieRecordGlobalLocation getRecordGlobalLocation() {
+    final UUID uuid = new UUID(recordIndexInfo.getFileIdHighBits(), recordIndexInfo.getFileIdLowBits());
+    final String partition = recordIndexInfo.getPartition();
+    String fileId = uuid.toString();
+    if (recordIndexInfo.getFileIndex() != MISSING_FILEINDEX) {
+      fileId += "-" + String.valueOf(recordIndexInfo.getFileIndex());
+    }
+    final Date instantDate = new Date(recordIndexInfo.getInstantTime());
+    return new HoodieRecordGlobalLocation(partition, HoodieActiveTimeline.formatDate(instantDate), fileId);
+  }
+
   private Stream> filterFileInfoEntries(boolean isDeleted) {
     if (filesystemMetadata == null) {
       return Stream.empty();
@@ -237,10 +344,14 @@ private Map combineFilesystemMetadata(HoodieMeta
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {");
-    sb.append(SCHEMA_FIELD_ID_KEY + "=").append(key).append(", ");
-    sb.append(SCHEMA_FIELD_ID_TYPE + "=").append(type).append(", ");
-    sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", ");
-    sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", ");
+    sb.append(KEY_FIELD_NAME + "=").append(key).append(", ");
+    sb.append(SCHEMA_FIELD_NAME_TYPE + "=").append(type).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_INDEX) {
+      sb.append("location=").append(getRecordGlobalLocation());
+    }
     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 d981b7085195b..1e05f073423bd 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,11 @@
 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.HoodieRecordGlobalLocation;
 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.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
@@ -53,6 +55,9 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
   // Base path of the Metadata Table relative to the dataset (.hoodie/metadata)
   static final String METADATA_TABLE_REL_PATH = HoodieTableMetaClient.METAFOLDER_NAME + Path.SEPARATOR + "metadata";
 
+  // Name of the metrics registry
+  static final String METRIC_REGISTRY_NAME = "HoodieMetadata";
+
   /**
    * Return the base path of the Metadata Table.
    *
@@ -74,19 +79,26 @@ static boolean isMetadataTable(String basePath) {
     return basePath.endsWith(METADATA_TABLE_REL_PATH);
   }
 
-  static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath,
-                                    String spillableMapPath) {
-    return create(engineContext, metadataConfig, datasetBasePath, spillableMapPath, false);
+  /**
+   * This method checks if metadata table is configured on the main table.
+   * @param fs Hadoop FileSystem object
+   * @param tableBasePath Basepath of the main table.
+   * @return true if metadata is configured or else returns false.
+   */
+  public static boolean isMetadataTableConfigured(FileSystem fs, String tableBasePath) throws IOException {
+    Path metadataTableBasePath = new Path(getMetadataTableBasePath(tableBasePath));
+    return fs.exists(metadataTableBasePath);
   }
 
-  static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath,
-                                    String spillableMapPath, boolean reuse) {
-    if (metadataConfig.enabled()) {
-      return new HoodieBackedTableMetadata(engineContext, metadataConfig, datasetBasePath, spillableMapPath, reuse);
-    } else {
-      return new FileSystemBackedTableMetadata(engineContext, new SerializableConfiguration(engineContext.getHadoopConf()),
-          datasetBasePath, metadataConfig.shouldAssumeDatePartitioning());
+  static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
+                                    String datasetBasePath) {
+    HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(engineContext, metadataConfig, datasetBasePath);
+    if (tableMetadata.enabled()) {
+      return tableMetadata;
     }
+
+    return new FileSystemBackedTableMetadata(engineContext, new SerializableConfiguration(engineContext.getHadoopConf()),
+          datasetBasePath, metadataConfig.shouldAssumeDatePartitioning());
   }
 
   /**
@@ -105,7 +117,12 @@ static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetad
   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 record keys which are found in the record index.
+   */
+  public Map readRecordIndex(List recordKeys);
+
+  /**
+   * Returns the timestamp of the latest synced instant.
    */
   Option getSyncedInstantTime();
 
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 58d63a194e81d..7602a114cbe07 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
@@ -21,11 +21,13 @@
 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.config.HoodieMetadataConfig;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 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.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
@@ -35,12 +37,12 @@
 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.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
+import java.io.FileNotFoundException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -62,20 +64,56 @@ public class HoodieTableMetadataUtil {
 
   private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class);
 
+  // Suffix to use for bootstrapping additional indexes. Should be less than other suffixes.
+  private static final String INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX = "001";
+  // Suffix to use for compaction
+  private static final String COMPACTION_TIMESTAMP_SUFFIX = "002";
+  // Suffix to use for clean
+  private static final String CLEAN_TIMESTAMP_SUFFIX = "003";
+
   /**
    * Delete the metadata table for the dataset. This will be invoked during upgrade/downgrade operation during which no other
    * process should be running.
    *
-   * @param basePath base path of the dataset
+   * @param metaClient {@code HoodieTableMetaClient} of the dataset for which metadata table is to be deleted
    * @param context instance of {@link HoodieEngineContext}.
+   * @param backup Whether metadata table should be backed up before deletion. If true, the table is backed up to the
+   *               directory with name metadata_.
    */
-  public static void deleteMetadataTable(String basePath, HoodieEngineContext context) {
-    final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
-    FileSystem fs = FSUtils.getFs(metadataTablePath, context.getHadoopConf().get());
+  public static void deleteMetadataTable(HoodieTableMetaClient dataMetaClient, HoodieEngineContext context, boolean backup) {
+    final Path metadataTablePath = new Path(HoodieTableMetadata.getMetadataTableBasePath(dataMetaClient.getBasePath()));
+    FileSystem fs = FSUtils.getFs(metadataTablePath.toString(), context.getHadoopConf().get());
+    setMetadataPartitionState(dataMetaClient, MetadataPartitionType.FILES, false);
+    try {
+      if (!fs.exists(metadataTablePath)) {
+        return;
+      }
+    } catch (FileNotFoundException e) {
+      // Ignoring exception as metadata table already does not exist
+      LOG.debug("Metadata table not found at path " + metadataTablePath);
+      return;
+    } catch (Exception e) {
+      throw new HoodieMetadataException("Failed to check metadata table existence", e);
+    }
+
+    if (backup) {
+      final Path metadataBackupPath = new Path(metadataTablePath.getParent(), "metadata_" + HoodieActiveTimeline.createNewInstantTime());
+      LOG.info("Backing up metadata directory to " + metadataBackupPath + " before deletion");
+      try {
+        if (fs.rename(metadataTablePath, metadataBackupPath)) {
+          return;
+        }
+      } catch (Exception e) {
+        // If rename fails, we will try to delete the table instead
+        LOG.error("Failed to backup metadata table using rename", e);
+      }
+    }
+
+    LOG.info("Deleting metadata table from " + metadataTablePath);
     try {
-      fs.delete(new Path(metadataTablePath), true);
+      fs.delete(metadataTablePath, true);
     } catch (Exception e) {
-      throw new HoodieMetadataException("Failed to remove metadata table from path " + metadataTablePath, e);
+      throw new HoodieMetadataException("Failed to delete metadata table from path " + metadataTablePath, e);
     }
   }
 
@@ -89,6 +127,7 @@ public static void deleteMetadataTable(String basePath, HoodieEngineContext cont
   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(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionStatName;
       allPartitions.add(partition);
@@ -113,14 +152,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;
   }
 
@@ -399,4 +439,124 @@ private static List getPartitionFileSlices(HoodieTableMetaClient meta
     return fileSliceStream.sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())).collect(Collectors.toList());
   }
 
+  /**
+   * Get the file slices for a given partition which have been initialized but dont appear in the timeline yet.
+   *
+   * @param metaClient      - Instance of {@link HoodieTableMetaClient}.
+   * @param partition       - The name of the partition whose file groups are to be loaded.
+   * @return List of latest file slices for all file groups in a given partition.
+   */
+  public static List getBootstrappedFileSlices(HoodieTableMetaClient metaClient, String partition) {
+    final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION,
+        HoodieActiveTimeline.createNewInstantTime());
+    HoodieDefaultTimeline timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails);
+
+    HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, timeline);
+    Stream fileSliceStream;
+    fileSliceStream = fsView.getLatestFileSlices(partition);
+    return fileSliceStream.sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())).collect(Collectors.toList());
+  }
+
+  /**
+   * Returns a {@code HoodieTableMetaClient} for the metadata table.
+   *
+   * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset.
+   * @return {@code HoodieTableMetaClient} for the metadata table.
+   */
+  public static HoodieTableMetaClient getMetadataTableMetaClient(HoodieTableMetaClient datasetMetaClient) {
+    final String metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetMetaClient.getBasePath());
+    return HoodieTableMetaClient.builder().setBasePath(metadataBasePath).setConf(datasetMetaClient.getHadoopConf())
+        .build();
+  }
+
+  /**
+   * Create the timestamp for a clean operation on the metadata table.
+   */
+  public static String createCleanTimestamp(String timestamp) {
+    return timestamp + CLEAN_TIMESTAMP_SUFFIX;
+  }
+
+  /**
+   * Create the timestamp for a compaction operation on the metadata table.
+   */
+  public static String createCompactionTimestamp(String timestamp) {
+    return timestamp + COMPACTION_TIMESTAMP_SUFFIX;
+  }
+
+  /**
+   * Create the timestamp for an index initialization operation on the metadata table.
+   */
+  public static String createIndexInitTimestamp(String timestamp) {
+    return timestamp + INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX;
+  }
+
+  /**
+   * Returns true if the given instant represents an index bootstrap operation on the metadata table.
+   *
+   * When an index is bootstrapped on its own, it will use a instant time which has the suffix
+   * INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX. Suppose the last deltacommit on the metadata table had the timestamp t1 then
+   * the index bootstrap operation will have the timstamp t1 + INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX = t1001.
+   *
+   * We have to satisfy the following constraints:
+   * 1. This timestamp t1001 wont be present on the dataset timeline.
+   * 2. The deltacommit t1 may itself have been archived.
+   * 3. Dataset instants have millisecond resolution timestamp so they themselves may have timestamp which is ending in
+   *    INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX. We need to ignore such instants in this method.
+   *
+   * If the HUDI generated timestamps have millisecond resolution and length 17, then due to the suffix of
+   * INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX, the length of index timestamp will be 20. This additional check ensures that
+   * we do not assume any dataset operation's timestamp ending in INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX as a bootstrap index
+   * operation.
+   *
+   */
+  public static boolean isIndexInitInstant(HoodieInstant instant) {
+    final String newInstantTime = HoodieActiveTimeline.createNewInstantTime();
+    if (!instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)
+        || !instant.getTimestamp().endsWith(INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX)
+        || instant.getTimestamp().length() != (newInstantTime.length() + INDEX_BOOTSTRAP_TIMESTAMP_SUFFIX.length())) {
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Set the state of the metadata table.
+   * @param dataMetaClient MetaClient for the dataset
+   * @param enabled If true, metadata table is being used for this dataset, false otherwise
+   */
+  public static HoodieTableMetaClient setMetadataPartitionState(HoodieTableMetaClient dataMetaClient, MetadataPartitionType partition,
+      boolean enabled) {
+    dataMetaClient.getTableConfig().setMetadataPartitionState(partition, enabled);
+    HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps());
+    dataMetaClient = HoodieTableMetaClient.reload(dataMetaClient);
+    ValidationUtils.checkState(dataMetaClient.getTableConfig().isMetadataPartitionEnabled(partition) == enabled,
+        "Metadata table state change should be persisted");
+
+    LOG.info(String.format("Metadata table %s partition %s has been %s", dataMetaClient.getBasePath(), partition,
+        enabled ? "enabled" : "disabled"));
+    return dataMetaClient;
+  }
+
+  /**
+   * Returns true if any enabled metadata partition in the given hoodie table requires WriteStatus to track the
+   * written records.
+   * @param config
+   *
+   * @param hoodieTable HoodieTable
+   * @return true if WriteStatus should track the written records else false.
+   */
+  public static boolean needsWriteStatusTracking(HoodieMetadataConfig config, HoodieTableMetaClient metaClient) {
+    // Does any enabled partition need to track the written records
+    if (MetadataPartitionType.needWriteStatusTracking().stream().anyMatch(p -> metaClient.getTableConfig().isMetadataPartitionEnabled(p))) {
+      return true;
+    }
+
+    // Does any enabled partition being enabled need to track the written records
+    if (config.createRecordIndex()) {
+      return true;
+    }
+
+    return false;
+  }
 }
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 380f4d04d34a6..f68a1b8fc2c7d 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
@@ -22,7 +22,8 @@
 import java.util.List;
 
 public enum MetadataPartitionType {
-  FILES("files", "files-");
+  FILES("files", "files-"),
+  RECORD_INDEX("record_index", "record-index-");
 
   // refers to partition path in metadata table.
   private final String partitionPath;
@@ -42,7 +43,16 @@ public String getFileIdPrefix() {
     return fileIdPrefix;
   }
 
-  public static List all() {
-    return Arrays.asList(MetadataPartitionType.FILES.partitionPath());
+  public static List all() {
+    return Arrays.asList(MetadataPartitionType.FILES, MetadataPartitionType.RECORD_INDEX);
+  }
+
+  /**
+   * Returns the list of metadata table partitions which require WriteStatus to track written records.
+   *
+   * These partitions need the list of written records so that they can update their metadata.
+   */
+  public static List needWriteStatusTracking() {
+    return Arrays.asList(MetadataPartitionType.RECORD_INDEX);
   }
 }
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java
index 32dfd7c1e4ec2..b5318890b4670 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java
@@ -183,7 +183,7 @@ public boolean accept(Path path) {
           }
 
           fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext,
-              metaClient, HoodieInputFormatUtils.buildMetadataConfig(getConf()));
+              metaClient, HoodieInputFormatUtils.useMetadataTable(getConf()));
           String partition = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), folder);
           List latestFiles = fsView.getLatestBaseFiles(partition).collect(Collectors.toList());
           // populate the cache
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 4f4b69f9813ac..135c5259b90aa 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
@@ -18,7 +18,6 @@
 
 package org.apache.hudi.hadoop.utils;
 
-import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.engine.HoodieLocalEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.FileSlice;
@@ -433,10 +432,8 @@ public static Map> groupSnapshotPathsByMetaCli
     return grouped;
   }
 
-  public static HoodieMetadataConfig buildMetadataConfig(Configuration conf) {
-    return HoodieMetadataConfig.newBuilder()
-        .enable(conf.getBoolean(ENABLE.key(), DEFAULT_METADATA_ENABLE_FOR_READERS))
-        .build();
+  public static boolean useMetadataTable(Configuration conf) {
+    return conf.getBoolean(ENABLE.key(), DEFAULT_METADATA_ENABLE_FOR_READERS);
   }
 
   public static List filterFileStatusForSnapshotMode(JobConf job, Map tableMetaClientMap,
@@ -464,7 +461,7 @@ public static List filterFileStatusForSnapshotMode(JobConf job, Map<
         HoodieTimeline timeline = HoodieHiveUtils.getTableTimeline(metaClient.getTableConfig().getTableName(), job, metaClient);
 
         HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(metaClient, tableMetaClient ->
-            FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, tableMetaClient, buildMetadataConfig(job), timeline));
+            FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, tableMetaClient, useMetadataTable(job), timeline));
         List filteredBaseFiles = new ArrayList<>();
         Map> filteredLogs = new HashMap<>();
         for (Path p : entry.getValue()) {
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java
index 6718642d22728..cc9fdf6c9c982 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java
@@ -95,7 +95,7 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream writeStatList) {
     try {
-      writeClient.commitStats(instantTime, writeStatList, Option.of(extraMetadata),
+      writeClient.commitStats(instantTime, writeClient.getEngineContext().emptyHoodieData(), writeStatList, Option.of(extraMetadata),
           CommitUtils.getCommitActionType(operationType, metaClient.getTableType()));
     } catch (Exception ioe) {
       throw new HoodieException(ioe.getMessage(), ioe);