-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-3365] Make sure Metadata Table records are updated appropriately on HDFS #4739
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
0b79b1d
a8e3cf0
4a2a40b
682bdbc
5d5641a
ff6b34d
b2ebb92
fe16268
11e5ecd
eca9828
ef1ab67
8dc1027
410f4d7
0c2028f
771a85d
3a0e432
89aa0ca
72d4237
44d3c26
b211322
eab17ed
2da986b
834260e
cfcaf5d
0ac5c2a
157095d
b3136ad
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -18,6 +18,8 @@ | |||
|
|
||||
| package org.apache.hudi.client; | ||||
|
|
||||
| import com.codahale.metrics.Timer; | ||||
| import org.apache.hadoop.conf.Configuration; | ||||
| import org.apache.hudi.async.AsyncArchiveService; | ||||
| import org.apache.hudi.async.AsyncCleanerService; | ||||
| import org.apache.hudi.avro.model.HoodieCleanMetadata; | ||||
|
|
@@ -45,6 +47,7 @@ | |||
| import org.apache.hudi.common.model.TableServiceType; | ||||
| import org.apache.hudi.common.model.WriteOperationType; | ||||
| import org.apache.hudi.common.table.HoodieTableMetaClient; | ||||
| import org.apache.hudi.common.table.HoodieTableVersion; | ||||
| import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; | ||||
| import org.apache.hudi.common.table.timeline.HoodieInstant; | ||||
| import org.apache.hudi.common.table.timeline.HoodieInstant.State; | ||||
|
|
@@ -73,9 +76,8 @@ | |||
| import org.apache.hudi.table.action.rollback.RollbackUtils; | ||||
| import org.apache.hudi.table.action.savepoint.SavepointHelpers; | ||||
| import org.apache.hudi.table.marker.WriteMarkersFactory; | ||||
|
|
||||
| import com.codahale.metrics.Timer; | ||||
| import org.apache.hadoop.conf.Configuration; | ||||
| import org.apache.hudi.table.upgrade.SupportsUpgradeDowngrade; | ||||
| import org.apache.hudi.table.upgrade.UpgradeDowngrade; | ||||
| import org.apache.log4j.LogManager; | ||||
| import org.apache.log4j.Logger; | ||||
|
|
||||
|
|
@@ -107,15 +109,16 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K, | |||
| private static final long serialVersionUID = 1L; | ||||
| private static final Logger LOG = LogManager.getLogger(BaseHoodieWriteClient.class); | ||||
|
|
||||
| protected final transient HoodieMetrics metrics; | ||||
| private final transient HoodieIndex<?, ?> index; | ||||
| private final SupportsUpgradeDowngrade upgradeDowngradeHelper; | ||||
| private transient WriteOperationType operationType; | ||||
| private transient HoodieWriteCommitCallback commitCallback; | ||||
|
|
||||
| protected final transient HoodieMetrics metrics; | ||||
| protected transient Timer.Context writeTimer = null; | ||||
| protected transient Timer.Context compactionTimer; | ||||
| protected transient Timer.Context clusteringTimer; | ||||
|
|
||||
| private transient WriteOperationType operationType; | ||||
| private transient HoodieWriteCommitCallback commitCallback; | ||||
| protected transient AsyncCleanerService asyncCleanerService; | ||||
| protected transient AsyncArchiveService asyncArchiveService; | ||||
| protected final TransactionManager txnManager; | ||||
|
|
@@ -125,25 +128,32 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K, | |||
| * Create a write client, with new hudi index. | ||||
| * @param context HoodieEngineContext | ||||
| * @param writeConfig instance of HoodieWriteConfig | ||||
| * @param upgradeDowngradeHelper engine-specific instance of {@link SupportsUpgradeDowngrade} | ||||
| */ | ||||
| @Deprecated | ||||
| public BaseHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig) { | ||||
| this(context, writeConfig, Option.empty()); | ||||
| public BaseHoodieWriteClient(HoodieEngineContext context, | ||||
| HoodieWriteConfig writeConfig, | ||||
| SupportsUpgradeDowngrade upgradeDowngradeHelper) { | ||||
| this(context, writeConfig, Option.empty(), upgradeDowngradeHelper); | ||||
| } | ||||
|
|
||||
| /** | ||||
| * Create a write client, allows to specify all parameters. | ||||
| * | ||||
| * @param context HoodieEngineContext | ||||
| * @param writeConfig instance of HoodieWriteConfig | ||||
| * @param writeConfig instance of HoodieWriteConfig | ||||
| * @param timelineService Timeline Service that runs as part of write client. | ||||
| */ | ||||
| @Deprecated | ||||
| public BaseHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, | ||||
| Option<EmbeddedTimelineService> timelineService) { | ||||
| public BaseHoodieWriteClient(HoodieEngineContext context, | ||||
| HoodieWriteConfig writeConfig, | ||||
| Option<EmbeddedTimelineService> timelineService, | ||||
| SupportsUpgradeDowngrade upgradeDowngradeHelper) { | ||||
| super(context, writeConfig, timelineService); | ||||
| this.metrics = new HoodieMetrics(config); | ||||
| this.index = createIndex(writeConfig); | ||||
| this.txnManager = new TransactionManager(config, fs); | ||||
| this.upgradeDowngradeHelper = upgradeDowngradeHelper; | ||||
| } | ||||
|
|
||||
| protected abstract HoodieIndex<?, ?> createIndex(HoodieWriteConfig writeConfig); | ||||
|
|
@@ -291,15 +301,15 @@ public void bootstrap(Option<Map<String, String>> extraMetadata) { | |||
| if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { | ||||
| throw new HoodieException("Cannot bootstrap the table in multi-writer mode"); | ||||
| } | ||||
| HoodieTable<T, I, K, O> table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS); | ||||
| HoodieTable<T, I, K, O> table = initTable(WriteOperationType.UPSERT, Option.ofNullable(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)); | ||||
| rollbackFailedBootstrap(); | ||||
| table.bootstrap(context, extraMetadata); | ||||
| } | ||||
|
|
||||
| /** | ||||
| * Main API to rollback failed bootstrap. | ||||
| */ | ||||
| public void rollbackFailedBootstrap() { | ||||
| protected void rollbackFailedBootstrap() { | ||||
| LOG.info("Rolling back pending bootstrap if present"); | ||||
| HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); | ||||
| HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); | ||||
|
|
@@ -628,33 +638,19 @@ public void deleteSavepoint(String savepointTime) { | |||
| * @return true if the savepoint was restored to successfully | ||||
| */ | ||||
| public void restoreToSavepoint(String savepointTime) { | ||||
| HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); | ||||
| HoodieTable<T, I, K, O> table = initTable(WriteOperationType.UNKNOWN, Option.empty()); | ||||
| SavepointHelpers.validateSavepointPresence(table, savepointTime); | ||||
| restoreToInstant(savepointTime); | ||||
| SavepointHelpers.validateSavepointRestore(table, savepointTime); | ||||
| } | ||||
|
|
||||
| @Deprecated | ||||
| public boolean rollback(final String commitInstantTime) throws HoodieRollbackException { | ||||
| HoodieTable<T, I, K, O> table = createTable(config, hadoopConf); | ||||
| HoodieTable<T, I, K, O> table = initTable(WriteOperationType.UNKNOWN, Option.empty()); | ||||
| Option<HoodiePendingRollbackInfo> pendingRollbackInfo = getPendingRollbackInfo(table.getMetaClient(), commitInstantTime); | ||||
| return rollback(commitInstantTime, pendingRollbackInfo, false); | ||||
| } | ||||
|
|
||||
| /** | ||||
| * @Deprecated | ||||
| * Rollback the inflight record changes with the given commit time. This | ||||
| * will be removed in future in favor of {@link BaseHoodieWriteClient#restoreToInstant(String)} | ||||
| * Adding this api for backwards compatability. | ||||
| * @param commitInstantTime Instant time of the commit | ||||
| * @param skipLocking if this is triggered by another parent transaction, locking can be skipped. | ||||
| * @throws HoodieRollbackException if rollback cannot be performed successfully | ||||
| */ | ||||
| @Deprecated | ||||
| public boolean rollback(final String commitInstantTime, boolean skipLocking) throws HoodieRollbackException { | ||||
| return rollback(commitInstantTime, Option.empty(), skipLocking); | ||||
| } | ||||
|
|
||||
| /** | ||||
| * @Deprecated | ||||
| * Rollback the inflight record changes with the given commit time. This | ||||
|
|
@@ -711,7 +707,7 @@ public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws H | |||
| final String restoreInstantTime = HoodieActiveTimeline.createNewInstantTime(); | ||||
| Timer.Context timerContext = metrics.getRollbackCtx(); | ||||
| try { | ||||
| HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); | ||||
| HoodieTable<T, I, K, O> table = initTable(WriteOperationType.UNKNOWN, Option.empty()); | ||||
| Option<HoodieRestorePlan> restorePlanOption = table.scheduleRestore(context, restoreInstantTime, instantTime); | ||||
| if (restorePlanOption.isPresent()) { | ||||
| HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime); | ||||
|
|
@@ -988,15 +984,15 @@ protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos | |||
| /** | ||||
| * Rollback all failed writes. | ||||
| */ | ||||
| public Boolean rollbackFailedWrites() { | ||||
| protected Boolean rollbackFailedWrites() { | ||||
| return rollbackFailedWrites(false); | ||||
| } | ||||
|
|
||||
| /** | ||||
| * Rollback all failed writes. | ||||
| * @param skipLocking if this is triggered by another parent transaction, locking can be skipped. | ||||
| */ | ||||
| public Boolean rollbackFailedWrites(boolean skipLocking) { | ||||
| protected Boolean rollbackFailedWrites(boolean skipLocking) { | ||||
| HoodieTable<T, I, K, O> table = createTable(config, hadoopConf); | ||||
| List<String> instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy(), Option.empty()); | ||||
| Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient()); | ||||
|
|
@@ -1246,17 +1242,79 @@ public HoodieMetrics getMetrics() { | |||
| } | ||||
|
|
||||
| /** | ||||
| * Get HoodieTable and init {@link Timer.Context}. | ||||
| * Instantiates engine-specific instance of {@link HoodieTable} as well as performs necessary | ||||
| * bootstrapping operations (for ex, validating whether Metadata Table has to be bootstrapped) | ||||
| * | ||||
| * NOTE: THIS OPERATION IS EXECUTED UNDER LOCK, THEREFORE SHOULD AVOID ANY OPERATIONS | ||||
| * NOT REQUIRING EXTERNAL SYNCHRONIZATION | ||||
| * | ||||
| * @param operationType write operation type | ||||
| * @param metaClient instance of {@link HoodieTableMetaClient} | ||||
| * @param instantTime current inflight instant time | ||||
| * @return HoodieTable | ||||
| * @return instantiated {@link HoodieTable} | ||||
| */ | ||||
| protected abstract HoodieTable<T, I, K, O> getTableAndInitCtx(WriteOperationType operationType, String instantTime); | ||||
| protected abstract HoodieTable<T, I, K, O> doInitTable(HoodieTableMetaClient metaClient, Option<String> instantTime); | ||||
|
|
||||
| /** | ||||
| * Sets write schema from last instant since deletes may not have schema set in the config. | ||||
| * Instantiates and initializes instance of {@link HoodieTable}, performing crucial bootstrapping | ||||
| * operations such as: | ||||
| * | ||||
| * NOTE: This method is engine-agnostic and SHOULD NOT be overloaded, please check on | ||||
| * {@link #doInitTable(HoodieTableMetaClient, Option<String>)} instead | ||||
| * | ||||
| * <ul> | ||||
| * <li>Checking whether upgrade/downgrade is required</li> | ||||
| * <li>Bootstrapping Metadata Table (if required)</li> | ||||
| * <li>Initializing metrics contexts</li> | ||||
| * </ul> | ||||
| */ | ||||
| protected final HoodieTable<T, I, K, O> initTable(WriteOperationType operationType, Option<String> instantTime) { | ||||
| HoodieTableMetaClient metaClient = createMetaClient(true); | ||||
| // Setup write schemas for deletes | ||||
| if (operationType == WriteOperationType.DELETE) { | ||||
| setWriteSchemaForDeletes(metaClient); | ||||
| } | ||||
|
|
||||
| HoodieTable<T, I, K, O> table; | ||||
|
|
||||
| this.txnManager.beginTransaction(); | ||||
| try { | ||||
| tryUpgrade(metaClient, instantTime); | ||||
| table = doInitTable(metaClient, instantTime); | ||||
| } finally { | ||||
| this.txnManager.endTransaction(); | ||||
| } | ||||
|
|
||||
| // Validate table properties | ||||
| metaClient.validateTableProperties(config.getProps(), operationType); | ||||
| // Make sure that FS View is in sync | ||||
| table.getHoodieView().sync(); | ||||
|
|
||||
| switch (operationType) { | ||||
| case INSERT: | ||||
| case INSERT_PREPPED: | ||||
| case UPSERT: | ||||
| case UPSERT_PREPPED: | ||||
| case BULK_INSERT: | ||||
| case BULK_INSERT_PREPPED: | ||||
| case INSERT_OVERWRITE: | ||||
| case INSERT_OVERWRITE_TABLE: | ||||
| setWriteTimer(table); | ||||
| break; | ||||
| case CLUSTER: | ||||
| clusteringTimer = metrics.getClusteringCtx(); | ||||
| break; | ||||
| case COMPACT: | ||||
| compactionTimer = metrics.getCompactionCtx(); | ||||
| break; | ||||
| default: | ||||
| } | ||||
|
|
||||
| return table; | ||||
| } | ||||
|
|
||||
| /** | ||||
| * Sets write schema from last instant since deletes may not have schema set in the config. | ||||
| */ | ||||
| protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { | ||||
| try { | ||||
| HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); | ||||
|
|
@@ -1301,4 +1359,33 @@ public void close() { | |||
| this.heartbeatClient.stop(); | ||||
| this.txnManager.close(); | ||||
| } | ||||
|
|
||||
| private void setWriteTimer(HoodieTable<T, I, K, O> table) { | ||||
| String commitType = table.getMetaClient().getCommitActionType(); | ||||
| if (commitType.equals(HoodieTimeline.COMMIT_ACTION)) { | ||||
| writeTimer = metrics.getCommitCtx(); | ||||
| } else if (commitType.equals(HoodieTimeline.DELTA_COMMIT_ACTION)) { | ||||
| writeTimer = metrics.getDeltaCommitCtx(); | ||||
| } | ||||
| } | ||||
|
|
||||
| private void tryUpgrade(HoodieTableMetaClient metaClient, Option<String> instantTime) { | ||||
| UpgradeDowngrade upgradeDowngrade = | ||||
| new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper); | ||||
|
|
||||
| if (upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) { | ||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. as per master, this code is available only in SparkEngine. Flink and java does not have this. Even if we wish to unify, I would do it in a separate patch and getting it reviewed by experts who have worked on it. Can you move this to SparkRDDWriteClient for now.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think this is accurate -- Flink is also using Can you please elaborate what your concern around it is? Frankly, am not a fan that i had to do budge this refactoring in this PR, but after adding validations into
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am fine. but just that I want to get a stamp it from devs who work in flink and java. Just don't want to miss anything, bcoz, not many times every engine follows same logic. I checked the code and refactoring once again, seems to be fine.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @yihua : we are making some minor refactoring relating to upgradedowngrade invocation for java engine. just wanted to keep you in the loop. Prior to this patch, we did not have any explicit upgrade downgrade. Just wanted to ensure unification does not cause any issues. if you can review it once and stamp it, would be nice. |
||||
| // Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits | ||||
| List<String> instantsToRollback = getInstantsToRollback(metaClient, HoodieFailedWritesCleaningPolicy.EAGER, instantTime); | ||||
|
|
||||
| Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(metaClient); | ||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. May i know why we rollback the failed commits before doing the upgrade ? We already try to do that when start commit: hudi/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java Line 935 in 1562bb6
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The code was migrated as is so can't speak up from historical context, but my hunch is that we do that to make sure table is in a consistent state (no leftovers of failed commits) when we start the upgrade process.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
My confusion is why we need to do that for upgrade ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The code was doing it in some branches (for Spark) but not in others. This PR is simply reconciling that and making behavior consistent across the board. In general, given that upgrade/downgrade could be acting upon and modifying table's current state it has to be in a consistent one.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I'm wondering the exact point for the background, i have two questions here:
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is how the code was in 0.9.0 and here is how it looks in 0.10.0 So, atleast from 0.9.0, its evident that we were adding this only for multi-writer scenario. i.e when someone migrates from a single writer to multi-writer, we just wanted to ensure we rollback any partially failed commits.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @danny0405 : happy to jam to see if there are any gaps here.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I have fired a fix here, i think there needs some improvement here: #5535 |
||||
| instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty())); | ||||
|
|
||||
| rollbackFailedWrites(pendingRollbacks, true); | ||||
|
|
||||
| new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper) | ||||
| .run(HoodieTableVersion.current(), instantTime.orElse(null)); | ||||
|
|
||||
| metaClient.reloadActiveTimeline(); | ||||
| } | ||||
| } | ||||
| } | ||||
Uh oh!
There was an error while loading. Please reload this page.