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

[GitHub] [hudi] beyond1920 opened a new pull request, #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   ### Change Logs
   
   Snapshot query result is wrong after apply insert overwrite to an existed table with simple bucket index.
   see detailed in [HUDI-5857](https://issues.apache.org/jira/browse/HUDI-5857).
   The root cause of the bug is the write handler reuses existed bucket file id for insert overwrite. Besides it use replace commit for insert overwrite operation and mark all the existed bucket file id as replaced. 
   So the snapshot query result is wrong.
   The pr aims to fix this bug by generating new file id for bucket if insert overwrite into bucket index table.
   
   ### Impact
   
   NA
   
   ### Risk level (write none, low medium or high below)
   
   NA
   
   ### Documentation Update
   
   NA
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fa9c6087f405cde05439d9e9c01d5abf20f712c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478) 
   * d4e9858122533fb916b589d633aeda44043a6a4d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505) 
   
   <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] KnightChess commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -66,7 +67,9 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
     return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
+        .map(c -> c.equals(HoodieLayoutConfig.SIMPLE_BUCKET_LAYOUT_PARTITIONER_CLASS_NAME)

Review Comment:
   @beyond1920 I read `consistentBucketIndex` implementation, found it must tag incomming records to allocation fgId, so #8073 will cause some quesion



-- 
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] beyond1920 commented on a diff in pull request #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -85,4 +102,19 @@ protected List<String> getAllExistingFileIds(String partitionPath) {
     // because new commit is not complete. it is safe to mark all existing file Ids as old files
     return table.getSliceView().getLatestFileSlices(partitionPath).map(FileSlice::getFileId).distinct().collect(Collectors.toList());
   }
+
+  @Override
+  protected Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr, Partitioner partitioner) {
+    SparkHoodiePartitioner upsertPartitioner = (SparkHoodiePartitioner) partitioner;
+    BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
+    BucketType btype = binfo.bucketType;
+    if (btype.equals(BucketType.INSERT)) {
+      return handleInsert(binfo.fileIdPrefix, recordItr);
+    } else if (btype.equals(BucketType.UPDATE)) {
+      throw new HoodieInsertOverwriteException(
+          "Insert overwrite should always use INSERT bucketType, please correct the logical of " + partitioner.getClass().getName());

Review Comment:
   This is a protected code to prevent hit this bug again when introduce new partitioner class in the future.



-- 
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] beyond1920 commented on a diff in pull request #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -65,9 +70,21 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
 
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
-    return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
-        .orElse(new SparkInsertOverwritePartitioner(profile, context, table, config));
+    // Fix HUDI-5857, insert overwrite should generate new file group id
+    if (config.getIndexType().equals(BUCKET)) {
+      switch (config.getBucketIndexEngineType()) {
+        case SIMPLE:
+          return new SparkInsertOverwriteSimpleBucketIndexPartitioner(profile, context, table, config);
+        case CONSISTENT_HASHING:
+          return new SparkInsertOverwriteConsistentBucketIndexPartitioner(profile, context, table, config);
+        default:
+          throw new HoodieNotSupportedException("Unknown bucket index engine type: " + config.getBucketIndexEngineType());

Review Comment:
   I move part of them which related to `ConsistentBucketIndex`  to `SparkInsertOverwritePartitioner`.
   And I left other part which related to `SimpleBucketIndex` in `SparkBucketIndexInsertOverwritePartitioner`.
   Because SimpleBucketIndex and ConsistentBucketIndex are different when creates new `BucketInfo`.
   



-- 
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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9a4747eedc49a941aec1608b143ebc3039e0e1b9 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476) 
   * 5fa9c6087f405cde05439d9e9c01d5abf20f712c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478) 
   
   <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] KnightChess commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -66,7 +67,9 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
     return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
