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/02/17 13:50:26 UTC

[GitHub] [hudi] nsivabalan opened a new pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

nsivabalan opened a new pull request #4840:
URL: https://github.com/apache/hudi/pull/4840


   ## What is the purpose of the pull request
   
   We may find some data which should be rollbacked in hudi table.
   
   Root cause:
   
   Let's first recall how rollback plan generated about log blocks for deltaCommit. Hudi takes two cases into consideration.
   
   For some log file with no base file, they are comprised by records which are all 'insert record'. Delete them directly. Here we assume all inserted record should be covered by this way.
   For those fileID which are updated according to inflight commit meta of instant we want to rollback, we append command block to these log file to rollback.  Here all updated record are handled.
   However, the first condition is not always true. For indexes which can index log file, they could insert record to some existing log file. In current process, inflight hoodieCommitMeta was generated before they are assigned to specific filegroup. 
   
   ## Brief change log
   
   1. make upsert partitioner generate an execution workload stats which including all fileGroup will be written into comparing with workload generated by input data. This will cover the case that insert data is written into some log files which is recognized as small file when using Hbase Index.
   2.  In such case, we cannot guarantee that all log files which contains only insert data could be deleted in rollback. They may be rollback using command block. So handle this case in compactor.
   
   ## Verify this pull request
     - Added integration tests for end-to-end: spark client
     
   ## 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] hudi-bot commented on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098) 
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee 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] nsivabalan commented on a change in pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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



##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
##########
@@ -139,6 +157,137 @@ public void testMergeOnReadRollbackActionExecutor(boolean isUsingMarkers) throws
     assertFalse(WriteMarkersFactory.get(cfg.getMarkersType(), table, "002").doesMarkerDirExist());
   }
 
