You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/09/15 07:05:49 UTC

[GitHub] [hudi] yihua opened a new pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

yihua opened a new pull request #3664:
URL: https://github.com/apache/hudi/pull/3664


   ## What is the purpose of the pull request
   
   This PR refactors the rollback actions in the package of `org.apache.hudi.table.action.rollback` in `hudi-client` module.  The refactoring focuses on reusing the code for the core Hudi rollback logic and eliminating any unnecessary duplication.
   
   ## Brief change log
   
   ## Verify this pull request
   
   This pull request is already covered by existing tests in `org.apache.hudi.table.action.rollback` package.  All existing tests on rollback pass.
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


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



[GitHub] [hudi] nsivabalan commented on a change in pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#discussion_r709432675



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/function/FunctionWrapper.java
##########
@@ -70,4 +71,14 @@
       }
     };
   }
+
+  public static <V> BinaryOperator<V> throwingReduceWrapper(SerializableBiFunction<V, V, V> throwingReduceFunction) {

Review comment:
       nit: can we name this as throwableReduceWrapper ("ble" instead of "ing")

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java
##########
@@ -68,107 +67,107 @@ public ListingBasedRollbackHelper(HoodieTableMetaClient metaClient, HoodieWriteC
   /**
    * Performs all rollback actions that we have collected in parallel.
    */
-  public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
-    Map<String, HoodieRollbackStat> partitionPathRollbackStatsPairs = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, true);
-
-    Map<String, List<Pair<String, HoodieRollbackStat>>> collect = partitionPathRollbackStatsPairs.entrySet()
-        .stream()
-        .map(x -> Pair.of(x.getKey(), x.getValue())).collect(Collectors.groupingBy(Pair::getLeft));
-    return collect.values().stream()
-        .map(pairs -> pairs.stream().map(Pair::getRight).reduce(RollbackUtils::mergeRollbackStat).orElse(null))
-        .filter(Objects::nonNull)
-        .collect(Collectors.toList());
+  public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback,
+                                                  List<ListingBasedRollbackRequest> rollbackRequests) {
+    int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
+    context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions");
+    return context.mapToPairAndReduceByKey(rollbackRequests,
+        rollbackRequest -> maybeDeleteAndCollectStats(rollbackRequest, instantToRollback, true),
+        RollbackUtils::mergeRollbackStat,
+        parallelism);
   }
 
   /**
    * Collect all file info that needs to be rollbacked.
    */
-  public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
-    Map<String, HoodieRollbackStat> partitionPathRollbackStatsPairs = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, false);
-    return new ArrayList<>(partitionPathRollbackStatsPairs.values());
+  public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback,
+                                                       List<ListingBasedRollbackRequest> rollbackRequests) {
+    int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
+    context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
+    return context.mapToPairAndReduceByKey(rollbackRequests,
+        rollbackRequest -> maybeDeleteAndCollectStats(rollbackRequest, instantToRollback, false),
+        RollbackUtils::mergeRollbackStat,
+        parallelism);
   }
 
   /**
    * May be delete interested files and collect stats or collect stats only.
    *
-   * @param context           instance of {@link HoodieEngineContext} to use.
    * @param instantToRollback {@link HoodieInstant} of interest for which deletion or collect stats is requested.
-   * @param rollbackRequests  List of {@link ListingBasedRollbackRequest} to be operated on.
-   * @param doDelete          {@code true} if deletion has to be done. {@code false} if only stats are to be collected w/o performing any deletes.
+   * @param doDelete          {@code true} if deletion has to be done.
+   *                          {@code false} if only stats are to be collected w/o performing any deletes.
    * @return stats collected with or w/o actual deletions.
    */