+        .map(c -> c.equals(HoodieLayoutConfig.SIMPLE_BUCKET_LAYOUT_PARTITIONER_CLASS_NAME)

Review Comment:
   > No, consistentBucketIndex works correctly, it would generate different file ids.
   
   `consistentBucketIndex` can not work correctly, change the ut case



-- 
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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621) 
   * f5503443fd6080f0ec93a9a21e14a33f0fa432c7 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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 41f51ef1391ec5b31b85351f0f4d6cd3b857faa6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469) 
   * 9a4747eedc49a941aec1608b143ebc3039e0e1b9 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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 41f51ef1391ec5b31b85351f0f4d6cd3b857faa6 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469) 
   * 9a4747eedc49a941aec1608b143ebc3039e0e1b9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476) 
   
   <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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -65,9 +70,21 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
 
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
-    return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
-        .orElse(new SparkInsertOverwritePartitioner(profile, context, table, config));
+    // Fix HUDI-5857, insert overwrite should generate new file group id
+    if (config.getIndexType().equals(BUCKET)) {
+      switch (config.getBucketIndexEngineType()) {
+        case SIMPLE:
+          return new SparkInsertOverwriteSimpleBucketIndexPartitioner(profile, context, table, config);
+        case CONSISTENT_HASHING:
+          return new SparkInsertOverwriteConsistentBucketIndexPartitioner(profile, context, table, config);
+        default:
+          throw new HoodieNotSupportedException("Unknown bucket index engine type: " + config.getBucketIndexEngineType());

Review Comment:
   Can we inline all the different handling for `getBucketInfo` into `SparkInsertOverwritePartitioner` ? Let's make the code cleaner.



-- 
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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15633",
       "triggerID" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15638",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7d6319dd8880f2f5af3ba0ea1c38058499585337",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7d6319dd8880f2f5af3ba0ea1c38058499585337",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621) 
   * 7d6319dd8880f2f5af3ba0ea1c38058499585337 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 a diff in pull request #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieInsertOverwriteException.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.exception;
+
+/**
+ * <p>
+ * Exception thrown for any higher level errors when <code>HoodieClient</code> is doing an insert overwrite.
+ * </p>
+ */
+public class HoodieInsertOverwriteException extends HoodieException {
+
+  public HoodieInsertOverwriteException(String msg, Throwable e) {

Review Comment:
   No need to introduce checked exception is there are no extra msg you wanna prompt.



-- 
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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java:
##########
@@ -41,6 +42,25 @@ public SparkInsertOverwritePartitioner(WorkloadProfile profile, HoodieEngineCont
     super(profile, context, table, config);
   }
 
+  @Override
+  public BucketInfo getBucketInfo(int bucketNumber) {
+    BucketInfo bucketInfo = super.getBucketInfo(bucketNumber);

Review Comment:
   We decide the operation is `INSERT_OVERWRITE` or `INSERT_OVERWRITE_PARTITION` instead so that the `SparkBucketIndexInsertOverwritePartitioner` is not neede anymore.



-- 
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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15633",
       "triggerID" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15638",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * f5503443fd6080f0ec93a9a21e14a33f0fa432c7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15633) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15638) 
   
   <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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15633",
       "triggerID" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15638",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7d6319dd8880f2f5af3ba0ea1c38058499585337",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15640",
       "triggerID" : "7d6319dd8880f2f5af3ba0ea1c38058499585337",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621) 
   * 7d6319dd8880f2f5af3ba0ea1c38058499585337 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15640) 
   
   <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] KnightChess commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -1125,4 +1123,152 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
       }
     }
   }
