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 2022/07/17 06:58:07 UTC

[GitHub] [hudi] chenshzh opened a new pull request, #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

chenshzh opened a new pull request, #6121:
URL: https://github.com/apache/hudi/pull/6121

   Currently CompactionCommitSink commit or rollback logics doesn't take the writestatus error under consideration (only consider null writestatus), which actually will cause data loss when compacting the delta commit log files into the new versioned data files.
   eg. org.apache.hudi.io.HoodieMergeHandle#writeRecord will lead to data loss from log files due to Exceptions.
   ```java
     protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord, boolean isDelete) {
       Option recordMetadata = hoodieRecord.getData().getMetadata();
       if (!partitionPath.equals(hoodieRecord.getPartitionPath())) {
         HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: "
             + hoodieRecord.getPartitionPath() + " but trying to insert into partition: " + partitionPath);
         writeStatus.markFailure(hoodieRecord, failureEx, recordMetadata);
         return false;
       }
       try {
         if (indexedRecord.isPresent() && !isDelete) {
           writeToFile(hoodieRecord.getKey(), (GenericRecord) indexedRecord.get(), preserveMetadata && useWriterSchemaForCompaction);
           recordsWritten++;
         } else {
           recordsDeleted++;
         }
         writeStatus.markSuccess(hoodieRecord, recordMetadata);
         // deflate record payload after recording success. This will help users access payload as a
         // part of marking
         // record successful.
         hoodieRecord.deflate();
         return true;
       } catch (Exception e) {
         LOG.error("Error writing record  " + hoodieRecord, e);
         writeStatus.markFailure(hoodieRecord, e, recordMetadata);
       }
       return false;
     }
   ```
   And it's known that StreamWriteOperatorCoordinator has related commit or rollback handle process. 
   
   So this pr will:
   
   a)  Also add writestatus error as rollback reason for CompactionCommitSink compaction rollback to avoid data loss
   
   b) Unify the handle procedure for write commit policy with its implementions, as described in org.apache.hudi.commit.policy.WriteCommitPolicy, which is consolidated with that of StreamWriteOperatorCoordinator.
   
   c) All control whether data quality or ingestion stability should be in high priority through FlinkOptions#IGNORE_FAILED.
   And, we suggest that FlinkOptions#IGNORE_FAILED be in true by default to avoid data loss.
   
   d) Optimize and fix some tiny bugs for log traces when commiting on error or rolling back.
   
   
   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## 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] danny0405 commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by GitBox <gi...@apache.org>.
danny0405 commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1207104389

   > @danny0405 would you pls help see this pr for avoiding data loss during compaction due to some write errors such as non-thrown exceptions ?
   
   What kind of exceptions in your production caused the write handle to encounter exceptions then ? And what would you want to do when you encounter that ?


