You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "nsivabalan (via GitHub)" <gi...@apache.org> on 2023/04/30 17:31:16 UTC

[GitHub] [hudi] nsivabalan commented on a diff in pull request #8604: [HUDI-6151] Rollback previously applied commits to MDT when operations are retried.

nsivabalan commented on code in PR #8604:
URL: https://github.com/apache/hudi/pull/8604#discussion_r1181266570


##########
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:
   Trying to gauge if we really need this. 
   
   I guess in next couple of patches, you are going to add below change:
   - Any rollback in DT will be an actual rollback in MDT as well. 
   
   having said that, lets go through this use-case. 
   
   Compaction Commit C5 is inflight in DT and succeeded in MDT, but crashed in DT. 
   so on restart, a rollback is triggered in DT. which when gets into MDT territory, will rollback the succeeded commit in MDT. So, it will be automatically taken care of. 
   
   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? 
   



##########
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:
   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. 
   On restart, any new operation in DT when it gets into MDT territory, on deducting a partial commit in MDT, a rollback will be triggered eagerly. Ref: https://github.com/apache/hudi/blob/04e54a6187d3aa4f0f05ff4f9ff4c1283a70208c/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java#L151
   
   So, this case is also taken care of. 
   



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org