+
+  test("Test Insert Overwrite Into Bucket Index COW Table") {
+    withRecordType()(withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  dt string
+           |) using hudi
+           |tblproperties (
+           |  type = 'cow',
+           |  primaryKey = 'id',
+           |  preCombineField = 'ts',
+           |  hoodie.index.type = 'BUCKET',
+           |  hoodie.bucket.index.num.buckets = '4'
+           |)
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}/$tableName'
+       """.stripMargin)
+
+      spark.sql(
+        s"""insert into $tableName  values
+           |(5, 'a', 35, 1000, '2021-01-05'),
+           |(1, 'a', 31, 1000, '2021-01-05'),
+           |(3, 'a', 33, 1000, '2021-01-05'),
+           |(4, 'b', 16, 1000, '2021-01-05'),
+           |(2, 'b', 18, 1000, '2021-01-05'),
+           |(6, 'b', 17, 1000, '2021-01-05'),
+           |(8, 'a', 21, 1000, '2021-01-05'),
+           |(9, 'a', 22, 1000, '2021-01-05'),
+           |(7, 'a', 23, 1000, '2021-01-05')
+           |""".stripMargin)
+
+      // Insert overwrite static partition
+      spark.sql(
+        s"""
+           | insert overwrite table $tableName partition(dt = '2021-01-05')
+           | select * from (select 13 , 'a2', 12, 1000) limit 10
+        """.stripMargin)
+      checkAnswer(s"select id, name, price, ts, dt from $tableName order by dt")(
+        Seq(13, "a2", 12.0, 1000, "2021-01-05")
+      )
+    })
+  }
+
+  test("Test Insert Overwrite Into Bucket Index MOR Table") {
+    withRecordType()(withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  dt string
+           |) using hudi
+           |tblproperties (
+           |  type = 'mor',
+           |  primaryKey = 'id',
+           |  preCombineField = 'ts',
+           |  hoodie.index.type = 'BUCKET',
+           |  hoodie.bucket.index.num.buckets = '4'
+           |)
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}/$tableName'
+       """.stripMargin)
+
+      spark.sql(
+        s"""insert into $tableName  values
+           |(5, 'a', 35, 1000, '2021-01-05'),
+           |(1, 'a', 31, 1000, '2021-01-05'),
+           |(3, 'a', 33, 1000, '2021-01-05'),
+           |(4, 'b', 16, 1000, '2021-01-05'),
+           |(2, 'b', 18, 1000, '2021-01-05'),
+           |(6, 'b', 17, 1000, '2021-01-05'),
+           |(8, 'a', 21, 1000, '2021-01-05'),
+           |(9, 'a', 22, 1000, '2021-01-05'),
+           |(7, 'a', 23, 1000, '2021-01-05')
+           |""".stripMargin)
+
+      // Insert overwrite static partition
+      spark.sql(
+        s"""
+           | insert overwrite table $tableName partition(dt = '2021-01-05')
+           | select * from (select 13 , 'a2', 12, 1000) limit 10
+        """.stripMargin)
+      checkAnswer(s"select id, name, price, ts, dt from $tableName order by dt")(
+        Seq(13, "a2", 12.0, 1000, "2021-01-05")
+      )
+    })
+  }
+
+  test("Test Insert Overwrite Into Consistent Bucket Index Table") {
+    withRecordType()(withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  dt string
+           |) using hudi
+           |tblproperties (
+           |  type = 'mor',
+           |  primaryKey = 'id',
+           |  preCombineField = 'ts',
+           |  hoodie.index.type = 'BUCKET',
+           |  hoodie.index.bucket.engine = "CONSISTENT_HASHING",
+           |  hoodie.bucket.index.num.buckets = '4'
+           |)
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}/$tableName'
+       """.stripMargin)
+
+      spark.sql(
+        s"""insert into $tableName  values
+           |(5, 'a', 35, 1000, '2021-01-05'),
+           |(1, 'a', 31, 1000, '2021-01-05'),
+           |(3, 'a', 33, 1000, '2021-01-05'),
+           |(4, 'b', 16, 1000, '2021-01-05'),
+           |(2, 'b', 18, 1000, '2021-01-05'),
+           |(6, 'b', 17, 1000, '2021-01-05'),
+           |(8, 'a', 21, 1000, '2021-01-05'),
+           |(9, 'a', 22, 1000, '2021-01-05'),
+           |(7, 'a', 23, 1000, '2021-01-05')
+           |""".stripMargin)
+
+      // Insert overwrite static partition
+      spark.sql(
+        s"""
+           | insert overwrite table $tableName partition(dt = '2021-01-05')