-- 
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] chenshzh commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by GitBox <gi...@apache.org>.
chenshzh commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1207662818

   > > @danny0405 would you pls help see this pr for avoiding data loss during compaction due to some write errors such as non-thrown exceptions ?
   > 
   > What kind of exceptions in your production caused the write handle to encounter exceptions then ? 
   
   @danny0405 Just as described in the opening part,we have encountered data loss during compaction. And after analyzing the code below and validating it online, we found that some Exceptions are catched during HoodieMergeHandle#writeRecord, then the writing handle process is interrupted, and only marked as failures in WriteStatus, therefore causes data loss from delta logs to data files. The exceptions might be like IOException from HoodieFileWriter#writeToFile, HoodieUpsertException, and so on.
   
   ```java
     protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord, boolean isDelete) {
       Option recordMetadata = hoodieRecord.getData().getMetadata();
       if (!partitionPath.equals(hoodieRecord.getPartitionPath())) {
         HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: "
             + hoodieRecord.getPartitionPath() + " but trying to insert into partition: " + partitionPath);
         writeStatus.markFailure(hoodieRecord, failureEx, recordMetadata);
         return false;
       }
       try {
         if (indexedRecord.isPresent() && !isDelete) {
           writeToFile(hoodieRecord.getKey(), (GenericRecord) indexedRecord.get(), preserveMetadata && useWriterSchemaForCompaction);
           recordsWritten++;
         } else {
           recordsDeleted++;
         }
         writeStatus.markSuccess(hoodieRecord, recordMetadata);
         // deflate record payload after recording success. This will help users access payload as a
         // part of marking
         // record successful.
         hoodieRecord.deflate();
         return true;
       } catch (Exception e) {
         LOG.error("Error writing record  " + hoodieRecord, e);
         writeStatus.markFailure(hoodieRecord, e, recordMetadata);
       }
       return false;
     }
   ```
   
   > And what would you want to do when you encounter that ?
   
   What we want to do has been detailedly described in the opening part as a) -> d). 
   
   Compaction completed commit will actually be regarded as the snapshot and affect the latest file slices, so it's better to take the writestatus errors under consideration when deciding to commit or rollback for compaction (Just as the StreamWriteOperatorCoordinator done for deltacommit showed below). 
   
   In a word, compaction commit with errors should be rolled back, warned and retried for next schedule if we think data quality is more important than job stablility.
   
   org.apache.hudi.sink.StreamWriteOperatorCoordinator#doCommit
   ```java
     private void doCommit(String instant, List<WriteStatus> writeResults) {
       // commit or rollback
       long totalErrorRecords = writeResults.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L);
       long totalRecords = writeResults.stream().map(WriteStatus::getTotalRecords).reduce(Long::sum).orElse(0L);
       boolean hasErrors = totalErrorRecords > 0;
   
       if (!hasErrors || this.conf.getBoolean(FlinkOptions.IGNORE_FAILED)) {
         HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
         if (hasErrors) {
           LOG.warn("Some records failed to merge but forcing commit since commitOnErrors set to true. Errors/Total="
               + totalErrorRecords + "/" + totalRecords);
         }
   
         final Map<String, List<String>> partitionToReplacedFileIds = tableState.isOverwrite
             ? writeClient.getPartitionToReplacedFileIds(tableState.operationType, writeResults)
             : Collections.emptyMap();
         boolean success = writeClient.commit(instant, writeResults, Option.of(checkpointCommitMetadata),
             tableState.commitAction, partitionToReplacedFileIds);
         if (success) {
           reset();
           this.ckpMetadata.commitInstant(instant);
           LOG.info("Commit instant [{}] success!", instant);
         } else {
           throw new HoodieException(String.format("Commit instant [%s] failed!", instant));
         }
       } else {
         LOG.error("Error when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords);
         LOG.error("The first 100 error messages");
         writeResults.stream().filter(WriteStatus::hasErrors).limit(100).forEach(ws -> {
           LOG.error("Global error for partition path {} and fileID {}: {}",
               ws.getGlobalError(), ws.getPartitionPath(), ws.getFileId());
           if (ws.getErrors().size() > 0) {
             ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" + key + " and value " + value));
           }
         });
         // Rolls back instant
         writeClient.rollback(instant);
         throw new HoodieException(String.format("Commit instant [%s] failed and rolled back !", instant));
       }
     }
   ```
   
   So we suggest to use FlinkOptions#IGNORE_FAILED for both delta_commit's and compaction commit's  commitOrRollback policy. Otherwise, we will still unexpectedly encounter data loss even though FlinkOptions#IGNORE_FAILED set false.


-- 
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] chenshzh commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by GitBox <gi...@apache.org>.
chenshzh commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1206340858

   @danny0405 would you pls help see this pr for avoiding data loss during compaction due to some write errors such as non-thrown exceptions ?


-- 
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] chenshzh commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by "chenshzh (via GitHub)" <gi...@apache.org>.
chenshzh commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1425538561

   > > @danny0405 @yuzhaojing could you help review this pr? It has been here for a long time.
   > > Its purpose is clear that we should include errors/exceptions wrapped in compaction write status into rollback policies.
   > > Otherwise, it will still be committed even if this compaction has met with some data loss exceptions.
   > 
   > I see there are so many code refactoring in the patch, can we have a quick fix in `CompactionCommitSink` and left the refactoring to maybe another PR?
   
   @danny0405 Already removed the refactoring part, and also added it to `ClusteringCommitSink` for the same reason.