-  Map<String, HoodieRollbackStat> maybeDeleteAndCollectStats(HoodieEngineContext context,
-                                                             HoodieInstant instantToRollback,
-                                                             List<ListingBasedRollbackRequest> rollbackRequests,
-                                                             boolean doDelete) {
-    return context.mapToPair(rollbackRequests, rollbackRequest -> {
-      switch (rollbackRequest.getType()) {
-        case DELETE_DATA_FILES_ONLY: {
-          final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseFiles(metaClient, config, instantToRollback.getTimestamp(),
-              rollbackRequest.getPartitionPath(), doDelete);
-          return new ImmutablePair<>(rollbackRequest.getPartitionPath(),
-              HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
-                  .withDeletedFileResults(filesToDeletedStatus).build());
-        }
-        case DELETE_DATA_AND_LOG_FILES: {
-          final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseAndLogFiles(metaClient, config, instantToRollback.getTimestamp(), rollbackRequest.getPartitionPath(), doDelete);
-          return new ImmutablePair<>(rollbackRequest.getPartitionPath(),
-              HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
-                  .withDeletedFileResults(filesToDeletedStatus).build());
-        }
-        case APPEND_ROLLBACK_BLOCK: {
-          String fileId = rollbackRequest.getFileId().get();
-          String latestBaseInstant = rollbackRequest.getLatestBaseInstant().get();
-
-          // collect all log files that is supposed to be deleted with this rollback
-          Map<FileStatus, Long> writtenLogFileSizeMap = FSUtils.getAllLogFiles(metaClient.getFs(),
-              FSUtils.getPartitionPath(config.getBasePath(), rollbackRequest.getPartitionPath()),
-              fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), latestBaseInstant)
-              .collect(Collectors.toMap(HoodieLogFile::getFileStatus, value -> value.getFileStatus().getLen()));
-
-          HoodieLogFormat.Writer writer = null;
+  private Pair<String, HoodieRollbackStat> maybeDeleteAndCollectStats(ListingBasedRollbackRequest rollbackRequest,
+                                                                      HoodieInstant instantToRollback,
+                                                                      boolean doDelete) throws IOException {
+    switch (rollbackRequest.getType()) {

Review comment:
       I assume you have not changed anything within this method. 




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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8798ec848605c413be660a58cb0b01aebaf0bf47 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239) 
   * 546ceddf55ba181711ad0073b464a03302650db6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * c070774d6c7d1c876eb3c529ba6cacf0bb58a948 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 546ceddf55ba181711ad0073b464a03302650db6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] yihua commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920330492


   @hudi-bot run azure


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8798ec848605c413be660a58cb0b01aebaf0bf47 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239) 
   * 546ceddf55ba181711ad0073b464a03302650db6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] yanghua commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yanghua commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919961961


   @yihua Can you make the CI successful?


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



[GitHub] [hudi] nsivabalan commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920393451


   @hudi-bot azure run


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920352971",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * c070774d6c7d1c876eb3c529ba6cacf0bb58a948 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245) 
   * af7a424d65677f10dba51836220751afb3fd6f4a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "920352971",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * c070774d6c7d1c876eb3c529ba6cacf0bb58a948 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * c070774d6c7d1c876eb3c529ba6cacf0bb58a948 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920352971",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "37b53485eaef00800e1ec5ce833c54b618cb3054",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2246",
       "triggerID" : "37b53485eaef00800e1ec5ce833c54b618cb3054",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c070774d6c7d1c876eb3c529ba6cacf0bb58a948 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245) 
   * af7a424d65677f10dba51836220751afb3fd6f4a UNKNOWN
   * 37b53485eaef00800e1ec5ce833c54b618cb3054 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2246) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * bc99d3169605da957d2b5c0807db351ffda9f845 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] nsivabalan commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920352971


   @hudi-bot run azure


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



[GitHub] [hudi] yihua commented on a change in pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#discussion_r711206432



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java
##########
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi.table.action.rollback;
+
+import org.apache.hudi.common.HoodieRollbackStat;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.log.HoodieLogFormat;
+import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
+import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieRollbackException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Performs Rollback of Hoodie Tables.
+ */
+public class ListingBasedRollbackHelper implements Serializable {
+  private static final Logger LOG = LogManager.getLogger(ListingBasedRollbackHelper.class);
+
+  private final HoodieTableMetaClient metaClient;
+  private final HoodieWriteConfig config;
+
+  public ListingBasedRollbackHelper(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
+    this.metaClient = metaClient;
+    this.config = config;
+  }
+
+  /**
+   * Performs all rollback actions that we have collected in parallel.
+   */
+  public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback,
+                                                  List<ListingBasedRollbackRequest> rollbackRequests) {
+    int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
+    context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions");
+    return context.mapToPairAndReduceByKey(rollbackRequests,
+        rollbackRequest -> maybeDeleteAndCollectStats(rollbackRequest, instantToRollback, true),
+        RollbackUtils::mergeRollbackStat,
+        parallelism);
+  }
+
+  /**
+   * Collect all file info that needs to be rollbacked.
+   */
+  public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback,
+                                                       List<ListingBasedRollbackRequest> rollbackRequests) {
+    int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
+    context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
+    return context.mapToPairAndReduceByKey(rollbackRequests,
+        rollbackRequest -> maybeDeleteAndCollectStats(rollbackRequest, instantToRollback, false),
+        RollbackUtils::mergeRollbackStat,
+        parallelism);

Review comment:
       @vinothchandar in the existing Flink/Spark client (like below), we collect the rollback stats without merging them based on the partition path.  @nsivabalan suggested that there should be a merge logic here as well.  Just to make sure this change is okay since it's changing the existing logic.
   ```
   int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
       context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
       JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, false);
       return partitionPathRollbackStatsPairRDD.map(Tuple2::_2).collect();
   ```




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



[GitHub] [hudi] yihua removed a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua removed a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920329753


   @hudi-bot run azure


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



[GitHub] [hudi] yihua commented on a change in pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#discussion_r709500966



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/function/FunctionWrapper.java
##########
@@ -70,4 +71,14 @@
       }
     };
   }