Review Comment:
   this will create a new parquet file with the same prefix against log file, but something diff in fgId suffix. just like the picture, create new parquet file will add `-0` after fgId(xxx-0-0_xxx), so it can be read if only insert overwrite onece, but if  insert overwrite again, will use the same fgId(xxx-0-0), result nothing.
   <img width="588" alt="image" src="https://user-images.githubusercontent.com/20125927/221895955-195a00a6-7eda-4b4a-ac3d-c6e5f10bfd27.png">
   



-- 
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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fa9c6087f405cde05439d9e9c01d5abf20f712c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478) 
   
   <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] beyond1920 commented on a diff in pull request #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java:
##########
@@ -41,6 +42,25 @@ public SparkInsertOverwritePartitioner(WorkloadProfile profile, HoodieEngineCont
     super(profile, context, table, config);
   }
 
+  @Override
+  public BucketInfo getBucketInfo(int bucketNumber) {
+    BucketInfo bucketInfo = super.getBucketInfo(bucketNumber);

Review Comment:
   Sorry, I don't understand what you mean here.
   How to remove `SparkBucketIndexInsertOverwritePartitioner `?  Most behaviors in consistent hash partitioner and bucket index partitioner are very different here.
   Could you please explain it more or show me the code?



-- 
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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexInsertOverwritePartitioner.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.commit;
+
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.WorkloadProfile;
+
+/**
+ * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition).
+ */
+public class SparkBucketIndexInsertOverwritePartitioner<T> extends SparkBucketIndexPartitioner<T> {
+
+  public SparkBucketIndexInsertOverwritePartitioner(
+      WorkloadProfile profile,
+      HoodieEngineContext context,
+      HoodieTable table,
+      HoodieWriteConfig config) {
+    super(profile, context, table, config);
+  }
+
+  @Override
+  public BucketInfo getBucketInfo(int bucketNumber) {
+    String partitionPath = partitionPaths.get(bucketNumber / numBuckets);

Review Comment:
   In `HoodieWriteConfig`, we can fetch the operation then decides whether it is INSERT_OVERWRITE, then the logic can be moved into `SparkBucketIndexPartitioner`.



-- 
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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9a4747eedc49a941aec1608b143ebc3039e0e1b9 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476) 
   * 5fa9c6087f405cde05439d9e9c01d5abf20f712c 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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fa9c6087f405cde05439d9e9c01d5abf20f712c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478) 
   * d4e9858122533fb916b589d633aeda44043a6a4d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505) 
   * d0099b9f29a6d7ff7582fd728f73fc216a4adb6a 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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0099b9f29a6d7ff7582fd728f73fc216a4adb6a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513) 
   
   <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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621) 
   
   <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] KnightChess commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -1125,4 +1123,152 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
       }
     }
   }
