You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/05/23 11:46:53 UTC

[GitHub] [hudi] nsivabalan opened a new pull request, #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   ## What is the purpose of the pull request
   
   - Bulk insert row writer code path had a gap wrt hive style partitioning and default partition when virtual keys are enabled with SimpleKeyGen. Fixing the same in this patch 
   
   ## Brief change log
   
   - Fixed BulkInsertDataInternalWriterHelper to honor hiveStyle Partitioning and default partition when virtual keys are enabled for SimpleKeyGen. 
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] yihua commented on a diff in pull request #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

Posted by GitBox <gi...@apache.org>.
yihua commented on code in PR #5664:
URL: https://github.com/apache/hudi/pull/5664#discussion_r889380010


##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -87,6 +89,7 @@ public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteCo
     this.populateMetaFields = populateMetaFields;
     this.arePartitionRecordsSorted = arePartitionRecordsSorted;
     this.fileIdPrefix = UUID.randomUUID().toString();
+    this.isHiveStylePartitioning = writeConfig.isHiveStylePartitioningEnabled();

Review Comment:
   nit: `writeConfig` is saved inside this helper so we don't need to have another member variable `isHiveStylePartitioning`?



##########
hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java:
##########
@@ -109,6 +109,48 @@ public void testDataInternalWriter(boolean sorted, boolean populateMetaFields) t
     }
   }
 
+  @Test
+  public void testDataInternalWriterHiveStylePartitioning() throws Exception {
+    boolean sorted = true;
+    boolean populateMetaFields = false;
+    // init config and table
+    HoodieWriteConfig cfg = getWriteConfig(populateMetaFields, "true");
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    for (int i = 0; i < 1; i++) {
+      String instantTime = "00" + i;
+      // init writer
+      HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(),
+          STRUCT_TYPE, populateMetaFields, sorted);
+
+      int size = 10 + RANDOM.nextInt(1000);
+      // write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
+      int batches = 3;
+      Dataset<Row> totalInputRows = null;
+
+      for (int j = 0; j < batches; j++) {
+        String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
+        Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
+        writeRows(inputRows, writer);
+        if (totalInputRows == null) {
+          totalInputRows = inputRows;
+        } else {
+          totalInputRows = totalInputRows.union(inputRows);
+        }
+      }
+
+      BaseWriterCommitMessage commitMetadata = (BaseWriterCommitMessage) writer.commit();
+      Option<List<String>> fileAbsPaths = Option.of(new ArrayList<>());
+      Option<List<String>> fileNames = Option.of(new ArrayList<>());
+
+      // verify write statuses
+      assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, sorted, fileAbsPaths, fileNames);
+
+      // verify rows
+      Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
+      assertOutput(totalInputRows, result, instantTime, fileNames, populateMetaFields);