-- 
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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4669ba516e07d648a3914400b8d73e8068f11748 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] danny0405 commented on pull request #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1427561661

   Thanks for the contribution, reviewed and attach a patch here: [4406.zip](https://github.com/apache/hudi/files/10720334/4406.zip)
   
   You can apply the patch with cmd:
   ```shell
   git apply xxx.patch
   ```


-- 
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 #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1426104617

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * c52a60118c2e7fba170ea1cea0c4105ff83c52f9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] chenshzh commented on a diff in pull request #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "chenshzh (via GitHub)" <gi...@apache.org>.
chenshzh commented on code in PR #6121:
URL: https://github.com/apache/hudi/pull/6121#discussion_r1103982916


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java:
##########
@@ -119,7 +119,16 @@ private void commitIfNecessary(String instant, List<ClusteringCommitEvent> event
       return;
     }
 
-    if (events.stream().anyMatch(ClusteringCommitEvent::isFailed)) {
+    // here we should take the write errors under consideration
+    // as some write errors might cause data loss when clustering
+    List<WriteStatus> statuses = events.stream()

Review Comment:
   Agree that `isFailed` indicates the execution failure always to be rollbacked. 
   
   So in the updated we will judge whether to rollback write status errors when the config `FlinkOptions.IGNORE_FAILED` false.
   
   Pls take a review.



-- 
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 #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1428247024

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126",
       "triggerID" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15138",
       "triggerID" : "c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15138) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fa26698982216d7eeb904b74e79302ae61a9f96c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 93c512171b45761868350d8347b78503428e675f Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419) 
   * 4669ba516e07d648a3914400b8d73e8068f11748 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1427335554

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126",
       "triggerID" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * c52a60118c2e7fba170ea1cea0c4105ff83c52f9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089) 
   * 5dc463fcade7c5a495cca1437fca8230b01d0229 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 802a86e59cabbf397e24b37031c6a78ebd9fabb8 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999) 
   * 93c512171b45761868350d8347b78503428e675f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fa26698982216d7eeb904b74e79302ae61a9f96c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998) 
   * 802a86e59cabbf397e24b37031c6a78ebd9fabb8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] danny0405 commented on pull request #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1429471911

   The failed test case `TestHoodieTableFactory#testTableTypeCheck` is unrelated with this patch and I test it locally to pass, would merge the PR soon ~


-- 
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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1425497116

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4669ba516e07d648a3914400b8d73e8068f11748 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421) 
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * c52a60118c2e7fba170ea1cea0c4105ff83c52f9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1421764507

   @danny0405 : is this still a valid patch. can you follow up. if not, close it out


-- 
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] chenshzh commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by GitBox <gi...@apache.org>.
chenshzh commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1326989265

   @danny0405 could we review this pr? It has been here for a long time. 
   
   Its purpose is clear that we should include errors/exceptions wrapped in compaction write status into rollback policies.


-- 
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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1425470017

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4669ba516e07d648a3914400b8d73e8068f11748 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421) 
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1427466320

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126",
       "triggerID" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * 5dc463fcade7c5a495cca1437fca8230b01d0229 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 802a86e59cabbf397e24b37031c6a78ebd9fabb8 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999) 
   * 93c512171b45761868350d8347b78503428e675f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   @danny0405 @yuzhaojing : gentle reminder about this PR as well. marked as critical for now. So, would appreciate if we can review it. no activity for quite sometime now. 
   


-- 
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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 802a86e59cabbf397e24b37031c6a78ebd9fabb8 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] danny0405 commented on a diff in pull request #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #6121:
URL: https://github.com/apache/hudi/pull/6121#discussion_r1103540589


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java:
##########
@@ -119,7 +119,16 @@ private void commitIfNecessary(String instant, List<ClusteringCommitEvent> event
       return;
     }
 