+  @Test
+  public void testRollbackForCanIndexLogFile() throws IOException {
+    cleanupResources();
+    setUpDFS();
+    //1. prepare data and assert data result
+    //just generate one partitions
+    dataGen = new HoodieTestDataGenerator(new String[]{DEFAULT_FIRST_PARTITION_PATH});
+    HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
+        .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
+        .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION)
+        .withWriteStatusClass(MetadataMergeWriteStatus.class)
+        .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
+        .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
+        .forTable("test-trip-table")
+        .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build())
+        .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
+            .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
+            .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()).withRollbackUsingMarkers(false).withAutoCommit(false).build();
+
+    //1. prepare data
+    HoodieTestDataGenerator.writePartitionMetadata(fs, new String[]{DEFAULT_FIRST_PARTITION_PATH}, basePath);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
+    /**

Review comment:
       sure.




-- 
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] codope commented on a change in pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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



##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
##########
@@ -169,4 +318,13 @@ public void testRollbackWhenFirstCommitFail() throws Exception {
       client.rollback("001");
     }
   }
+
+  private void setUpDFS() throws IOException {
+    initDFS();
+    initSparkContexts();
+    //just generate tow partitions

Review comment:
       nit: `two` spell-check.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
##########
@@ -156,19 +156,21 @@ private void initKeyGenIfNeeded(boolean populateMetaFields) {
       LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel());
     }
 
-    WorkloadProfile profile = null;
+    WorkloadProfile workloadProfile = null;
     if (isWorkloadProfileNeeded()) {
       context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile");
-      profile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType);
-      LOG.info("Workload profile :" + profile);
-      saveWorkloadProfileMetadataToInflight(profile, instantTime);
+      workloadProfile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType);
+      LOG.info("Input workload profile :" + workloadProfile);
     }
 
     // handle records update with clustering
     JavaRDD<HoodieRecord<T>> inputRecordsRDDWithClusteringUpdate = clusteringHandleUpdate(inputRecordsRDD);
 
     // partition using the insert partitioner
-    final Partitioner partitioner = getPartitioner(profile);
+    final Partitioner partitioner = getPartitioner(workloadProfile);
+    if (isWorkloadProfileNeeded()) {
+      saveWorkloadProfileMetadataToInflight(workloadProfile, instantTime);

Review comment:
       Just for my understanding, why do we want to save workload profile after handling clustering updates? Suppose `clusteringHandleUpdate` threw exception then the profile will never be saved in metadata right?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java
##########
@@ -235,6 +243,7 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context)
         LOG.info("Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
         partitionPathToInsertBucketInfos.put(partitionPath, insertBuckets);
       }
+      profile.getOutputPartitionPathStatMap().putIfAbsent(partitionPath, outputWorkloadStats);

Review comment:
       Why `putIfAbsent`? Should we not update the stat in this case?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
##########
@@ -156,19 +156,21 @@ private void initKeyGenIfNeeded(boolean populateMetaFields) {
       LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel());
     }
 
-    WorkloadProfile profile = null;
+    WorkloadProfile workloadProfile = null;
     if (isWorkloadProfileNeeded()) {
       context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile");
-      profile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType);
-      LOG.info("Workload profile :" + profile);
-      saveWorkloadProfileMetadataToInflight(profile, instantTime);
+      workloadProfile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType);
+      LOG.info("Input workload profile :" + workloadProfile);
     }
 
     // handle records update with clustering
     JavaRDD<HoodieRecord<T>> inputRecordsRDDWithClusteringUpdate = clusteringHandleUpdate(inputRecordsRDD);
 
     // partition using the insert partitioner
-    final Partitioner partitioner = getPartitioner(profile);
+    final Partitioner partitioner = getPartitioner(workloadProfile);
+    if (isWorkloadProfileNeeded()) {

Review comment:
       Alternatively, the condition can be changed to `if (workloadProfile != null)` to make it more readbale imo. But, i'll leave it upto you. 

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
##########
@@ -139,6 +157,137 @@ public void testMergeOnReadRollbackActionExecutor(boolean isUsingMarkers) throws
     assertFalse(WriteMarkersFactory.get(cfg.getMarkersType(), table, "002").doesMarkerDirExist());
   }
 
+  @Test
+  public void testRollbackForCanIndexLogFile() throws IOException {
+    cleanupResources();
+    setUpDFS();
+    //1. prepare data and assert data result
+    //just generate one partitions
+    dataGen = new HoodieTestDataGenerator(new String[]{DEFAULT_FIRST_PARTITION_PATH});
+    HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
+        .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
+        .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION)
+        .withWriteStatusClass(MetadataMergeWriteStatus.class)
+        .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
+        .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
+        .forTable("test-trip-table")
+        .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build())
+        .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
+            .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
+            .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()).withRollbackUsingMarkers(false).withAutoCommit(false).build();
+
+    //1. prepare data
+    HoodieTestDataGenerator.writePartitionMetadata(fs, new String[]{DEFAULT_FIRST_PARTITION_PATH}, basePath);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
+    /**

Review comment:
       maybe we can remove this multi-line comment here and below?




-- 
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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174) 
   * 43859892f433775b00097e573b3c4e9971f8f3fb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 43859892f433775b00097e573b3c4e9971f8f3fb Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239) 
   
   <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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6366",
       "triggerID" : "1053615649",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6366) 
   
   <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 merged pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   


-- 
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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6366",
       "triggerID" : "1053615649",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6366) 
   
   <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] codope commented on a change in pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
##########
@@ -182,14 +182,30 @@ public abstract void preCompact(
         .withOperationField(config.allowOperationMetadataField())
         .withPartition(operation.getPartitionPath())
         .build();
-    if (!scanner.iterator().hasNext()) {
-      scanner.close();
-      return new ArrayList<>();
-    }
 
     Option<HoodieBaseFile> oldDataFileOpt =
         operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath());
 
+    // Considering following scenario: if all log blocks in this fileSlice is rollback, it returns an empty scanner.
+    // But in this case, we need to give it a base file. Otherwise, it will lose base file in following fileSlice.
+    if (!scanner.iterator().hasNext()) {
+      if (!oldDataFileOpt.isPresent()) {
+        scanner.close();
+        return new ArrayList<>();
+      } else {
+        // TODO: we may directly rename original parquet file if there is not evolution/devolution of schema
+        /*
+        TaskContextSupplier taskContextSupplier = hoodieCopyOnWriteTable.getTaskContextSupplier();
+        String newFileName = FSUtils.makeDataFileName(instantTime,
+            FSUtils.makeWriteToken(taskContextSupplier.getPartitionIdSupplier().get(), taskContextSupplier.getStageIdSupplier().get(), taskContextSupplier.getAttemptIdSupplier().get()),
+            operation.getFileId(), hoodieCopyOnWriteTable.getBaseFileExtension());
+        Path oldFilePath = new Path(oldDataFileOpt.get().getPath());
+        Path newFilePath = new Path(oldFilePath.getParent(), newFileName);
+        FileUtil.copy(fs,oldFilePath, fs, newFilePath, false, fs.getConf());
+        */
+      }

Review comment:
       Let's remove this else?




-- 
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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
##########
@@ -156,19 +156,21 @@ private void initKeyGenIfNeeded(boolean populateMetaFields) {
       LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel());
     }
 
-    WorkloadProfile profile = null;
+    WorkloadProfile workloadProfile = null;
     if (isWorkloadProfileNeeded()) {
       context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile");
-      profile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType);
-      LOG.info("Workload profile :" + profile);
-      saveWorkloadProfileMetadataToInflight(profile, instantTime);
+      workloadProfile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType);
+      LOG.info("Input workload profile :" + workloadProfile);
     }
 
     // handle records update with clustering
     JavaRDD<HoodieRecord<T>> inputRecordsRDDWithClusteringUpdate = clusteringHandleUpdate(inputRecordsRDD);
 
     // partition using the insert partitioner
-    final Partitioner partitioner = getPartitioner(profile);
+    final Partitioner partitioner = getPartitioner(workloadProfile);
+    if (isWorkloadProfileNeeded()) {
+      saveWorkloadProfileMetadataToInflight(workloadProfile, instantTime);

Review comment:
       makes sense. will move getPartitioner() and saveworkloadProfile before checking for clustering handle update. 




-- 
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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352) 
   
   <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 a change in pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