Review Comment:
   Do we want to validate the hive-style partition path value somewhere?



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -128,7 +133,11 @@ public void write(InternalRow record) throws IOException {
         if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen
           partitionPath = "";
         } else if (simpleKeyGen) { // SimpleKeyGen
-          partitionPath = (record.get(simplePartitionFieldIndex, simplePartitionFieldDataType)).toString();
+          Object parititionPathValue = record.get(simplePartitionFieldIndex, simplePartitionFieldDataType);
+          partitionPath = parititionPathValue != null ? parititionPathValue.toString() : PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
+          if (isHiveStylePartitioning) {
+            partitionPath = (keyGeneratorOpt.get()).getPartitionPathFields().get(0) + "=" + partitionPath;

Review Comment:
   For `SimpleKeyGenerator`, there could be only one partition path field.  Is that correct?



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -128,7 +133,11 @@ public void write(InternalRow record) throws IOException {
         if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen
           partitionPath = "";
         } else if (simpleKeyGen) { // SimpleKeyGen
-          partitionPath = (record.get(simplePartitionFieldIndex, simplePartitionFieldDataType)).toString();
+          Object parititionPathValue = record.get(simplePartitionFieldIndex, simplePartitionFieldDataType);
+          partitionPath = parititionPathValue != null ? parititionPathValue.toString() : PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
+          if (isHiveStylePartitioning) {
+            partitionPath = (keyGeneratorOpt.get()).getPartitionPathFields().get(0) + "=" + partitionPath;

Review Comment:
   @nsivabalan could you simply leverage `SimpleKeyGenerator::getPartitionPath(GenericRecord record)` or `KeyGenUtils::getPartitionPath` API instead of hardcoding the value construction here?



-- 
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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075",
       "triggerID" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8fbd946b6c584a7794ef61098aa5fab540861541 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074) 
   * 9fe648a4dd99d9c77dcf0be386d453646895a9d8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on a diff in pull request #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on code in PR #5664:
URL: https://github.com/apache/hudi/pull/5664#discussion_r889197028


##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -87,6 +89,9 @@ public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteCo
     this.populateMetaFields = populateMetaFields;
     this.arePartitionRecordsSorted = arePartitionRecordsSorted;
     this.fileIdPrefix = UUID.randomUUID().toString();
+    this.isHiveStylePartitioning = writeConfig.getProps().containsKey(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING().key())
+        ? Boolean.parseBoolean((String) writeConfig.getProps().get(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING().key()))
+        : Boolean.parseBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING().defaultValue());

Review Comment:
   Initially I did not add since DataSourceOptions wasn't reachable from WriteConfig. but found out that  I can use KeyGeneratorOptions. will fix it



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -128,7 +133,11 @@ public void write(InternalRow record) throws IOException {
         if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen
           partitionPath = "";
         } else if (simpleKeyGen) { // SimpleKeyGen
-          partitionPath = (record.get(simplePartitionFieldIndex, simplePartitionFieldDataType)).toString();
+          Object parititionPathValue = record.get(simplePartitionFieldIndex, simplePartitionFieldDataType);
+          partitionPath = parititionPathValue != null ? parititionPathValue.toString() : PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
+          if (isHiveStylePartitioning) {
+            partitionPath = (keyGeneratorOpt.get()).getPartitionPathFields().get(0) + "=" + partitionPath;

Review Comment:
   I have filed a follow up jira around this https://issues.apache.org/jira/browse/HUDI-4199. there could be more gaps in here. for now, this patch focuses on hive style partitioning. 



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -128,7 +133,11 @@ public void write(InternalRow record) throws IOException {
         if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen
           partitionPath = "";
         } else if (simpleKeyGen) { // SimpleKeyGen
-          partitionPath = (record.get(simplePartitionFieldIndex, simplePartitionFieldDataType)).toString();
+          Object parititionPathValue = record.get(simplePartitionFieldIndex, simplePartitionFieldDataType);
+          partitionPath = parititionPathValue != null ? parititionPathValue.toString() : PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
+          if (isHiveStylePartitioning) {
+            partitionPath = (keyGeneratorOpt.get()).getPartitionPathFields().get(0) + "=" + partitionPath;

Review Comment:
   this block is applicable only for simple key generator. for other key generators, L143 is used. so we should be 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] hudi-bot commented on pull request #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 63c2aa08ecec2dbbe98823f2b88b52874346a085 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 63c2aa08ecec2dbbe98823f2b88b52874346a085 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997) 
   * db498bb903cdd264f29c3db616dab75bccffddaf 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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 18654512f52bf46f458d0275a844fd4e625e32e4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8fbd946b6c584a7794ef61098aa5fab540861541 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074) 
   * 9fe648a4dd99d9c77dcf0be386d453646895a9d8 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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2de64cf0fa22cae39cbfe3f7902b0a45371b7c56 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 63c2aa08ecec2dbbe98823f2b88b52874346a085 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997) 
   * db498bb903cdd264f29c3db616dab75bccffddaf Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062) 
   * 18654512f52bf46f458d0275a844fd4e625e32e4 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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075",
       "triggerID" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9fe648a4dd99d9c77dcf0be386d453646895a9d8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2de64cf0fa22cae39cbfe3f7902b0a45371b7c56 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861) 
   * 2e659b9a80a72a6487d1f9f0023154f0570c134f 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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2de64cf0fa22cae39cbfe3f7902b0a45371b7c56 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861) 
   * 2e659b9a80a72a6487d1f9f0023154f0570c134f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2e659b9a80a72a6487d1f9f0023154f0570c134f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867) 
   * 63c2aa08ecec2dbbe98823f2b88b52874346a085 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997) 
   
   <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] xushiyan commented on a diff in pull request #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #5664:
URL: https://github.com/apache/hudi/pull/5664#discussion_r884231341