+
+  public static <V> BinaryOperator<V> throwingReduceWrapper(SerializableBiFunction<V, V, V> throwingReduceFunction) {

Review comment:
       Synced offline.  We'll still use `throwing` to be consistent with other methods in the class.




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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8798ec848605c413be660a58cb0b01aebaf0bf47 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] nsivabalan merged pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
nsivabalan merged pull request #3664:
URL: https://github.com/apache/hudi/pull/3664


   


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * bc99d3169605da957d2b5c0807db351ffda9f845 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920352971",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c070774d6c7d1c876eb3c529ba6cacf0bb58a948 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245) 
   * af7a424d65677f10dba51836220751afb3fd6f4a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8798ec848605c413be660a58cb0b01aebaf0bf47 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] yihua commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920201009


   @hudi-bot run azure


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8798ec848605c413be660a58cb0b01aebaf0bf47 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] vinothchandar commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920035893


   cc @nsivabalan healthy competition on who's going to be rebasing :) . 
   
   can you review this please? since you are actively working on this. My take would be land this and redo yours on top, that way, we get a "deeper" review as you rethink/rework 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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 8798ec848605c413be660a58cb0b01aebaf0bf47 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920352971",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "37b53485eaef00800e1ec5ce833c54b618cb3054",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "37b53485eaef00800e1ec5ce833c54b618cb3054",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c070774d6c7d1c876eb3c529ba6cacf0bb58a948 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245) 
   * af7a424d65677f10dba51836220751afb3fd6f4a UNKNOWN
   * 37b53485eaef00800e1ec5ce833c54b618cb3054 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] yihua commented on a change in pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#discussion_r709498852



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java
##########
@@ -68,107 +67,107 @@ public ListingBasedRollbackHelper(HoodieTableMetaClient metaClient, HoodieWriteC
   /**
    * Performs all rollback actions that we have collected in parallel.
    */
-  public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
-    Map<String, HoodieRollbackStat> partitionPathRollbackStatsPairs = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, true);
-
-    Map<String, List<Pair<String, HoodieRollbackStat>>> collect = partitionPathRollbackStatsPairs.entrySet()
-        .stream()
-        .map(x -> Pair.of(x.getKey(), x.getValue())).collect(Collectors.groupingBy(Pair::getLeft));
-    return collect.values().stream()
-        .map(pairs -> pairs.stream().map(Pair::getRight).reduce(RollbackUtils::mergeRollbackStat).orElse(null))
-        .filter(Objects::nonNull)
-        .collect(Collectors.toList());
+  public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback,
+                                                  List<ListingBasedRollbackRequest> rollbackRequests) {
+    int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
+    context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions");
+    return context.mapToPairAndReduceByKey(rollbackRequests,
+        rollbackRequest -> maybeDeleteAndCollectStats(rollbackRequest, instantToRollback, true),
+        RollbackUtils::mergeRollbackStat,
+        parallelism);
   }
 
   /**
    * Collect all file info that needs to be rollbacked.
    */
-  public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
-    Map<String, HoodieRollbackStat> partitionPathRollbackStatsPairs = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, false);
-    return new ArrayList<>(partitionPathRollbackStatsPairs.values());
+  public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback,
+                                                       List<ListingBasedRollbackRequest> rollbackRequests) {
+    int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
+    context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
+    return context.mapToPairAndReduceByKey(rollbackRequests,
+        rollbackRequest -> maybeDeleteAndCollectStats(rollbackRequest, instantToRollback, false),
+        RollbackUtils::mergeRollbackStat,
+        parallelism);
   }
 
   /**
    * May be delete interested files and collect stats or collect stats only.
    *
-   * @param context           instance of {@link HoodieEngineContext} to use.
    * @param instantToRollback {@link HoodieInstant} of interest for which deletion or collect stats is requested.
-   * @param rollbackRequests  List of {@link ListingBasedRollbackRequest} to be operated on.
-   * @param doDelete          {@code true} if deletion has to be done. {@code false} if only stats are to be collected w/o performing any deletes.
+   * @param doDelete          {@code true} if deletion has to be done.
+   *                          {@code false} if only stats are to be collected w/o performing any deletes.
    * @return stats collected with or w/o actual deletions.
    */