+
+  test("Test Insert Overwrite Into Bucket Index COW Table") {
+    withRecordType()(withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  dt string
+           |) using hudi
+           |tblproperties (
+           |  type = 'cow',
+           |  primaryKey = 'id',
+           |  preCombineField = 'ts',
+           |  hoodie.index.type = 'BUCKET',
+           |  hoodie.bucket.index.num.buckets = '4'
+           |)
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}/$tableName'
+       """.stripMargin)
+
+      spark.sql(
+        s"""insert into $tableName  values
+           |(5, 'a', 35, 1000, '2021-01-05'),
+           |(1, 'a', 31, 1000, '2021-01-05'),
+           |(3, 'a', 33, 1000, '2021-01-05'),
+           |(4, 'b', 16, 1000, '2021-01-05'),
+           |(2, 'b', 18, 1000, '2021-01-05'),
+           |(6, 'b', 17, 1000, '2021-01-05'),
+           |(8, 'a', 21, 1000, '2021-01-05'),
+           |(9, 'a', 22, 1000, '2021-01-05'),
+           |(7, 'a', 23, 1000, '2021-01-05')
+           |""".stripMargin)
+
+      // Insert overwrite static partition
+      spark.sql(
+        s"""
+           | insert overwrite table $tableName partition(dt = '2021-01-05')
+           | select * from (select 13 , 'a2', 12, 1000) limit 10
+        """.stripMargin)
+      checkAnswer(s"select id, name, price, ts, dt from $tableName order by dt")(
+        Seq(13, "a2", 12.0, 1000, "2021-01-05")
+      )
+    })
+  }
+
+  test("Test Insert Overwrite Into Bucket Index MOR Table") {
+    withRecordType()(withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  dt string
+           |) using hudi
+           |tblproperties (
+           |  type = 'mor',
+           |  primaryKey = 'id',
+           |  preCombineField = 'ts',
+           |  hoodie.index.type = 'BUCKET',
+           |  hoodie.bucket.index.num.buckets = '4'
+           |)
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}/$tableName'
+       """.stripMargin)
+
+      spark.sql(
+        s"""insert into $tableName  values
+           |(5, 'a', 35, 1000, '2021-01-05'),
+           |(1, 'a', 31, 1000, '2021-01-05'),
+           |(3, 'a', 33, 1000, '2021-01-05'),
+           |(4, 'b', 16, 1000, '2021-01-05'),
+           |(2, 'b', 18, 1000, '2021-01-05'),
+           |(6, 'b', 17, 1000, '2021-01-05'),
+           |(8, 'a', 21, 1000, '2021-01-05'),
+           |(9, 'a', 22, 1000, '2021-01-05'),
+           |(7, 'a', 23, 1000, '2021-01-05')
+           |""".stripMargin)
+
+      // Insert overwrite static partition
+      spark.sql(
+        s"""
+           | insert overwrite table $tableName partition(dt = '2021-01-05')
+           | select * from (select 13 , 'a2', 12, 1000) limit 10
+        """.stripMargin)
+      checkAnswer(s"select id, name, price, ts, dt from $tableName order by dt")(
+        Seq(13, "a2", 12.0, 1000, "2021-01-05")
+      )
+    })
+  }
+
+  test("Test Insert Overwrite Into Consistent Bucket Index Table") {
+    withRecordType()(withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  dt string
+           |) using hudi
+           |tblproperties (
+           |  type = 'mor',
+           |  primaryKey = 'id',
+           |  preCombineField = 'ts',
+           |  hoodie.index.type = 'BUCKET',
+           |  hoodie.index.bucket.engine = "CONSISTENT_HASHING",
+           |  hoodie.bucket.index.num.buckets = '4'
+           |)
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}/$tableName'
+       """.stripMargin)
+
+      spark.sql(
+        s"""insert into $tableName  values
+           |(5, 'a', 35, 1000, '2021-01-05'),
+           |(1, 'a', 31, 1000, '2021-01-05'),
+           |(3, 'a', 33, 1000, '2021-01-05'),
+           |(4, 'b', 16, 1000, '2021-01-05'),
+           |(2, 'b', 18, 1000, '2021-01-05'),
+           |(6, 'b', 17, 1000, '2021-01-05'),
+           |(8, 'a', 21, 1000, '2021-01-05'),
+           |(9, 'a', 22, 1000, '2021-01-05'),
+           |(7, 'a', 23, 1000, '2021-01-05')
+           |""".stripMargin)
+
+      // Insert overwrite static partition
+      spark.sql(
+        s"""
+           | insert overwrite table $tableName partition(dt = '2021-01-05')

Review Comment:
   this will create a new parquet file with the same prefix against log file, but something diff in fgId suffix. just like the picture, create new parquet file will add `-0` after fgId(xxx-0-0_xxx), so it can be read if only insert onece, but if  insert overwrite again, will use the same fgId(xxx-0-0), result nothing.
   <img width="588" alt="image" src="https://user-images.githubusercontent.com/20125927/221895955-195a00a6-7eda-4b4a-ac3d-c6e5f10bfd27.png">
   



