-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-6151] Rollback previously applied commits to MDT when operations are retried. #8604
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
451d219
dd8db0f
e687900
370799c
eb39bc7
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 | ||
|---|---|---|---|---|
|
|
@@ -29,13 +29,13 @@ | |||
| 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.timeline.HoodieInstant; | ||||
| import org.apache.hudi.common.util.CommitUtils; | ||||
| 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.HoodieMetadataException; | ||||
| import org.apache.hudi.metrics.DistributedRegistry; | ||||
| import org.apache.hadoop.conf.Configuration; | ||||
| import org.apache.hudi.table.BulkInsertPartitioner; | ||||
|
|
@@ -144,27 +144,29 @@ private void commitInternal(String instantTime, Map<MetadataPartitionType, Hoodi | |||
|
|
||||
| if (!metadataMetaClient.getActiveTimeline().getCommitsTimeline().containsInstant(instantTime)) { | ||||
| // if this is a new commit being applied to metadata for the first time | ||||
| writeClient.startCommitWithTime(instantTime); | ||||
| LOG.info("New commit at " + instantTime + " being applied to MDT."); | ||||
| } else { | ||||
| Option<HoodieInstant> alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.getTimestamp().equals(instantTime)).lastInstant(); | ||||
| if (alreadyCompletedInstant.isPresent()) { | ||||
| // this code path refers to a re-attempted commit that got committed to metadata table, but failed in datatable. | ||||
| // for eg, lets say compaction c1 on 1st attempt succeeded in metadata table and failed before committing to datatable. | ||||
| // when retried again, data table will first rollback pending compaction. these will be applied to metadata table, but all changes | ||||
| // are upserts to metadata table and so only a new delta commit will be created. | ||||
| // once rollback is complete, compaction will be retried again, which will eventually hit this code block where the respective commit is | ||||
| // already part of completed commit. So, we have to manually remove the completed instant and proceed. | ||||
| // and it is for the same reason we enabled withAllowMultiWriteOnSameInstant for metadata table. | ||||
| HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant.get()); | ||||
| metadataMetaClient.reloadActiveTimeline(); | ||||
| // this code path refers to a re-attempted commit that: | ||||
| // 1. got committed to metadata table, but failed in datatable. | ||||
| // 2. failed while committing to metadata table | ||||
| // for e.g., let's say compaction c1 on 1st attempt succeeded in metadata table and failed before committing to datatable. | ||||
| // when retried again, data table will first rollback pending compaction. these will be applied to metadata table, but all changes | ||||
xushiyan marked this conversation as resolved.
Show resolved
Hide resolved
|
||||
| // are upserts to metadata table and so only a new delta commit will be created. | ||||
| // once rollback is complete in datatable, compaction will be retried again, which will eventually hit this code block where the respective commit is | ||||
| // already part of completed commit. So, we have to manually rollback the completed instant and proceed. | ||||
| Option<HoodieInstant> alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.getTimestamp().equals(instantTime)) | ||||
| .lastInstant(); | ||||
| LOG.info(String.format("%s completed commit at %s being applied to MDT.", | ||||
| alreadyCompletedInstant.isPresent() ? "Already" : "Partially", instantTime)); | ||||
xushiyan marked this conversation as resolved.
Show resolved
Hide resolved
|
||||
|
|
||||
| // Rollback the previous commit | ||||
| if (!writeClient.rollback(instantTime)) { | ||||
|
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. Trying to gauge if we really need this. I guess in next couple of patches, you are going to add below change:
having said that, lets go through this use-case. Compaction Commit C5 is inflight in DT and succeeded in MDT, but crashed in DT. After rollback of C5 is completed, C5 will be re-attempted in DT. and when it gets into MDT territory, there won't be any traces of DC5 at all. So, wondering when exactly we will hit this case?
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. if we are talking about a partially failed commit in MDT: Compaction Commit C5 is inflight in DT and DC5 in MDT is also partitally committed and crashed. Line 151 in 04e54a6
So, this case is also taken care of.
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.
The current code on master just removes the
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. Older solution of removing the completed action and reattempt won't work in all scenarios. We will have to consider the following scenarios:
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. Yeah, fix the rollback in sync with normal DT can avoid many potential bugs, +1 for this direction. |
||||
| throw new HoodieMetadataException("Failed to rollback deltacommit at " + instantTime + " from MDT"); | ||||
| } | ||||
| // If the alreadyCompletedInstant is empty, that means there is a requested or inflight | ||||
| // instant with the same instant time. This happens for data table clean action which | ||||
| // reuses the same instant time without rollback first. It is a no-op here as the | ||||
| // clean plan is the same, so we don't need to delete the requested and inflight instant | ||||
| // files in the active timeline. | ||||
| metadataMetaClient.reloadActiveTimeline(); | ||||
| } | ||||
|
|
||||
| writeClient.startCommitWithTime(instantTime); | ||||
| if (bulkInsertPartitioner.isPresent()) { | ||||
| writeClient.bulkInsertPreppedRecords(preppedRecordRDD, instantTime, bulkInsertPartitioner).collect(); | ||||
| } else { | ||||
|
|
||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
to MDT -> to MDT.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we sync the logic also to
FlinkHoodieBackedTableMetadataWriter?