-  Map<String, HoodieRollbackStat> maybeDeleteAndCollectStats(HoodieEngineContext context,
-                                                             HoodieInstant instantToRollback,
-                                                             List<ListingBasedRollbackRequest> rollbackRequests,
-                                                             boolean doDelete) {
-    return context.mapToPair(rollbackRequests, rollbackRequest -> {
-      switch (rollbackRequest.getType()) {
-        case DELETE_DATA_FILES_ONLY: {
-          final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseFiles(metaClient, config, instantToRollback.getTimestamp(),
-              rollbackRequest.getPartitionPath(), doDelete);
-          return new ImmutablePair<>(rollbackRequest.getPartitionPath(),
-              HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
-                  .withDeletedFileResults(filesToDeletedStatus).build());
-        }
-        case DELETE_DATA_AND_LOG_FILES: {
-          final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseAndLogFiles(metaClient, config, instantToRollback.getTimestamp(), rollbackRequest.getPartitionPath(), doDelete);
-          return new ImmutablePair<>(rollbackRequest.getPartitionPath(),
-              HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
-                  .withDeletedFileResults(filesToDeletedStatus).build());
-        }
-        case APPEND_ROLLBACK_BLOCK: {
-          String fileId = rollbackRequest.getFileId().get();
-          String latestBaseInstant = rollbackRequest.getLatestBaseInstant().get();
-
-          // collect all log files that is supposed to be deleted with this rollback
-          Map<FileStatus, Long> writtenLogFileSizeMap = FSUtils.getAllLogFiles(metaClient.getFs(),
-              FSUtils.getPartitionPath(config.getBasePath(), rollbackRequest.getPartitionPath()),
-              fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), latestBaseInstant)
-              .collect(Collectors.toMap(HoodieLogFile::getFileStatus, value -> value.getFileStatus().getLen()));
-
-          HoodieLogFormat.Writer writer = null;
+  private Pair<String, HoodieRollbackStat> maybeDeleteAndCollectStats(ListingBasedRollbackRequest rollbackRequest,
+                                                                      HoodieInstant instantToRollback,
+                                                                      boolean doDelete) throws IOException {
+    switch (rollbackRequest.getType()) {

Review comment:
       I only changed the return type so that it can be reused across different engines.




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



[GitHub] [hudi] yihua commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920187126


   @hudi-bot run azure


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 8798ec848605c413be660a58cb0b01aebaf0bf47 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920352971",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * c070774d6c7d1c876eb3c529ba6cacf0bb58a948 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] yihua commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920329753


   @hudi-bot run azure


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



[GitHub] [hudi] yihua commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920313306


   > @yihua Can you make the CI successful?
   
   Looks like there are two flaky tests around rollback.  Another Azure CI run is triggered.


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



[GitHub] [hudi] hudi-bot edited a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-919758805


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2231",
       "triggerID" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920187126",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2240",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8798ec848605c413be660a58cb0b01aebaf0bf47",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2239",
       "triggerID" : "920201009",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "546ceddf55ba181711ad0073b464a03302650db6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2241",
       "triggerID" : "546ceddf55ba181711ad0073b464a03302650db6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2244",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bc99d3169605da957d2b5c0807db351ffda9f845",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2243",
       "triggerID" : "920329753",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c070774d6c7d1c876eb3c529ba6cacf0bb58a948",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2245",
       "triggerID" : "920352971",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af7a424d65677f10dba51836220751afb3fd6f4a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "920328060",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "920330492",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "37b53485eaef00800e1ec5ce833c54b618cb3054",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2246",
       "triggerID" : "37b53485eaef00800e1ec5ce833c54b618cb3054",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af7a424d65677f10dba51836220751afb3fd6f4a UNKNOWN
   * 37b53485eaef00800e1ec5ce833c54b618cb3054 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2246) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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



[GitHub] [hudi] yihua removed a comment on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua removed a comment on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920328060






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



[GitHub] [hudi] yihua commented on pull request #3664: [HUDI-2433] Refactor rollback actions in hudi-client module

Posted by GitBox <gi...@apache.org>.
yihua commented on pull request #3664:
URL: https://github.com/apache/hudi/pull/3664#issuecomment-920328060


   @hudi-bot run azure


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