-- 
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] beyond1920 commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -66,7 +67,9 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
     return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
+        .map(c -> c.equals(HoodieLayoutConfig.SIMPLE_BUCKET_LAYOUT_PARTITIONER_CLASS_NAME)

Review Comment:
   No, consistentBucketIndex works correctly, it would generate different file ids.
   



-- 
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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 41f51ef1391ec5b31b85351f0f4d6cd3b857faa6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469) 
   
   <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] beyond1920 commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -66,7 +67,9 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
     return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
+        .map(c -> c.equals(HoodieLayoutConfig.SIMPLE_BUCKET_LAYOUT_PARTITIONER_CLASS_NAME)

Review Comment:
   emm, sorry for the hurry response before.
   Thank you for point it out. 
   I need to spend more time to get familiar with `ConsistentBucketIndex`. I would response ASAP.



-- 
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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15633",
       "triggerID" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15638",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7d6319dd8880f2f5af3ba0ea1c38058499585337",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15640",
       "triggerID" : "7d6319dd8880f2f5af3ba0ea1c38058499585337",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7d6319dd8880f2f5af3ba0ea1c38058499585337 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15640) 
   
   <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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 41f51ef1391ec5b31b85351f0f4d6cd3b857faa6 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] XuQianJin-Stars commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

Posted by "XuQianJin-Stars (via GitHub)" <gi...@apache.org>.
XuQianJin-Stars commented on code in PR #8072:
URL: https://github.com/apache/hudi/pull/8072#discussion_r1119615721


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -354,7 +352,7 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
     }
   }
 
-  test("Test Insert Overwrite") {
+  test("Test Insert Overwrite for bucket ") {

Review Comment:
   add test for consistentBucketIndex



-- 
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] KnightChess commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -66,7 +67,9 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
     return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
+        .map(c -> c.equals(HoodieLayoutConfig.SIMPLE_BUCKET_LAYOUT_PARTITIONER_CLASS_NAME)

Review Comment:
   does consistentBucketIndex will not cause the same problem?



-- 
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] beyond1920 commented on pull request #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   @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 commented on pull request #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15633",
       "triggerID" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621) 
   * f5503443fd6080f0ec93a9a21e14a33f0fa432c7 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15633) 
   
   <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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15633",
       "triggerID" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5503443fd6080f0ec93a9a21e14a33f0fa432c7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15638",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "1461972444",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621) 
   
   <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] beyond1920 commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -66,7 +67,9 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
     return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
+        .map(c -> c.equals(HoodieLayoutConfig.SIMPLE_BUCKET_LAYOUT_PARTITIONER_CLASS_NAME)

Review Comment:
   @KnightChess Thanks for your advice.
   Remove tagLocation could also fixed this problem. However I prefer to fix this problem by generate new file ids because:
   1. Remove tag location would change stats, for example, miss updated count
   2. It's better to keep same behavior for all index types instead of only remove tag location in insert overwrite for bucket index table.
   But remove tag location is a good improvement to speed up insert overwrite. I would created a new JIRA to track this issue. Maybe using bulk insert to do insert overwrite for all index typed. WDYT?



-- 
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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 41f51ef1391ec5b31b85351f0f4d6cd3b857faa6 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469) 
   * 9a4747eedc49a941aec1608b143ebc3039e0e1b9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476) 
   * 5fa9c6087f405cde05439d9e9c01d5abf20f712c 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] XuQianJin-Stars commented on pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   Thanks @beyond1920 overall looks good.


-- 
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] beyond1920 commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -354,7 +352,7 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
     }
   }
 