##########
@@ -182,14 +182,30 @@ public abstract void preCompact(
         .withOperationField(config.allowOperationMetadataField())
         .withPartition(operation.getPartitionPath())
         .build();
-    if (!scanner.iterator().hasNext()) {
-      scanner.close();
-      return new ArrayList<>();
-    }
 
     Option<HoodieBaseFile> oldDataFileOpt =
         operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath());
 
+    // Considering following scenario: if all log blocks in this fileSlice is rollback, it returns an empty scanner.
+    // But in this case, we need to give it a base file. Otherwise, it will lose base file in following fileSlice.
+    if (!scanner.iterator().hasNext()) {
+      if (!oldDataFileOpt.isPresent()) {
+        scanner.close();
+        return new ArrayList<>();
+      } else {
+        // TODO: we may directly rename original parquet file if there is not evolution/devolution of schema
+        /*
+        TaskContextSupplier taskContextSupplier = hoodieCopyOnWriteTable.getTaskContextSupplier();
+        String newFileName = FSUtils.makeDataFileName(instantTime,
+            FSUtils.makeWriteToken(taskContextSupplier.getPartitionIdSupplier().get(), taskContextSupplier.getStageIdSupplier().get(), taskContextSupplier.getAttemptIdSupplier().get()),
+            operation.getFileId(), hoodieCopyOnWriteTable.getBaseFileExtension());
+        Path oldFilePath = new Path(oldDataFileOpt.get().getPath());
+        Path newFilePath = new Path(oldFilePath.getParent(), newFileName);
+        FileUtil.copy(fs,oldFilePath, fs, newFilePath, false, fs.getConf());
+        */
+      }

Review comment:
       I intentionally left it there if we ever want to fix 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] hudi-bot removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da 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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da 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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098) 
   
   <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 a change in pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java
##########
@@ -235,6 +243,7 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context)
         LOG.info("Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
         partitionPathToInsertBucketInfos.put(partitionPath, insertBuckets);
       }
+      profile.getOutputPartitionPathStatMap().putIfAbsent(partitionPath, outputWorkloadStats);

Review comment:
       good catch. will fix 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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098) 
   
   <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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098) 
   
   <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 a change in pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
##########
@@ -156,19 +156,21 @@ private void initKeyGenIfNeeded(boolean populateMetaFields) {
       LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel());
     }
 
-    WorkloadProfile profile = null;
+    WorkloadProfile workloadProfile = null;
     if (isWorkloadProfileNeeded()) {
       context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile");
-      profile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType);
-      LOG.info("Workload profile :" + profile);
-      saveWorkloadProfileMetadataToInflight(profile, instantTime);
+      workloadProfile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType);
+      LOG.info("Input workload profile :" + workloadProfile);
     }
 
     // handle records update with clustering
     JavaRDD<HoodieRecord<T>> inputRecordsRDDWithClusteringUpdate = clusteringHandleUpdate(inputRecordsRDD);
 
     // partition using the insert partitioner
-    final Partitioner partitioner = getPartitioner(profile);
+    final Partitioner partitioner = getPartitioner(workloadProfile);
+    if (isWorkloadProfileNeeded()) {
+      saveWorkloadProfileMetadataToInflight(workloadProfile, instantTime);

Review comment:
       if exception is thrown, the write will fail right anyways. I am not sure I get your question




-- 
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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   @codope : can you review this 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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 43859892f433775b00097e573b3c4e9971f8f3fb Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239) 
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f 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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098) 
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee 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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174) 
   * 43859892f433775b00097e573b3c4e9971f8f3fb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239) 
   
   <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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6366",
       "triggerID" : "1053615649",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6366) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   @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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098) 
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174) 
   
   <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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174) 
   * 43859892f433775b00097e573b3c4e9971f8f3fb 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] nsivabalan commented on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   @codope : good to review again. fixed unnecessary updates/populating output workload stats if not required. 


-- 
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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 43859892f433775b00097e573b3c4e9971f8f3fb Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239) 
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 43859892f433775b00097e573b3c4e9971f8f3fb Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239) 
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6352) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174) 
   * 43859892f433775b00097e573b3c4e9971f8f3fb 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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 43859892f433775b00097e573b3c4e9971f8f3fb Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239) 
   * 260c29fa4d9e1d0e45e9ee771d51ed3f6b2f7c5f 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 removed a comment on pull request #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239",
       "triggerID" : "43859892f433775b00097e573b3c4e9971f8f3fb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 43859892f433775b00097e573b3c4e9971f8f3fb Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6239) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d014c22ead28e4594cd9ebc78ec72396b8033da Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098) 
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6098",
       "triggerID" : "4d014c22ead28e4594cd9ebc78ec72396b8033da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174",
       "triggerID" : "c2ebd32467cdcb907d22bea0e9a78f7e25902eee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c2ebd32467cdcb907d22bea0e9a78f7e25902eee Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6174) 
   
   <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 #4840: [HUDI-2917] rollback insert data appended to log file when using Hbase Index

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


   @guanziyue : feel free to review the 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