##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -128,7 +133,11 @@ public void write(InternalRow record) throws IOException {
         if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen
           partitionPath = "";
         } else if (simpleKeyGen) { // SimpleKeyGen
-          partitionPath = (record.get(simplePartitionFieldIndex, simplePartitionFieldDataType)).toString();
+          Object parititionPathValue = record.get(simplePartitionFieldIndex, simplePartitionFieldDataType);
+          partitionPath = parititionPathValue != null ? parititionPathValue.toString() : PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
+          if (isHiveStylePartitioning) {
+            partitionPath = (keyGeneratorOpt.get()).getPartitionPathFields().get(0) + "=" + partitionPath;

Review Comment:
   `(keyGeneratorOpt.get())` the bracket is redundant



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -87,6 +89,9 @@ public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteCo
     this.populateMetaFields = populateMetaFields;
     this.arePartitionRecordsSorted = arePartitionRecordsSorted;
     this.fileIdPrefix = UUID.randomUUID().toString();
+    this.isHiveStylePartitioning = writeConfig.getProps().containsKey(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING().key())
+        ? Boolean.parseBoolean((String) writeConfig.getProps().get(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING().key()))
+        : Boolean.parseBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING().defaultValue());

Review Comment:
   this is very verbose.. why not add an API in write config say `isHiveStylePartitioning()` ?



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java:
##########
@@ -128,7 +133,11 @@ public void write(InternalRow record) throws IOException {
         if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen
           partitionPath = "";
         } else if (simpleKeyGen) { // SimpleKeyGen
-          partitionPath = (record.get(simplePartitionFieldIndex, simplePartitionFieldDataType)).toString();
+          Object parititionPathValue = record.get(simplePartitionFieldIndex, simplePartitionFieldDataType);
+          partitionPath = parititionPathValue != null ? parititionPathValue.toString() : PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
+          if (isHiveStylePartitioning) {
+            partitionPath = (keyGeneratorOpt.get()).getPartitionPathFields().get(0) + "=" + partitionPath;

Review Comment:
   why only get the first partition path field and add `=` sign? what about multi-level partition like `year=2020/month=01/day=01` ?



-- 
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] alexeykudinkin commented on a diff in pull request #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5664:
URL: https://github.com/apache/hudi/pull/5664#discussion_r914070359


##########
hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java:
##########
@@ -109,6 +112,51 @@ public void testDataInternalWriter(boolean sorted, boolean populateMetaFields) t
     }
   }
 
+  @Test
+  public void testDataInternalWriterHiveStylePartitioning() throws Exception {
+    boolean sorted = true;
+    boolean populateMetaFields = false;
+    // init config and table
+    HoodieWriteConfig cfg = getWriteConfig(populateMetaFields, "true");
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    for (int i = 0; i < 1; i++) {
+      String instantTime = "00" + i;
+      // init writer
+      HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(),

Review Comment:
   Is this RANDOM a fixed-seed one?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on pull request #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   CI run succeeded: https://dev.azure.com/apache-hudi-ci-org/apache-hudi-ci/_build/results?buildId=9100&view=results


-- 
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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 18654512f52bf46f458d0275a844fd4e625e32e4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063) 
   * 8fbd946b6c584a7794ef61098aa5fab540861541 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2de64cf0fa22cae39cbfe3f7902b0a45371b7c56 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861) 
   * 2e659b9a80a72a6487d1f9f0023154f0570c134f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 63c2aa08ecec2dbbe98823f2b88b52874346a085 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997) 
   * db498bb903cdd264f29c3db616dab75bccffddaf Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 18654512f52bf46f458d0275a844fd4e625e32e4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063) 
   * 8fbd946b6c584a7794ef61098aa5fab540861541 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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2de64cf0fa22cae39cbfe3f7902b0a45371b7c56 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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8fbd946b6c584a7794ef61098aa5fab540861541 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075",
       "triggerID" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9100",
       "triggerID" : "cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9fe648a4dd99d9c77dcf0be386d453646895a9d8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075) 
   * cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9100) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075",
       "triggerID" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9fe648a4dd99d9c77dcf0be386d453646895a9d8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075) 
   * cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834 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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2e659b9a80a72a6487d1f9f0023154f0570c134f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2e659b9a80a72a6487d1f9f0023154f0570c134f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867) 
   * 63c2aa08ecec2dbbe98823f2b88b52874346a085 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] yihua merged pull request #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

Posted by GitBox <gi...@apache.org>.
yihua merged PR #5664:
URL: https://github.com/apache/hudi/pull/5664


-- 
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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * db498bb903cdd264f29c3db616dab75bccffddaf Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062) 
   * 18654512f52bf46f458d0275a844fd4e625e32e4 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063) 
   
   <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 #5664: [HUDI-4140] Fixing hive style partitioning and default partition with bulk insert row writer with SimpleKeyGen and virtual keys

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8861",
       "triggerID" : "2de64cf0fa22cae39cbfe3f7902b0a45371b7c56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8867",
       "triggerID" : "2e659b9a80a72a6487d1f9f0023154f0570c134f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8997",
       "triggerID" : "63c2aa08ecec2dbbe98823f2b88b52874346a085",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9062",
       "triggerID" : "db498bb903cdd264f29c3db616dab75bccffddaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9063",
       "triggerID" : "18654512f52bf46f458d0275a844fd4e625e32e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9074",
       "triggerID" : "8fbd946b6c584a7794ef61098aa5fab540861541",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9075",
       "triggerID" : "9fe648a4dd99d9c77dcf0be386d453646895a9d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9100",
       "triggerID" : "cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * cd3ddb0102915ebdb9d8ae968a34c9a8ca9b6834 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9100) 
   
   <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