-  test("Test Insert Overwrite") {
+  test("Test Insert Overwrite for bucket ") {

Review Comment:
   ConsistentBucketIndex works correctly, it would generate different file ids.
   However, I add the test cases for consitentBucketIndex too.



-- 
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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -65,9 +70,21 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
 
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
-    return table.getStorageLayout().layoutPartitionerClass()
-        .map(c -> getLayoutPartitioner(profile, c))
-        .orElse(new SparkInsertOverwritePartitioner(profile, context, table, config));
+    // Fix HUDI-5857, insert overwrite should generate new file group id

Review Comment:
   No need to add the JIRA id explicitly because the modifications belong to the current commit.



-- 
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] XuQianJin-Stars merged pull request #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


-- 
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] beyond1920 commented on a diff in pull request #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java:
##########
@@ -41,6 +42,25 @@ public SparkInsertOverwritePartitioner(WorkloadProfile profile, HoodieEngineCont
     super(profile, context, table, config);
   }
 
+  @Override
+  public BucketInfo getBucketInfo(int bucketNumber) {
+    BucketInfo bucketInfo = super.getBucketInfo(bucketNumber);

Review Comment:
   Sorry, I don't understand what you mean here.
   Could you please explain it 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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0099b9f29a6d7ff7582fd728f73fc216a4adb6a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513) 
   * a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15621) 
   
   <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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0099b9f29a6d7ff7582fd728f73fc216a4adb6a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513) 
   * a76dc55ea3de9fc9b5f886dc5e5162c29b651a7a 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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fa9c6087f405cde05439d9e9c01d5abf20f712c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478) 
   * d4e9858122533fb916b589d633aeda44043a6a4d 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] XuQianJin-Stars commented on a diff in pull request #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

Posted by "XuQianJin-Stars (via GitHub)" <gi...@apache.org>.
XuQianJin-Stars commented on code in PR #8072:
URL: https://github.com/apache/hudi/pull/8072#discussion_r1119586364


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -1125,4 +1123,52 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
       }
     }
   }
+
+  test("Test Insert Overwrite Into Bucket Index Table") {
+    withRecordType()(withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  dt string
+           |) using hudi
+           | tblproperties (
+           | primaryKey = 'id',
+           | preCombineField = 'ts',
+           | hoodie.index.type = 'BUCKET',
+           | 'hoodie.bucket.index.num.buckets'='4'

Review Comment:
   replace `hoodie.bucket.index.num.buckets = '4'` ?



-- 
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 #8072: [HUDI-5857] Index overwrite into bucket table would generate new file group id

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15469",
       "triggerID" : "41f51ef1391ec5b31b85351f0f4d6cd3b857faa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15476",
       "triggerID" : "9a4747eedc49a941aec1608b143ebc3039e0e1b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15478",
       "triggerID" : "5fa9c6087f405cde05439d9e9c01d5abf20f712c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505",
       "triggerID" : "d4e9858122533fb916b589d633aeda44043a6a4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513",
       "triggerID" : "d0099b9f29a6d7ff7582fd728f73fc216a4adb6a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4e9858122533fb916b589d633aeda44043a6a4d Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15505) 
   * d0099b9f29a6d7ff7582fd728f73fc216a4adb6a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15513) 
   
   <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 #8072: [HUDI-5857] Insert overwrite into bucket table would generate new file group id

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java:
##########
@@ -85,4 +102,19 @@ protected List<String> getAllExistingFileIds(String partitionPath) {
     // because new commit is not complete. it is safe to mark all existing file Ids as old files
     return table.getSliceView().getLatestFileSlices(partitionPath).map(FileSlice::getFileId).distinct().collect(Collectors.toList());
   }
+
+  @Override
+  protected Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr, Partitioner partitioner) {
+    SparkHoodiePartitioner upsertPartitioner = (SparkHoodiePartitioner) partitioner;
+    BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
+    BucketType btype = binfo.bucketType;
+    if (btype.equals(BucketType.INSERT)) {
+      return handleInsert(binfo.fileIdPrefix, recordItr);
+    } else if (btype.equals(BucketType.UPDATE)) {
+      throw new HoodieInsertOverwriteException(
+          "Insert overwrite should always use INSERT bucketType, please correct the logical of " + partitioner.getClass().getName());

Review Comment:
   In which case we can hit the code path for `BucketType.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