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/01/23 21:52:37 UTC

[GitHub] [hudi] nsivabalan commented on a diff in pull request #7739: [HUDI-5593] Fixing deadlocks due to async cleaner interplay w/ main thread

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##########
@@ -308,26 +308,6 @@ public boolean scheduleClusteringAtInstant(String instantTime, Option<Map<String
     return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLUSTER).isPresent();
   }
 
-  /**
-   * Schedules a new cleaning instant.

Review Comment:
   NTR (not to reviewer): these methods are not used. removing dead code. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##########
@@ -527,16 +507,31 @@ protected void writeTableMetadata(HoodieTable table, String instantTime, String
    *
    * @param cleanInstantTime instant time for clean.
    * @param scheduleInline   true if needs to be scheduled inline. false otherwise.
-   * @param skipLocking      if this is triggered by another parent transaction, locking can be skipped.
    */
   @Nullable
+  @Deprecated

Review Comment:
   NTR: have deprecated the methods which was taking in "skipLocking" as an argument as we don't need it anymore. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##########
@@ -670,23 +664,18 @@ protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos
    * @return true if rollback was triggered. false otherwise.
    */
   protected Boolean rollbackFailedWrites() {
-    return rollbackFailedWrites(false);
-  }
-
-  /**
-   * Rollback all failed writes.
-   * @param skipLocking if this is triggered by another parent transaction, locking can be skipped.
-   * @return true if rollback was triggered. false otherwise.
-   */
-  protected Boolean rollbackFailedWrites(boolean skipLocking) {
     HoodieTable table = createTable(config, hadoopConf);
     List<String> instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy(), Option.empty());
     Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient());
     instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
-    rollbackFailedWrites(pendingRollbacks, skipLocking);
+    rollbackFailedWrites(pendingRollbacks);
     return !pendingRollbacks.isEmpty();
   }
 
+  protected void rollbackFailedWrites(Map<String, Option<HoodiePendingRollbackInfo>> instantsToRollback) {
+    rollbackFailedWrites(instantsToRollback, false);
+  }
+

Review Comment:
   NTR: 
   rollbackFailedWrites in L679, is being invoked from upgrade code path which is within lock. so, could not deprecate this method to avoid "skipLocking" 



-- 
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