-    if (events.stream().anyMatch(ClusteringCommitEvent::isFailed)) {
+    // here we should take the write errors under consideration
+    // as some write errors might cause data loss when clustering
+    List<WriteStatus> statuses = events.stream()

Review Comment:
   Things are a little different for when `events.stream().anyMatch(ClusteringCommitEvent::isFailed)`, the `isFailed` flag always indicates some errors that are related with the service execution, not the data record quality, we should always try to rollback in this case, not just throws.



-- 
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 #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1427328065

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * c52a60118c2e7fba170ea1cea0c4105ff83c52f9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089) 
   * 5dc463fcade7c5a495cca1437fca8230b01d0229 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fa26698982216d7eeb904b74e79302ae61a9f96c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998) 
   * 802a86e59cabbf397e24b37031c6a78ebd9fabb8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] danny0405 commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1421938531

   > @danny0405 @yuzhaojing could you help review this pr? It has been here for a long time.
   > 
   > Its purpose is clear that we should include errors/exceptions wrapped in compaction write status into rollback policies.
   > 
   > Otherwise, it will still be committed even if this compaction has met with some data loss exceptions.
   
   I see there are so many code refactoring in the patch, can we have a quick fix in `CompactionCommitSink` and left the refactoring to maybe another PR?


-- 
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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1425481633

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4669ba516e07d648a3914400b8d73e8068f11748 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421) 
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * c52a60118c2e7fba170ea1cea0c4105ff83c52f9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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 #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1427779190

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126",
       "triggerID" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * 5dc463fcade7c5a495cca1437fca8230b01d0229 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126) 
   * c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] danny0405 merged pull request #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 merged PR #6121:
URL: https://github.com/apache/hudi/pull/6121


-- 
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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 802a86e59cabbf397e24b37031c6a78ebd9fabb8 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999) 
   * 93c512171b45761868350d8347b78503428e675f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] chenshzh commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by GitBox <gi...@apache.org>.
chenshzh commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1220699843

   @danny0405  hi, are there any further improvements or supplements needed for this pr? Or if you will, maybe we could have a discussion offline about it.


-- 
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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fa26698982216d7eeb904b74e79302ae61a9f96c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998) 
   * 802a86e59cabbf397e24b37031c6a78ebd9fabb8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] chenshzh commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by GitBox <gi...@apache.org>.
chenshzh commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1303549222

   @nsivabalan it will be appreciated much if you can help me push on this PR.
   In fact it has been made me puzzled bacause quite several of my PRs stayed there for a long time. Just will been answered sometimes or not and then ignored. 


-- 
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 #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 93c512171b45761868350d8347b78503428e675f Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419) 
   * 4669ba516e07d648a3914400b8d73e8068f11748 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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] chenshzh commented on pull request #6121: [HUDI-4406] Support Flink compaction commit write error resolvement to avoid data loss

Posted by GitBox <gi...@apache.org>.
chenshzh commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1286686301

   @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] chenshzh commented on pull request #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "chenshzh (via GitHub)" <gi...@apache.org>.
chenshzh commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1427748025

   > Thanks for the contribution, reviewed and attach a patch here: [4406.zip](https://github.com/apache/hudi/files/10720334/4406.zip)
   > 
   > You can apply the patch with cmd:
   > 
   > ```shell
   > git apply xxx.patch
   > ```
   
   applied and commit updated, pls review it once more


-- 
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 #6121: [HUDI-4406] Support Flink compaction/clustering write error resolvement to avoid data loss

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #6121:
URL: https://github.com/apache/hudi/pull/6121#issuecomment-1427805620

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9998",
       "triggerID" : "fa26698982216d7eeb904b74e79302ae61a9f96c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9999",
       "triggerID" : "802a86e59cabbf397e24b37031c6a78ebd9fabb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12416",
       "triggerID" : "93c512171b45761868350d8347b78503428e675f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93c512171b45761868350d8347b78503428e675f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12419",
       "triggerID" : "1286686301",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12421",
       "triggerID" : "4669ba516e07d648a3914400b8d73e8068f11748",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52b6f55e196007f993b0506d899c48bb80b36546",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15089",
       "triggerID" : "c52a60118c2e7fba170ea1cea0c4105ff83c52f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126",
       "triggerID" : "5dc463fcade7c5a495cca1437fca8230b01d0229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15138",
       "triggerID" : "c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 52b6f55e196007f993b0506d899c48bb80b36546 UNKNOWN
   * 5dc463fcade7c5a495cca1437fca8230b01d0229 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15126) 
   * c846b9e08ee1c9f1ee39d7076a3a24cebcb162f8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15138) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@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