nbalajee commented on code in PR #8604:
URL: https://github.com/apache/hudi/pull/8604#discussion_r1183087756
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java:
##########
@@ -161,27 +161,28 @@ protected void commit(String instantTime,
Map<MetadataPartitionType, HoodieData<
if
(!metadataMetaClient.getActiveTimeline().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 {
+ // this code path refers to a re-attempted commit that:
+ // 1. got committed to metadata table, but failed in datatable.
+ // 2. failed while commiting to metadata table
+ // 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 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();
- 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();
+ LOG.info(String.format("%s completed commit at %s being applied to
metadata table",
+ alreadyCompletedInstant.isPresent() ? "Already" : "Partially",
instantTime));
+
+ // Rollback the previous committed commit
+ if (!writeClient.rollback(instantTime)) {
Review Comment:
Older solution of removing the completed action and reattempt won't work in
all scenarios. We will have to consider the following scenarios:
(1) c1.commit failed on the main dataset; On MDT, c1.deltacommit was
completed.
(a) with record index enabled, new log block was added to the log file
by c1.deltacommit. Simply removing deltacommit, may not be enough and will
require additional action to rollback the logblock, to keep the log file
consistent.
(2) c1.clean was attempted. c1.deltacommit was completed. When clean is
retried, second attempt could bring in some of the files that were in the
"failed" list of the first attempt (vs the "success" list).
(3) c1.rollback was attempted. c1.deltacommit was completed. (We fixed an
issue with incomplete rollback, with MDT updated with deltacommit, scenario.
This change played a role in this scenario as well).
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]