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

[GitHub] [hudi] boneanxs opened a new pull request, #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   ### Change Logs
   
   1. Will not delete the table location if using `SaveMode.Overwrite`
   2. Add `bulk_insert` support for `insert_overwrite` and `insert_overwrite_table`
   3. Add tests to cover new feature
   
   ### Impact
   
   This is a breaking change for users who rely on `SaveMode.Overwrite` to drop the table, now old table data will still exist, but replaced by the new write operation.
   
   ### Risk level (write none, low medium or high below)
   low
   
   ### Documentation Update
   
   _Describe any necessary documentation update if there is any new feature, config, or user-facing change_
   
   - _The config description must be updated if new configs are added or the default value of the configs are changed_
   - _Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
     ticket number here and follow the [instruction](https://hudi.apache.org/contribute/developer-setup#website) to make
     changes to the website._
   
   ### 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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

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


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

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

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


[GitHub] [hudi] codope commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java:
##########
@@ -77,8 +78,11 @@ public HoodieData<WriteStatus> performClusteringWithRecordsAsRow(Dataset<Row> in
     // Since clustering will write to single file group using HoodieUnboundedCreateHandle, set max file size to a large value.
     newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(Long.MAX_VALUE));
 
-    return HoodieDatasetBulkInsertHelper.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig,
-        getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata);
+    BulkInsertPartitioner<Dataset<Row>> partitioner = getRowPartitioner(strategyParams, schema);
+    Dataset<Row> repartitionedRecords = partitioner.repartitionRecords(inputRecords, numOutputGroups);

Review Comment:
   Doesn't this already happen inside `HoodieDatasetBulkInsertHelper.bulkInsert`?



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala:
##########
@@ -88,17 +88,15 @@ object InsertIntoHoodieTableCommand extends Logging with ProvidesHoodieConfig wi
           extraOptions: Map[String, String] = Map.empty): Boolean = {
     val catalogTable = new HoodieCatalogTable(sparkSession, table)
 
-    var mode = SaveMode.Append
-    var isOverWriteTable = false
-    var isOverWritePartition = false
-    if (overwrite && partitionSpec.isEmpty) {
-      // insert overwrite table
-      mode = SaveMode.Overwrite
-      isOverWriteTable = true
+    val mode = if (overwrite) {
+      SaveMode.Overwrite

Review Comment:
   I think that's a good suggestion. cc @nsivabalan @yihua 



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkCommitActionExecutor.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.HoodieDatasetBulkInsertHelper;
+import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseDatasetBulkCommitActionExecutor implements Serializable {

Review Comment:
   Do we need this abstraction at a higher layer i.e. in `hudi-client-common`? And then maybe extend in hudi-spark-common for Dataset?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java:
##########
@@ -69,8 +70,11 @@ public HoodieData<WriteStatus> performClusteringWithRecordsAsRow(Dataset<Row> in
 
     newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(getWriteConfig().getClusteringMaxBytesInGroup()));
 
-    return HoodieDatasetBulkInsertHelper.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig,
-        getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata);
+    BulkInsertPartitioner<Dataset<Row>> partitioner = getRowPartitioner(strategyParams, schema);
+    Dataset<Row> repartitionedRecords = partitioner.repartitionRecords(inputRecords, numOutputGroups);

Review Comment:
   same here..why do we need to repartition before calling bulk insert?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieInternalConfig.java:
##########
@@ -31,11 +31,21 @@ public class HoodieInternalConfig extends HoodieConfig {
   public static final String BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED = "hoodie.bulkinsert.are.partitioner.records.sorted";
   public static final Boolean DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED = false;
 
+  public static final String BULK_INSERT_OVERWRITE_TABLE = "INSERT_OVERWRITE_TABLE";
+  public static final String BULK_INSERT_OVERWRITE = "INSERT_OVERWRITE";
+
   public static final ConfigProperty<String> BULKINSERT_INPUT_DATA_SCHEMA_DDL = ConfigProperty
       .key("hoodie.bulkinsert.schema.ddl")
       .noDefaultValue()
       .withDocumentation("Schema set for row writer/bulk insert.");
 
+  public static final ConfigProperty<String> BULKINSERT_OVERWRITE_MODE = ConfigProperty
+      .key("hoodie.bulkinsert.overwrite.mode")

Review Comment:
   The value for this config is a write operation type. So, its key should be named accordingly.



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkCommitActionExecutor.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.HoodieDatasetBulkInsertHelper;
+import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseDatasetBulkCommitActionExecutor implements Serializable {

Review Comment:
   ```suggestion
   public abstract class BaseDatasetBulkInsertCommitActionExecutor implements Serializable {
   ```



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkCommitActionExecutor.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.HoodieDatasetBulkInsertHelper;
+import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseDatasetBulkCommitActionExecutor implements Serializable {
+
+  protected final HoodieWriteConfig writeConfig;

Review Comment:
   Do we need to serialize write config too or can it be transient?



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 9fa7e9923c01a635744d23314bd7bafce83aee76 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591) 
   * 8becde27e89beaa60930f6ddff5d681971092836 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 1fadedfb975375bba6571e7ecf51de55d7e8dae2 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 4d8ba7c81f123af23dca05642c5562ac79eed4ff Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 1fadedfb975375bba6571e7ecf51de55d7e8dae2 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674) 
   * 851a1c3d1bbb317a01822b67ac14ea16ba49fa5e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284) 
   
   <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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSourceDAGExecution.scala:
##########
@@ -89,7 +89,7 @@ class TestSparkDataSourceDAGExecution extends HoodieSparkClientTestBase with Sca
   @CsvSource(Array(
     "upsert,org.apache.hudi.client.SparkRDDWriteClient.commit",
     "insert,org.apache.hudi.client.SparkRDDWriteClient.commit",
-    "bulk_insert,org.apache.hudi.HoodieSparkSqlWriter$.bulkInsertAsRow"))
+    "bulk_insert,org.apache.hudi.HoodieDatasetBulkInsertHelper.bulkInsert"))

Review Comment:
   `bulk_insert` with `Overwrite` mode will change to use `DatasetBulkInsertOverwriteTableCommitActionExecutor` to avoid deleting basePath first. So it would call `FSUtils.getAllPartitionPaths` to get all existing partition paths, which would introduce extra stages containing `HoodieSparkSqlWriter$.bulkInsertAsRow`, but it won't call write operation multi times, can see from the screenshot.
   
   ![Screenshot 2023-05-23 at 14 45 58](https://github.com/apache/hudi/assets/10115332/d2e0cf3b-65ed-45a3-b786-0778f9b8e395)
   
   
   



-- 
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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * f384bbc843028360687903b3b6de835685235b68 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570) 
   
   <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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8432800aa63cc5e4d4384f2ade7747aff96bc1c0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506) 
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * f384bbc843028360687903b3b6de835685235b68 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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8432800aa63cc5e4d4384f2ade7747aff96bc1c0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506) 
   
   <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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala:
##########
@@ -88,17 +88,15 @@ object InsertIntoHoodieTableCommand extends Logging with ProvidesHoodieConfig wi
           extraOptions: Map[String, String] = Map.empty): Boolean = {
     val catalogTable = new HoodieCatalogTable(sparkSession, table)
 
-    var mode = SaveMode.Append
-    var isOverWriteTable = false
-    var isOverWritePartition = false
-    if (overwrite && partitionSpec.isEmpty) {
-      // insert overwrite table
-      mode = SaveMode.Overwrite
-      isOverWriteTable = true
+    val mode = if (overwrite) {
+      SaveMode.Overwrite

Review Comment:
   Given the `Overwrite` mode doesn't care abt the old data, do we need to enable `bulk_insert` by default if it's `Overwrite` mode?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -850,9 +854,12 @@ object HoodieSparkSqlWriter {
     if (operation != WriteOperationType.DELETE) {
       if (mode == SaveMode.ErrorIfExists && tableExists) {
         throw new HoodieException(s"hoodie table at $tablePath already exists.")
-      } else if (mode == SaveMode.Overwrite && tableExists && operation != WriteOperationType.INSERT_OVERWRITE_TABLE) {
-        // When user set operation as INSERT_OVERWRITE_TABLE,
-        // overwrite will use INSERT_OVERWRITE_TABLE operator in doWriteOperation
+      } else if (mode == SaveMode.Overwrite && tableExists &&

Review Comment:
   Not sure why we need to explicitly delete old data if it's `Overwrite` mode, this behavior actually make the HUDI not ACID-compliant(I keep it here to make the tests pass).
   
   Maybe we should only delete old data if using `drop table` command?



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * be0af151295a90e2a8878b02f2dec13d6e1b6bbf Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017) 
   * 4d8ba7c81f123af23dca05642c5562ac79eed4ff Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089) 
   
   <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] boneanxs commented on pull request #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   @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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 8becde27e89beaa60930f6ddff5d681971092836 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101",
       "triggerID" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b796358f1e324044cf0dfb9afe10cb62e9b132e7",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18161",
       "triggerID" : "b796358f1e324044cf0dfb9afe10cb62e9b132e7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * b796358f1e324044cf0dfb9afe10cb62e9b132e7 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18161) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

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

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


[GitHub] [hudi] codope commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieInternalConfig.java:
##########
@@ -37,6 +38,13 @@ public class HoodieInternalConfig extends HoodieConfig {
       .markAdvanced()
       .withDocumentation("Schema set for row writer/bulk insert.");
 
+  public static final ConfigProperty<String> BULKINSERT_OVERWRITE_OPERATION_TYPE = ConfigProperty
+      .key("hoodie.bulkinsert.overwrite.operation.type")
+      .noDefaultValue()

Review Comment:
   Let's also mark it advanced (`.markAdvanced()`). 



-- 
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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -850,9 +854,12 @@ object HoodieSparkSqlWriter {
     if (operation != WriteOperationType.DELETE) {
       if (mode == SaveMode.ErrorIfExists && tableExists) {
         throw new HoodieException(s"hoodie table at $tablePath already exists.")
-      } else if (mode == SaveMode.Overwrite && tableExists && operation != WriteOperationType.INSERT_OVERWRITE_TABLE) {
-        // When user set operation as INSERT_OVERWRITE_TABLE,
-        // overwrite will use INSERT_OVERWRITE_TABLE operator in doWriteOperation
+      } else if (mode == SaveMode.Overwrite && tableExists &&

Review Comment:
   > Do you mean, for Overwrite mode, we should not delete the basePath. Just overwrite the existing data.
   
   Yea
   
   > Probably something to tackle in another PR.
   
   Sure, will fix it in another PR



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

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

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


[GitHub] [hudi] codope commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -850,9 +854,12 @@ object HoodieSparkSqlWriter {
     if (operation != WriteOperationType.DELETE) {
       if (mode == SaveMode.ErrorIfExists && tableExists) {
         throw new HoodieException(s"hoodie table at $tablePath already exists.")
-      } else if (mode == SaveMode.Overwrite && tableExists && operation != WriteOperationType.INSERT_OVERWRITE_TABLE) {
-        // When user set operation as INSERT_OVERWRITE_TABLE,
-        // overwrite will use INSERT_OVERWRITE_TABLE operator in doWriteOperation
+      } else if (mode == SaveMode.Overwrite && tableExists &&

Review Comment:
   Please file a JIRA to track this change.



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertCommitActionExecutor.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieInternalConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.DataSourceInternalWriterHelper;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertCommitActionExecutor extends BaseDatasetBulkInsertCommitActionExecutor {
+
+  public DatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config,
+                                               SparkRDDWriteClient writeClient,
+                                               String instantTime) {
+    super(config, writeClient, instantTime);
+  }
+
+  @Override
+  protected void preExecute() {
+    // no op
+  }
+
+  @Override
+  protected HoodieData<WriteStatus> doExecute(Dataset<Row> records, boolean arePartitionRecordsSorted) {
+    Map<String, String> opts = writeConfig.getProps().entrySet().stream().collect(Collectors.toMap(
+        e -> String.valueOf(e.getKey()),
+        e -> String.valueOf(e.getValue())));
+    Map<String, String> optsOverrides = Collections.singletonMap(
+        HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, String.valueOf(arePartitionRecordsSorted));
+
+    String targetFormat;
+    Map<String, String> customOpts = new HashMap<>(1);
+    if (HoodieSparkUtils.isSpark2()) {
+      targetFormat = "org.apache.hudi.internal";
+    } else if (HoodieSparkUtils.isSpark3()) {
+      targetFormat = "org.apache.hudi.spark3.internal";
+      customOpts.put(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key(), records.schema().json());
+    } else {
+      throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
+          + " To use row writer please switch to spark 2 or spark 3");
+    }
+
+    records.write().format(targetFormat)
+        .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
+        .options(opts)
+        .options(customOpts)
+        .options(optsOverrides)
+        .mode(SaveMode.Append)
+        .save();
+    return null;

Review Comment:
   Then how about returning `Option<HoodieData<WriteStatus>>` or maybe empty HoodieData if the return is not needed at the call site? Returning null can be potentially dangerous, if another author adds some change with the assumption that WriteStatus will always be present.



-- 
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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkInsertCommitActionExecutor.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.HoodieDatasetBulkInsertHelper;
+import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseDatasetBulkInsertCommitActionExecutor implements Serializable {
+
+  protected final transient HoodieWriteConfig writeConfig;
+  protected final transient SparkRDDWriteClient writeClient;
+  protected final String instantTime;
+  protected HoodieTable table;
+
+  public BaseDatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config,
+                                                   SparkRDDWriteClient writeClient,
+                                                   String instantTime) {
+    this.writeConfig = config;
+    this.writeClient = writeClient;
+    this.instantTime = instantTime;
+  }
+
+  protected void preExecute() {
+    table.validateInsertSchema();
+    writeClient.startCommitWithTime(instantTime, getCommitActionType());
+    writeClient.preWrite(instantTime, getWriteOperationType(), table.getMetaClient());
+  }
+
+  protected abstract HoodieData<WriteStatus> doExecute(Dataset<Row> records, boolean arePartitionRecordsSorted);
+
+  protected void afterExecute(HoodieWriteMetadata<JavaRDD<WriteStatus>> result) {
+    writeClient.postWrite(result, instantTime, table);
+  }
+
+  private HoodieWriteMetadata<JavaRDD<WriteStatus>> buildHoodieWriteMetadata(HoodieData<WriteStatus> writeStatuses) {
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> hoodieWriteMetadata = new HoodieWriteMetadata<>();
+    if (writeStatuses != null) {
+      hoodieWriteMetadata.setWriteStatuses(HoodieJavaRDD.getJavaRDD(writeStatuses));
+      hoodieWriteMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeStatuses));
+    }
+    return hoodieWriteMetadata;
+  }
+
+  public final HoodieWriteResult execute(Dataset<Row> records, boolean isTablePartitioned) {
+    if (writeConfig.getBoolean(DataSourceWriteOptions.INSERT_DROP_DUPS())) {
+      throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet");
+    }
+
+    boolean populateMetaFields = writeConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS);
+
+    BulkInsertPartitioner<Dataset<Row>> bulkInsertPartitionerRows = getPartitioner(populateMetaFields, isTablePartitioned);
+    boolean shouldDropPartitionColumns = writeConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS());
+    Dataset<Row> hoodieDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(records, writeConfig, bulkInsertPartitionerRows, shouldDropPartitionColumns);
+
+    table = writeClient.initTable(WriteOperationType.INSERT_OVERWRITE, Option.ofNullable(instantTime));

Review Comment:
   Fixed



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertCommitActionExecutor.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieInternalConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.DataSourceInternalWriterHelper;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertCommitActionExecutor extends BaseDatasetBulkInsertCommitActionExecutor {
+
+  public DatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config,
+                                               SparkRDDWriteClient writeClient,
+                                               String instantTime) {
+    super(config, writeClient, instantTime);
+  }
+
+  @Override
+  protected void preExecute() {
+    // no op
+  }
+
+  @Override
+  protected HoodieData<WriteStatus> doExecute(Dataset<Row> records, boolean arePartitionRecordsSorted) {
+    Map<String, String> opts = writeConfig.getProps().entrySet().stream().collect(Collectors.toMap(
+        e -> String.valueOf(e.getKey()),
+        e -> String.valueOf(e.getValue())));
+    Map<String, String> optsOverrides = Collections.singletonMap(
+        HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, String.valueOf(arePartitionRecordsSorted));
+
+    String targetFormat;
+    Map<String, String> customOpts = new HashMap<>(1);
+    if (HoodieSparkUtils.isSpark2()) {
+      targetFormat = "org.apache.hudi.internal";
+    } else if (HoodieSparkUtils.isSpark3()) {
+      targetFormat = "org.apache.hudi.spark3.internal";
+      customOpts.put(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key(), records.schema().json());
+    } else {
+      throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
+          + " To use row writer please switch to spark 2 or spark 3");
+    }
+
+    records.write().format(targetFormat)
+        .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
+        .options(opts)
+        .options(customOpts)
+        .options(optsOverrides)
+        .mode(SaveMode.Append)
+        .save();
+    return null;

Review Comment:
   `BULK_INSERT` doesn't need to return `WriteStatus`(don't need to execute `afterExecute` method), since it call dataframe api `records.write()` to perform write operation, it will write the commit data after the write operation is done(in `HoodieDataSourceInternalBatchWrite#commit`, `dataSourceInternalWriterHelper.commit`)



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -772,66 +772,71 @@ object HoodieSparkSqlWriter {
     }
   }
 
-  def bulkInsertAsRow(sqlContext: SQLContext,
+  def bulkInsertAsRow(writeClient: SparkRDDWriteClient[_],
+                      parameters: Map[String, String],
                       hoodieConfig: HoodieConfig,
                       df: DataFrame,
+                      mode: SaveMode,
                       tblName: String,
                       basePath: Path,
-                      path: String,
                       instantTime: String,
                       writerSchema: Schema,
-                      isTablePartitioned: Boolean): (Boolean, common.util.Option[String]) = {
+                      tableConfig: HoodieTableConfig):
+  (Boolean, HOption[String], HOption[String], HOption[String], SparkRDDWriteClient[_], HoodieTableConfig) = {
     if (hoodieConfig.getBoolean(INSERT_DROP_DUPS)) {
       throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
     }
+    val sqlContext = writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getSqlContext
+    val jsc = writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getJavaSparkContext
 
     val writerSchemaStr = writerSchema.toString
 
-    val opts = hoodieConfig.getProps.toMap ++
+    // Make opts mutable since it could be modified by tryOverrideParquetWriteLegacyFormatProperty
+    val opts = mutable.Map() ++ hoodieConfig.getProps.toMap ++
       Map(HoodieWriteConfig.AVRO_SCHEMA_STRING.key -> writerSchemaStr)
 
-    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, path, tblName, mapAsJavaMap(opts))
-    val populateMetaFields = hoodieConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)
-
-    val bulkInsertPartitionerRows: BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
-      val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
-      if (userDefinedBulkInsertPartitionerOpt.isPresent) {
-        userDefinedBulkInsertPartitionerOpt.get
-      } else {
-        BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig, isTablePartitioned)
-      }
-    } else {
-      // Sort modes are not yet supported when meta fields are disabled
-      new NonSortPartitionerWithRows()
+    // Auto set the value of "hoodie.parquet.writelegacyformat.enabled"
+    tryOverrideParquetWriteLegacyFormatProperty(opts, convertAvroSchemaToStructType(writerSchema))
+    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, basePath.toString, tblName, opts)
+    val executor = mode match {
+      case SaveMode.Append =>
+        new DatasetBulkInsertCommitActionExecutor(writeConfig, writeClient, instantTime)
+      case SaveMode.Overwrite =>
+        WriteOperationType.fromValue(hoodieConfig
+          .getStringOrDefault(HoodieInternalConfig.BULKINSERT_OVERWRITE_OPERATION_TYPE)) match {
+          case WriteOperationType.INSERT_OVERWRITE =>
+            new DatasetBulkInsertOverwriteCommitActionExecutor(writeConfig, writeClient, instantTime)
+          case WriteOperationType.INSERT_OVERWRITE_TABLE =>
+            new DatasetBulkInsertOverwriteTableCommitActionExecutor(writeConfig, writeClient, instantTime);
+        }
+      case _ =>
+        throw new HoodieException(s"$mode with bulk_insert in row writer path is not supported yet");
     }
 
-    val shouldDropPartitionColumns = hoodieConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS)
-    val hoodieDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(df, writeConfig, bulkInsertPartitionerRows, shouldDropPartitionColumns)
+    val writeResult = executor.execute(df, tableConfig.isTablePartitioned)
 
-    val optsOverrides = Map(
-      HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED ->
-        bulkInsertPartitionerRows.arePartitionRecordsSorted().toString
-    )
+    try {
+      val (writeSuccessful, compactionInstant, clusteringInstant) = mode match {
+        case SaveMode.Append =>
+          val syncHiveSuccess = metaSync(sqlContext.sparkSession, writeConfig, basePath, df.schema)
+          (syncHiveSuccess, HOption.empty().asInstanceOf[HOption[String]], HOption.empty().asInstanceOf[HOption[String]])
+        case SaveMode.Overwrite =>
+          try {
+            commitAndPerformPostOperations(sqlContext.sparkSession, df.schema, writeResult, parameters, writeClient, tableConfig, jsc,
+                TableInstantInfo(basePath, instantTime, executor.getCommitActionType, executor.getWriteOperationType), Option.empty)
 
-    val (targetFormat, customOpts) = if (HoodieSparkUtils.isSpark2) {
-      ("org.apache.hudi.internal", Map())
-    } else if (HoodieSparkUtils.isSpark3) {
-      ("org.apache.hudi.spark3.internal", Map(
-        HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key -> hoodieDF.schema.json
-      ))
-    } else {
-      throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
-        + " To use row writer please switch to spark 2 or spark 3")
+          }
+      }
+      (writeSuccessful, common.util.Option.ofNullable(instantTime), compactionInstant, clusteringInstant, writeClient, tableConfig)

Review Comment:
   done



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 851a1c3d1bbb317a01822b67ac14ea16ba49fa5e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284) 
   
   <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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8432800aa63cc5e4d4384f2ade7747aff96bc1c0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506) 
   * 6a239ada8998fd440f19c0082b26d206ed589870 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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   @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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 80d97a0abaf04b69d0b0887463794396f0ace49f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479) 
   * 84f88aaf71b5e7e2040059ac567cf255002cfcb5 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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertCommitActionExecutor.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieInternalConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.DataSourceInternalWriterHelper;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertCommitActionExecutor extends BaseDatasetBulkInsertCommitActionExecutor {
+
+  public DatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config,
+                                               SparkRDDWriteClient writeClient,
+                                               String instantTime) {
+    super(config, writeClient, instantTime);
+  }
+
+  @Override
+  protected void preExecute() {
+    // no op
+  }
+
+  @Override
+  protected HoodieData<WriteStatus> doExecute(Dataset<Row> records, boolean arePartitionRecordsSorted) {
+    Map<String, String> opts = writeConfig.getProps().entrySet().stream().collect(Collectors.toMap(
+        e -> String.valueOf(e.getKey()),
+        e -> String.valueOf(e.getValue())));
+    Map<String, String> optsOverrides = Collections.singletonMap(
+        HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, String.valueOf(arePartitionRecordsSorted));
+
+    String targetFormat;
+    Map<String, String> customOpts = new HashMap<>(1);
+    if (HoodieSparkUtils.isSpark2()) {
+      targetFormat = "org.apache.hudi.internal";
+    } else if (HoodieSparkUtils.isSpark3()) {
+      targetFormat = "org.apache.hudi.spark3.internal";
+      customOpts.put(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key(), records.schema().json());
+    } else {
+      throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
+          + " To use row writer please switch to spark 2 or spark 3");
+    }
+
+    records.write().format(targetFormat)
+        .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
+        .options(opts)
+        .options(customOpts)
+        .options(optsOverrides)
+        .mode(SaveMode.Append)
+        .save();
+    return null;

Review Comment:
   make sense, let me change it



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

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

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


[GitHub] [hudi] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   > @boneanxs Can you please rebase?
   
   @codope done, and all comments are addressed.


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

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

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


[GitHub] [hudi] codope merged pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * f4cf8eb1001906a9f93677f37c2cb028dd049106 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094) 
   
   <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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * be2128221d76e0d0f6c26db5c87122c6efdc726c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481) 
   * 6b1fd158b5e0a0102cc5a5312a31a34828b7a06c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503) 
   
   <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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * be2128221d76e0d0f6c26db5c87122c6efdc726c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481) 
   * 6b1fd158b5e0a0102cc5a5312a31a34828b7a06c 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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6b1fd158b5e0a0102cc5a5312a31a34828b7a06c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503) 
   * 8432800aa63cc5e4d4384f2ade7747aff96bc1c0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506) 
   
   <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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * f384bbc843028360687903b3b6de835685235b68 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570) 
   * 9fa7e9923c01a635744d23314bd7bafce83aee76 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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 1fadedfb975375bba6571e7ecf51de55d7e8dae2 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674) 
   * 851a1c3d1bbb317a01822b67ac14ea16ba49fa5e 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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -599,138 +582,250 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
     }
   }
 
-  test("Test bulk insert") {
+  test("Test bulk insert with insert into for single partitioned table") {
     withSQLConf("hoodie.sql.insert.mode" -> "non-strict") {
       withRecordType()(withTempDir { tmp =>
         Seq("cow", "mor").foreach {tableType =>
-          // Test bulk insert for single partition
-          val tableName = generateTableName
-          spark.sql(
-            s"""
-               |create table $tableName (
-               |  id int,
-               |  name string,
-               |  price double,
-               |  dt string
-               |) using hudi
-               | tblproperties (
-               |  type = '$tableType',
-               |  primaryKey = 'id'
-               | )
-               | partitioned by (dt)
-               | location '${tmp.getCanonicalPath}/$tableName'
-       """.stripMargin)
-          spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false")
+          withTable(generateTableName) { tableName =>
+            spark.sql(
+              s"""
+                 |create table $tableName (
+                 |  id int,
+                 |  name string,
+                 |  price double,
+                 |  dt string
+                 |) using hudi
+                 | tblproperties (
+                 |  type = '$tableType',
+                 |  primaryKey = 'id'
+                 | )
+                 | partitioned by (dt)
+                 | location '${tmp.getCanonicalPath}/$tableName'
+         """.stripMargin)
+            spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false")
 
-          // Enable the bulk insert
-          spark.sql("set hoodie.sql.bulk.insert.enable = true")
-          spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')")
+            // Enable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = true")
+            spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')")
 
-          assertResult(WriteOperationType.BULK_INSERT) {
-            getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            assertResult(WriteOperationType.BULK_INSERT) {
+              getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            }
+            checkAnswer(s"select id, name, price, dt from $tableName")(
+              Seq(1, "a1", 10.0, "2021-07-18")
+            )
+
+            // Disable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = false")
+            spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')")
+
+            assertResult(WriteOperationType.INSERT) {
+              getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            }
+            checkAnswer(s"select id, name, price, dt from $tableName order by id")(
+              Seq(1, "a1", 10.0, "2021-07-18"),
+              Seq(2, "a2", 10.0, "2021-07-18")
+            )
           }
-          checkAnswer(s"select id, name, price, dt from $tableName")(
-            Seq(1, "a1", 10.0, "2021-07-18")
-          )
+        }
+      })
+    }
+  }
 
-          // Disable the bulk insert
-          spark.sql("set hoodie.sql.bulk.insert.enable = false")
-          spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')")
+  test("Test bulk insert with insert into for multi partitioned table") {
+    withSQLConf("hoodie.sql.insert.mode" -> "non-strict") {
+      withRecordType()(withTempDir { tmp =>
+        Seq("cow", "mor").foreach { tableType =>
+          withTable(generateTableName) { tableMultiPartition =>
+            spark.sql(
+              s"""
+                 |create table $tableMultiPartition (
+                 |  id int,
+                 |  name string,
+                 |  price double,
+                 |  dt string,
+                 |  hh string
+                 |) using hudi
+                 | tblproperties (
+                 |  type = '$tableType',
+                 |  primaryKey = 'id'
+                 | )
+                 | partitioned by (dt, hh)
+                 | location '${tmp.getCanonicalPath}/$tableMultiPartition'
+         """.stripMargin)
 
-          assertResult(WriteOperationType.INSERT) {
-            getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            // Enable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = true")
+            spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, '2021-07-18', '12')")
+
+            checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition")(
+              Seq(1, "a1", 10.0, "2021-07-18", "12")
+            )
+            // Disable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = false")
+            spark.sql(s"insert into $tableMultiPartition " +
+              s"values(2, 'a2', 10, '2021-07-18','12')")
+
+            checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition order by id")(
+              Seq(1, "a1", 10.0, "2021-07-18", "12"),
+              Seq(2, "a2", 10.0, "2021-07-18", "12")
+            )
           }
-          checkAnswer(s"select id, name, price, dt from $tableName order by id")(
-            Seq(1, "a1", 10.0, "2021-07-18"),
-            Seq(2, "a2", 10.0, "2021-07-18")
-          )
+        }
+      })
+    }
+  }
 
-          // Test bulk insert for multi-level partition
-          val tableMultiPartition = generateTableName
-          spark.sql(
-            s"""
-               |create table $tableMultiPartition (
-               |  id int,
-               |  name string,
-               |  price double,
-               |  dt string,
-               |  hh string
-               |) using hudi
-               | tblproperties (
-               |  type = '$tableType',
-               |  primaryKey = 'id'
-               | )
-               | partitioned by (dt, hh)
-               | location '${tmp.getCanonicalPath}/$tableMultiPartition'
-       """.stripMargin)
+  test("Test bulk insert with insert into for non partitioned table") {

Review Comment:
   `Test bulk insert with insert overwrite table` test `INSERT_OVERWRITE_TABLE`,
   
   `Test bulk insert with insert overwrite partition` test `INSERT_OVERWRITE`
   
   These two tests test all values of `BULKINSERT_OVERWRITE_OPERATION_TYPE`



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 851a1c3d1bbb317a01822b67ac14ea16ba49fa5e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284) 
   * 03ad0c018d1a929b55d30934d74c9ba84509e88b 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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   Hey, @codope, all comments are resolved, could you pls review it again?
   


-- 
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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   Hi @codope @stream2000  Gentle ping... Could you please take a look again?


-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 03ad0c018d1a929b55d30934d74c9ba84509e88b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300) 
   * 80d97a0abaf04b69d0b0887463794396f0ace49f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

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

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


[GitHub] [hudi] codope commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSourceDAGExecution.scala:
##########
@@ -89,7 +89,7 @@ class TestSparkDataSourceDAGExecution extends HoodieSparkClientTestBase with Sca
   @CsvSource(Array(
     "upsert,org.apache.hudi.client.SparkRDDWriteClient.commit",
     "insert,org.apache.hudi.client.SparkRDDWriteClient.commit",
-    "bulk_insert,org.apache.hudi.HoodieSparkSqlWriter$.bulkInsertAsRow"))
+    "bulk_insert,org.apache.hudi.HoodieDatasetBulkInsertHelper.bulkInsert"))

Review Comment:
   Got it.



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

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

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


[GitHub] [hudi] hudi-bot commented on pull request #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * be2128221d76e0d0f6c26db5c87122c6efdc726c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481) 
   
   <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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   Hey @alexeykudinkin @nsivabalan, could you please take a look?


-- 
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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 9fa7e9923c01a635744d23314bd7bafce83aee76 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591) 
   
   <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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   Hi, @stream2000 Could you also please review this, this fixes the pr https://github.com/apache/hudi/pull/8015


-- 
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] stream2000 commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -770,66 +770,70 @@ object HoodieSparkSqlWriter {
     }
   }
 
-  def bulkInsertAsRow(sqlContext: SQLContext,
+  def bulkInsertAsRow(writeClient: SparkRDDWriteClient[_],
+                      parameters: Map[String, String],
                       hoodieConfig: HoodieConfig,
                       df: DataFrame,
+                      mode: SaveMode,
                       tblName: String,
                       basePath: Path,
-                      path: String,
                       instantTime: String,
                       writerSchema: Schema,
-                      isTablePartitioned: Boolean): (Boolean, common.util.Option[String]) = {
+                      tableConfig: HoodieTableConfig):
+  (Boolean, HOption[String], HOption[String], HOption[String], SparkRDDWriteClient[_], HoodieTableConfig) = {
     if (hoodieConfig.getBoolean(INSERT_DROP_DUPS)) {
       throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
     }
+    val sqlContext = writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getSqlContext
+    val jsc = writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getJavaSparkContext
 
     val writerSchemaStr = writerSchema.toString
 
-    val opts = hoodieConfig.getProps.toMap ++
+    // Make opts mutable since it could be modified by tryOverrideParquetWriteLegacyFormatProperty
+    val opts = mutable.Map() ++ hoodieConfig.getProps.toMap ++
       Map(HoodieWriteConfig.AVRO_SCHEMA_STRING.key -> writerSchemaStr)
 
-    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, path, tblName, mapAsJavaMap(opts))
-    val populateMetaFields = hoodieConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)
-
-    val bulkInsertPartitionerRows: BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
-      val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
-      if (userDefinedBulkInsertPartitionerOpt.isPresent) {
-        userDefinedBulkInsertPartitionerOpt.get
-      } else {
-        BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig, isTablePartitioned)
-      }
-    } else {
-      // Sort modes are not yet supported when meta fields are disabled
-      new NonSortPartitionerWithRows()
+    // Auto set the value of "hoodie.parquet.writelegacyformat.enabled"
+    tryOverrideParquetWriteLegacyFormatProperty(opts, convertAvroSchemaToStructType(writerSchema))
+    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, basePath.toString, tblName, opts)
+    val executor = mode match {
+      case SaveMode.Append =>
+        new DatasetBulkInsertActionExecutor(writeConfig, writeClient, instantTime)

Review Comment:
   Could we use writeClient to do the insert overwrite indead of calling the xxxActionExecutor directly? 



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteTableActionExecutor.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaPairRDD;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class DatasetBulkInsertOverwriteTableActionExecutor extends DatasetBulkInsertOverwriteActionExecutor {

Review Comment:
   Ditto, DatasetBulkInsertOverwriteTableCommitActionExecutor



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteActionExecutor.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.HoodieDatasetBulkInsertHelper;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaPairRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertOverwriteActionExecutor extends BaseDatasetBulkCommitActionExecutor {

Review Comment:
   Ditto, DatasetBulkInsertOverwriteCommitActionExecutor



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertActionExecutor.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieInternalConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.DataSourceInternalWriterHelper;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertActionExecutor extends BaseDatasetBulkCommitActionExecutor {

Review Comment:
   Maybe we should change DatasetBulkInsertActionExecutor -> DatasetBulkInsertCommitActionExecutor  since it is sub class of BaseCommitActionExecutor



-- 
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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   > can we just keep the bulk insert part, while extract the behavior change (i.e. not delete the table location if using SaveMode.Overwrite for bulk_insert, insert_overwrite) to a separate PR
   
   @codope removed the breaking changes


-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 851a1c3d1bbb317a01822b67ac14ea16ba49fa5e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284) 
   * 03ad0c018d1a929b55d30934d74c9ba84509e88b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 4d8ba7c81f123af23dca05642c5562ac79eed4ff Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089) 
   * f4cf8eb1001906a9f93677f37c2cb028dd049106 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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * f4cf8eb1001906a9f93677f37c2cb028dd049106 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094) 
   * 02b3132bf72d0d456d495a06501fd051d50b381e 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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertActionExecutor.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieInternalConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.DataSourceInternalWriterHelper;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertActionExecutor extends BaseDatasetBulkCommitActionExecutor {
+
+  public DatasetBulkInsertActionExecutor(HoodieWriteConfig config, SparkRDDWriteClient writeClient, String instantTime) {
+    super(config, writeClient, instantTime);
+  }
+
+  @Override
+  protected void preExecute() {
+    // no op
+  }
+
+  @Override
+  protected HoodieData<WriteStatus> doExecute(Dataset<Row> records, boolean arePartitionRecordsSorted) {
+    Map<String, String> opts = writeConfig.getProps().entrySet().stream().collect(Collectors.toMap(
+        e -> String.valueOf(e.getKey()),
+        e -> String.valueOf(e.getValue())));
+    Map<String, String> optsOverrides = Collections.singletonMap(
+        HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, String.valueOf(arePartitionRecordsSorted));
+
+    String targetFormat;
+    Map<String, String> customOpts = new HashMap<>(1);
+    if (HoodieSparkUtils.isSpark2()) {
+      targetFormat = "org.apache.hudi.internal";
+    } else if (HoodieSparkUtils.isSpark3()) {
+      targetFormat = "org.apache.hudi.spark3.internal";
+      customOpts.put(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key(), records.schema().json());
+    } else {
+      throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
+          + " To use row writer please switch to spark 2 or spark 3");
+    }
+
+    records.write().format(targetFormat)

Review Comment:
   Still keep the old behavior here to do the `bulk_insert`, maybe we should also use `HoodieDatasetBulkInsertHelper.bulkInsert` to perform write operation? We can reduce many codes for handling `commit` behavior(Here will add a complete commit, while `HoodieDatasetBulkInsertHelper.bulkInsert` doesn't, we need to handle this differently in `bulkInsertAsRow`)



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 8becde27e89beaa60930f6ddff5d681971092836 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733) 
   * 5914b1a1fedea0ae708e5e2f96130c2a73dd5b66 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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -770,66 +770,70 @@ object HoodieSparkSqlWriter {
     }
   }
 
-  def bulkInsertAsRow(sqlContext: SQLContext,
+  def bulkInsertAsRow(writeClient: SparkRDDWriteClient[_],
+                      parameters: Map[String, String],
                       hoodieConfig: HoodieConfig,
                       df: DataFrame,
+                      mode: SaveMode,
                       tblName: String,
                       basePath: Path,
-                      path: String,
                       instantTime: String,
                       writerSchema: Schema,
-                      isTablePartitioned: Boolean): (Boolean, common.util.Option[String]) = {
+                      tableConfig: HoodieTableConfig):
+  (Boolean, HOption[String], HOption[String], HOption[String], SparkRDDWriteClient[_], HoodieTableConfig) = {
     if (hoodieConfig.getBoolean(INSERT_DROP_DUPS)) {
       throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
     }
+    val sqlContext = writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getSqlContext
+    val jsc = writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getJavaSparkContext
 
     val writerSchemaStr = writerSchema.toString
 
-    val opts = hoodieConfig.getProps.toMap ++
+    // Make opts mutable since it could be modified by tryOverrideParquetWriteLegacyFormatProperty
+    val opts = mutable.Map() ++ hoodieConfig.getProps.toMap ++
       Map(HoodieWriteConfig.AVRO_SCHEMA_STRING.key -> writerSchemaStr)
 
-    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, path, tblName, mapAsJavaMap(opts))
-    val populateMetaFields = hoodieConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)
-
-    val bulkInsertPartitionerRows: BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
-      val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
-      if (userDefinedBulkInsertPartitionerOpt.isPresent) {
-        userDefinedBulkInsertPartitionerOpt.get
-      } else {
-        BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig, isTablePartitioned)
-      }
-    } else {
-      // Sort modes are not yet supported when meta fields are disabled
-      new NonSortPartitionerWithRows()
+    // Auto set the value of "hoodie.parquet.writelegacyformat.enabled"
+    tryOverrideParquetWriteLegacyFormatProperty(opts, convertAvroSchemaToStructType(writerSchema))
+    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, basePath.toString, tblName, opts)
+    val executor = mode match {
+      case SaveMode.Append =>
+        new DatasetBulkInsertActionExecutor(writeConfig, writeClient, instantTime)

Review Comment:
   writeClient is specifically for `RDD[HoodieRecord]`, since all `xxxActionExecutor` here are `Dataset[Row]` based, I didn't put these logic there before.



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 5914b1a1fedea0ae708e5e2f96130c2a73dd5b66 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791) 
   * 1fadedfb975375bba6571e7ecf51de55d7e8dae2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101",
       "triggerID" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * f4cf8eb1001906a9f93677f37c2cb028dd049106 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094) 
   * 02b3132bf72d0d456d495a06501fd051d50b381e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101",
       "triggerID" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b796358f1e324044cf0dfb9afe10cb62e9b132e7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b796358f1e324044cf0dfb9afe10cb62e9b132e7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 02b3132bf72d0d456d495a06501fd051d50b381e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101) 
   * b796358f1e324044cf0dfb9afe10cb62e9b132e7 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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 851a1c3d1bbb317a01822b67ac14ea16ba49fa5e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284) 
   * e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b 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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 03ad0c018d1a929b55d30934d74c9ba84509e88b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 80d97a0abaf04b69d0b0887463794396f0ace49f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479) 
   
   <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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8432800aa63cc5e4d4384f2ade7747aff96bc1c0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506) 
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * f384bbc843028360687903b3b6de835685235b68 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570) 
   
   <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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * be2128221d76e0d0f6c26db5c87122c6efdc726c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481) 
   
   <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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * be2128221d76e0d0f6c26db5c87122c6efdc726c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481) 
   * 6b1fd158b5e0a0102cc5a5312a31a34828b7a06c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503) 
   * 8432800aa63cc5e4d4384f2ade7747aff96bc1c0 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 commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   > @yihua Gentle ping... could you pls help to review it?
   
   Sorry for the delay.  I will review this PR this week.


-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 80d97a0abaf04b69d0b0887463794396f0ace49f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479) 
   * 84f88aaf71b5e7e2040059ac567cf255002cfcb5 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 84f88aaf71b5e7e2040059ac567cf255002cfcb5 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013) 
   * be0af151295a90e2a8878b02f2dec13d6e1b6bbf Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017) 
   
   <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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -850,9 +854,12 @@ object HoodieSparkSqlWriter {
     if (operation != WriteOperationType.DELETE) {
       if (mode == SaveMode.ErrorIfExists && tableExists) {
         throw new HoodieException(s"hoodie table at $tablePath already exists.")
-      } else if (mode == SaveMode.Overwrite && tableExists && operation != WriteOperationType.INSERT_OVERWRITE_TABLE) {
-        // When user set operation as INSERT_OVERWRITE_TABLE,
-        // overwrite will use INSERT_OVERWRITE_TABLE operator in doWriteOperation
+      } else if (mode == SaveMode.Overwrite && tableExists &&

Review Comment:
   Sure, created: [HUDI-6286](https://issues.apache.org/jira/browse/HUDI-6286)



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 03ad0c018d1a929b55d30934d74c9ba84509e88b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300) 
   * 80d97a0abaf04b69d0b0887463794396f0ace49f 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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101",
       "triggerID" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b796358f1e324044cf0dfb9afe10cb62e9b132e7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18161",
       "triggerID" : "b796358f1e324044cf0dfb9afe10cb62e9b132e7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 02b3132bf72d0d456d495a06501fd051d50b381e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101) 
   * b796358f1e324044cf0dfb9afe10cb62e9b132e7 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18161) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

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

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


[GitHub] [hudi] codope commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -599,138 +582,250 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
     }
   }
 
-  test("Test bulk insert") {
+  test("Test bulk insert with insert into for single partitioned table") {
     withSQLConf("hoodie.sql.insert.mode" -> "non-strict") {
       withRecordType()(withTempDir { tmp =>
         Seq("cow", "mor").foreach {tableType =>
-          // Test bulk insert for single partition
-          val tableName = generateTableName
-          spark.sql(
-            s"""
-               |create table $tableName (
-               |  id int,
-               |  name string,
-               |  price double,
-               |  dt string
-               |) using hudi
-               | tblproperties (
-               |  type = '$tableType',
-               |  primaryKey = 'id'
-               | )
-               | partitioned by (dt)
-               | location '${tmp.getCanonicalPath}/$tableName'
-       """.stripMargin)
-          spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false")
+          withTable(generateTableName) { tableName =>
+            spark.sql(
+              s"""
+                 |create table $tableName (
+                 |  id int,
+                 |  name string,
+                 |  price double,
+                 |  dt string
+                 |) using hudi
+                 | tblproperties (
+                 |  type = '$tableType',
+                 |  primaryKey = 'id'
+                 | )
+                 | partitioned by (dt)
+                 | location '${tmp.getCanonicalPath}/$tableName'
+         """.stripMargin)
+            spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false")
 
-          // Enable the bulk insert
-          spark.sql("set hoodie.sql.bulk.insert.enable = true")
-          spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')")
+            // Enable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = true")
+            spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')")
 
-          assertResult(WriteOperationType.BULK_INSERT) {
-            getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            assertResult(WriteOperationType.BULK_INSERT) {
+              getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            }
+            checkAnswer(s"select id, name, price, dt from $tableName")(
+              Seq(1, "a1", 10.0, "2021-07-18")
+            )
+
+            // Disable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = false")
+            spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')")
+
+            assertResult(WriteOperationType.INSERT) {
+              getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            }
+            checkAnswer(s"select id, name, price, dt from $tableName order by id")(
+              Seq(1, "a1", 10.0, "2021-07-18"),
+              Seq(2, "a2", 10.0, "2021-07-18")
+            )
           }
-          checkAnswer(s"select id, name, price, dt from $tableName")(
-            Seq(1, "a1", 10.0, "2021-07-18")
-          )
+        }
+      })
+    }
+  }
 
-          // Disable the bulk insert
-          spark.sql("set hoodie.sql.bulk.insert.enable = false")
-          spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')")
+  test("Test bulk insert with insert into for multi partitioned table") {
+    withSQLConf("hoodie.sql.insert.mode" -> "non-strict") {
+      withRecordType()(withTempDir { tmp =>
+        Seq("cow", "mor").foreach { tableType =>
+          withTable(generateTableName) { tableMultiPartition =>
+            spark.sql(
+              s"""
+                 |create table $tableMultiPartition (
+                 |  id int,
+                 |  name string,
+                 |  price double,
+                 |  dt string,
+                 |  hh string
+                 |) using hudi
+                 | tblproperties (
+                 |  type = '$tableType',
+                 |  primaryKey = 'id'
+                 | )
+                 | partitioned by (dt, hh)
+                 | location '${tmp.getCanonicalPath}/$tableMultiPartition'
+         """.stripMargin)
 
-          assertResult(WriteOperationType.INSERT) {
-            getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            // Enable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = true")
+            spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, '2021-07-18', '12')")
+
+            checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition")(
+              Seq(1, "a1", 10.0, "2021-07-18", "12")
+            )
+            // Disable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = false")
+            spark.sql(s"insert into $tableMultiPartition " +
+              s"values(2, 'a2', 10, '2021-07-18','12')")
+
+            checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition order by id")(
+              Seq(1, "a1", 10.0, "2021-07-18", "12"),
+              Seq(2, "a2", 10.0, "2021-07-18", "12")
+            )
           }
-          checkAnswer(s"select id, name, price, dt from $tableName order by id")(
-            Seq(1, "a1", 10.0, "2021-07-18"),
-            Seq(2, "a2", 10.0, "2021-07-18")
-          )
+        }
+      })
+    }
+  }
 
-          // Test bulk insert for multi-level partition
-          val tableMultiPartition = generateTableName
-          spark.sql(
-            s"""
-               |create table $tableMultiPartition (
-               |  id int,
-               |  name string,
-               |  price double,
-               |  dt string,
-               |  hh string
-               |) using hudi
-               | tblproperties (
-               |  type = '$tableType',
-               |  primaryKey = 'id'
-               | )
-               | partitioned by (dt, hh)
-               | location '${tmp.getCanonicalPath}/$tableMultiPartition'
-       """.stripMargin)
+  test("Test bulk insert with insert into for non partitioned table") {

Review Comment:
   These tests are only testing for default values of `BULKINSERT_OVERWRITE_OPERATION_TYPE` right? Can we also test for the other possible value?



##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala:
##########
@@ -106,8 +106,14 @@ private class HoodieV1WriteBuilder(writeOptions: CaseInsensitiveStringMap,
     override def toInsertableRelation: InsertableRelation = {
       new InsertableRelation {
         override def insert(data: DataFrame, overwrite: Boolean): Unit = {
+          val mode = if (overwriteTable || overwritePartition) {

Review Comment:
   Can you confirm if it's insert_overwrite_table then then table basePath will still be removed?



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertCommitActionExecutor.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieInternalConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.DataSourceInternalWriterHelper;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertCommitActionExecutor extends BaseDatasetBulkInsertCommitActionExecutor {
+
+  public DatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config,
+                                               SparkRDDWriteClient writeClient,
+                                               String instantTime) {
+    super(config, writeClient, instantTime);
+  }
+
+  @Override
+  protected void preExecute() {
+    // no op
+  }
+
+  @Override
+  protected HoodieData<WriteStatus> doExecute(Dataset<Row> records, boolean arePartitionRecordsSorted) {
+    Map<String, String> opts = writeConfig.getProps().entrySet().stream().collect(Collectors.toMap(
+        e -> String.valueOf(e.getKey()),
+        e -> String.valueOf(e.getValue())));
+    Map<String, String> optsOverrides = Collections.singletonMap(
+        HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, String.valueOf(arePartitionRecordsSorted));
+
+    String targetFormat;
+    Map<String, String> customOpts = new HashMap<>(1);
+    if (HoodieSparkUtils.isSpark2()) {
+      targetFormat = "org.apache.hudi.internal";
+    } else if (HoodieSparkUtils.isSpark3()) {
+      targetFormat = "org.apache.hudi.spark3.internal";
+      customOpts.put(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key(), records.schema().json());
+    } else {
+      throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
+          + " To use row writer please switch to spark 2 or spark 3");
+    }
+
+    records.write().format(targetFormat)
+        .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
+        .options(opts)
+        .options(customOpts)
+        .options(optsOverrides)
+        .mode(SaveMode.Append)
+        .save();
+    return null;

Review Comment:
   why return null here?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -850,9 +854,12 @@ object HoodieSparkSqlWriter {
     if (operation != WriteOperationType.DELETE) {
       if (mode == SaveMode.ErrorIfExists && tableExists) {
         throw new HoodieException(s"hoodie table at $tablePath already exists.")
-      } else if (mode == SaveMode.Overwrite && tableExists && operation != WriteOperationType.INSERT_OVERWRITE_TABLE) {
-        // When user set operation as INSERT_OVERWRITE_TABLE,
-        // overwrite will use INSERT_OVERWRITE_TABLE operator in doWriteOperation
+      } else if (mode == SaveMode.Overwrite && tableExists &&

Review Comment:
   Do you mean, for `Overwrite` mode, we should not delete the basePath. Just overwrite the existing data. If so, I agree with you. Probably something to tackle in another PR.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -772,66 +772,71 @@ object HoodieSparkSqlWriter {
     }
   }
 
-  def bulkInsertAsRow(sqlContext: SQLContext,
+  def bulkInsertAsRow(writeClient: SparkRDDWriteClient[_],
+                      parameters: Map[String, String],
                       hoodieConfig: HoodieConfig,
                       df: DataFrame,
+                      mode: SaveMode,
                       tblName: String,
                       basePath: Path,
-                      path: String,
                       instantTime: String,
                       writerSchema: Schema,
-                      isTablePartitioned: Boolean): (Boolean, common.util.Option[String]) = {
+                      tableConfig: HoodieTableConfig):
+  (Boolean, HOption[String], HOption[String], HOption[String], SparkRDDWriteClient[_], HoodieTableConfig) = {
     if (hoodieConfig.getBoolean(INSERT_DROP_DUPS)) {
       throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
     }
+    val sqlContext = writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getSqlContext
+    val jsc = writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getJavaSparkContext
 
     val writerSchemaStr = writerSchema.toString
 
-    val opts = hoodieConfig.getProps.toMap ++
+    // Make opts mutable since it could be modified by tryOverrideParquetWriteLegacyFormatProperty
+    val opts = mutable.Map() ++ hoodieConfig.getProps.toMap ++
       Map(HoodieWriteConfig.AVRO_SCHEMA_STRING.key -> writerSchemaStr)
 
-    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, path, tblName, mapAsJavaMap(opts))
-    val populateMetaFields = hoodieConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)
-
-    val bulkInsertPartitionerRows: BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
-      val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
-      if (userDefinedBulkInsertPartitionerOpt.isPresent) {
-        userDefinedBulkInsertPartitionerOpt.get
-      } else {
-        BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig, isTablePartitioned)
-      }
-    } else {
-      // Sort modes are not yet supported when meta fields are disabled
-      new NonSortPartitionerWithRows()
+    // Auto set the value of "hoodie.parquet.writelegacyformat.enabled"
+    tryOverrideParquetWriteLegacyFormatProperty(opts, convertAvroSchemaToStructType(writerSchema))
+    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, basePath.toString, tblName, opts)
+    val executor = mode match {
+      case SaveMode.Append =>
+        new DatasetBulkInsertCommitActionExecutor(writeConfig, writeClient, instantTime)
+      case SaveMode.Overwrite =>
+        WriteOperationType.fromValue(hoodieConfig
+          .getStringOrDefault(HoodieInternalConfig.BULKINSERT_OVERWRITE_OPERATION_TYPE)) match {
+          case WriteOperationType.INSERT_OVERWRITE =>
+            new DatasetBulkInsertOverwriteCommitActionExecutor(writeConfig, writeClient, instantTime)
+          case WriteOperationType.INSERT_OVERWRITE_TABLE =>
+            new DatasetBulkInsertOverwriteTableCommitActionExecutor(writeConfig, writeClient, instantTime);
+        }
+      case _ =>
+        throw new HoodieException(s"$mode with bulk_insert in row writer path is not supported yet");
     }
 
-    val shouldDropPartitionColumns = hoodieConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS)
-    val hoodieDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(df, writeConfig, bulkInsertPartitionerRows, shouldDropPartitionColumns)
+    val writeResult = executor.execute(df, tableConfig.isTablePartitioned)
 
-    val optsOverrides = Map(
-      HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED ->
-        bulkInsertPartitionerRows.arePartitionRecordsSorted().toString
-    )
+    try {
+      val (writeSuccessful, compactionInstant, clusteringInstant) = mode match {
+        case SaveMode.Append =>
+          val syncHiveSuccess = metaSync(sqlContext.sparkSession, writeConfig, basePath, df.schema)
+          (syncHiveSuccess, HOption.empty().asInstanceOf[HOption[String]], HOption.empty().asInstanceOf[HOption[String]])
+        case SaveMode.Overwrite =>
+          try {
+            commitAndPerformPostOperations(sqlContext.sparkSession, df.schema, writeResult, parameters, writeClient, tableConfig, jsc,
+                TableInstantInfo(basePath, instantTime, executor.getCommitActionType, executor.getWriteOperationType), Option.empty)
 
-    val (targetFormat, customOpts) = if (HoodieSparkUtils.isSpark2) {
-      ("org.apache.hudi.internal", Map())
-    } else if (HoodieSparkUtils.isSpark3) {
-      ("org.apache.hudi.spark3.internal", Map(
-        HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key -> hoodieDF.schema.json
-      ))
-    } else {
-      throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
-        + " To use row writer please switch to spark 2 or spark 3")
+          }
+      }
+      (writeSuccessful, common.util.Option.ofNullable(instantTime), compactionInstant, clusteringInstant, writeClient, tableConfig)

Review Comment:
   use `HOption.ofNullable`?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java:
##########
@@ -77,8 +78,11 @@ public HoodieData<WriteStatus> performClusteringWithRecordsAsRow(Dataset<Row> in
     // Since clustering will write to single file group using HoodieUnboundedCreateHandle, set max file size to a large value.
     newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(Long.MAX_VALUE));
 
-    return HoodieDatasetBulkInsertHelper.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig,
-        getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata);
+    BulkInsertPartitioner<Dataset<Row>> partitioner = getRowPartitioner(strategyParams, schema);
+    Dataset<Row> repartitionedRecords = partitioner.repartitionRecords(inputRecords, numOutputGroups);

Review Comment:
   Got it. Thanks for the explanation.



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkInsertCommitActionExecutor.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.HoodieDatasetBulkInsertHelper;
+import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseDatasetBulkInsertCommitActionExecutor implements Serializable {
+
+  protected final transient HoodieWriteConfig writeConfig;
+  protected final transient SparkRDDWriteClient writeClient;
+  protected final String instantTime;
+  protected HoodieTable table;
+
+  public BaseDatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config,
+                                                   SparkRDDWriteClient writeClient,
+                                                   String instantTime) {
+    this.writeConfig = config;
+    this.writeClient = writeClient;
+    this.instantTime = instantTime;
+  }
+
+  protected void preExecute() {
+    table.validateInsertSchema();
+    writeClient.startCommitWithTime(instantTime, getCommitActionType());
+    writeClient.preWrite(instantTime, getWriteOperationType(), table.getMetaClient());
+  }
+
+  protected abstract HoodieData<WriteStatus> doExecute(Dataset<Row> records, boolean arePartitionRecordsSorted);
+
+  protected void afterExecute(HoodieWriteMetadata<JavaRDD<WriteStatus>> result) {
+    writeClient.postWrite(result, instantTime, table);
+  }
+
+  private HoodieWriteMetadata<JavaRDD<WriteStatus>> buildHoodieWriteMetadata(HoodieData<WriteStatus> writeStatuses) {
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> hoodieWriteMetadata = new HoodieWriteMetadata<>();
+    if (writeStatuses != null) {
+      hoodieWriteMetadata.setWriteStatuses(HoodieJavaRDD.getJavaRDD(writeStatuses));
+      hoodieWriteMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeStatuses));
+    }
+    return hoodieWriteMetadata;
+  }
+
+  public final HoodieWriteResult execute(Dataset<Row> records, boolean isTablePartitioned) {
+    if (writeConfig.getBoolean(DataSourceWriteOptions.INSERT_DROP_DUPS())) {
+      throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet");
+    }
+
+    boolean populateMetaFields = writeConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS);
+
+    BulkInsertPartitioner<Dataset<Row>> bulkInsertPartitionerRows = getPartitioner(populateMetaFields, isTablePartitioned);
+    boolean shouldDropPartitionColumns = writeConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS());
+    Dataset<Row> hoodieDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(records, writeConfig, bulkInsertPartitionerRows, shouldDropPartitionColumns);
+
+    table = writeClient.initTable(WriteOperationType.INSERT_OVERWRITE, Option.ofNullable(instantTime));

Review Comment:
   Will the operation type not depend on the specific implementation of this base class? I don't see `execute` being overridden in subclasses.



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 851a1c3d1bbb317a01822b67ac14ea16ba49fa5e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284) 
   
   <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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java:
##########
@@ -77,8 +78,11 @@ public HoodieData<WriteStatus> performClusteringWithRecordsAsRow(Dataset<Row> in
     // Since clustering will write to single file group using HoodieUnboundedCreateHandle, set max file size to a large value.
     newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(Long.MAX_VALUE));
 
-    return HoodieDatasetBulkInsertHelper.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig,
-        getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata);
+    BulkInsertPartitioner<Dataset<Row>> partitioner = getRowPartitioner(strategyParams, schema);
+    Dataset<Row> repartitionedRecords = partitioner.repartitionRecords(inputRecords, numOutputGroups);

Review Comment:
   Yes, I changed `HoodieDatasetBulkInsertHelper.bulkInsert`, deleting repartition in that method. so adding repartition here and `SparkSortAndSizeExecutionStrategy`
   
   ``` scala
   // deleted
       val repartitionedDataset = partitioner.repartitionRecords(dataset, parallelism)
       val arePartitionRecordsSorted = partitioner.arePartitionRecordsSorted
   ```
   
   `BaseDatasetBulkCommitActionExecutor` would first call `HoodieDatasetBulkInsertHelper.prepareForBulkInsert`, and then call `HoodieDatasetBulkInsertHelper.bulkInsert` to perform write operation. And in `HoodieDatasetBulkInsertHelper.prepareForBulkInsert`, it would do the repartition work, so I delete the repartition in `HoodieDatasetBulkInsertHelper.bulkInsert`. And it looks more reasonable, `HoodieDatasetBulkInsertHelper.bulkInsert` should only perform the write operation, it shouldn't change the records layout.



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertActionExecutor.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieInternalConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.DataSourceInternalWriterHelper;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertActionExecutor extends BaseDatasetBulkCommitActionExecutor {

Review Comment:
   make sense, will change



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkCommitActionExecutor.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.commit;
+
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.HoodieDatasetBulkInsertHelper;
+import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseDatasetBulkCommitActionExecutor implements Serializable {

Review Comment:
   Yes, at first I tried to put this in `hudi-client-common`, but since `BaseDatasetBulkCommitActionExecutor` needs to access `DataSourceUtils` and `DataSourceWriteOptions`, not sure it's reasonable to move these classes there, and I'm afraid there are other dependencies for these two classes, we may also need to change those dependencies.



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 9fa7e9923c01a635744d23314bd7bafce83aee76 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591) 
   * 8becde27e89beaa60930f6ddff5d681971092836 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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 8becde27e89beaa60930f6ddff5d681971092836 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733) 
   * 5914b1a1fedea0ae708e5e2f96130c2a73dd5b66 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 851a1c3d1bbb317a01822b67ac14ea16ba49fa5e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284) 
   * e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290) 
   
   <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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -599,138 +582,250 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
     }
   }
 
-  test("Test bulk insert") {
+  test("Test bulk insert with insert into for single partitioned table") {
     withSQLConf("hoodie.sql.insert.mode" -> "non-strict") {
       withRecordType()(withTempDir { tmp =>
         Seq("cow", "mor").foreach {tableType =>
-          // Test bulk insert for single partition
-          val tableName = generateTableName
-          spark.sql(
-            s"""
-               |create table $tableName (
-               |  id int,
-               |  name string,
-               |  price double,
-               |  dt string
-               |) using hudi
-               | tblproperties (
-               |  type = '$tableType',
-               |  primaryKey = 'id'
-               | )
-               | partitioned by (dt)
-               | location '${tmp.getCanonicalPath}/$tableName'
-       """.stripMargin)
-          spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false")
+          withTable(generateTableName) { tableName =>
+            spark.sql(
+              s"""
+                 |create table $tableName (
+                 |  id int,
+                 |  name string,
+                 |  price double,
+                 |  dt string
+                 |) using hudi
+                 | tblproperties (
+                 |  type = '$tableType',
+                 |  primaryKey = 'id'
+                 | )
+                 | partitioned by (dt)
+                 | location '${tmp.getCanonicalPath}/$tableName'
+         """.stripMargin)
+            spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false")
 
-          // Enable the bulk insert
-          spark.sql("set hoodie.sql.bulk.insert.enable = true")
-          spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')")
+            // Enable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = true")
+            spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')")
 
-          assertResult(WriteOperationType.BULK_INSERT) {
-            getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            assertResult(WriteOperationType.BULK_INSERT) {
+              getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            }
+            checkAnswer(s"select id, name, price, dt from $tableName")(
+              Seq(1, "a1", 10.0, "2021-07-18")
+            )
+
+            // Disable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = false")
+            spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')")
+
+            assertResult(WriteOperationType.INSERT) {
+              getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            }
+            checkAnswer(s"select id, name, price, dt from $tableName order by id")(
+              Seq(1, "a1", 10.0, "2021-07-18"),
+              Seq(2, "a2", 10.0, "2021-07-18")
+            )
           }
-          checkAnswer(s"select id, name, price, dt from $tableName")(
-            Seq(1, "a1", 10.0, "2021-07-18")
-          )
+        }
+      })
+    }
+  }
 
-          // Disable the bulk insert
-          spark.sql("set hoodie.sql.bulk.insert.enable = false")
-          spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')")
+  test("Test bulk insert with insert into for multi partitioned table") {
+    withSQLConf("hoodie.sql.insert.mode" -> "non-strict") {
+      withRecordType()(withTempDir { tmp =>
+        Seq("cow", "mor").foreach { tableType =>
+          withTable(generateTableName) { tableMultiPartition =>
+            spark.sql(
+              s"""
+                 |create table $tableMultiPartition (
+                 |  id int,
+                 |  name string,
+                 |  price double,
+                 |  dt string,
+                 |  hh string
+                 |) using hudi
+                 | tblproperties (
+                 |  type = '$tableType',
+                 |  primaryKey = 'id'
+                 | )
+                 | partitioned by (dt, hh)
+                 | location '${tmp.getCanonicalPath}/$tableMultiPartition'
+         """.stripMargin)
 
-          assertResult(WriteOperationType.INSERT) {
-            getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            // Enable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = true")
+            spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, '2021-07-18', '12')")
+
+            checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition")(
+              Seq(1, "a1", 10.0, "2021-07-18", "12")
+            )
+            // Disable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = false")
+            spark.sql(s"insert into $tableMultiPartition " +
+              s"values(2, 'a2', 10, '2021-07-18','12')")
+
+            checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition order by id")(
+              Seq(1, "a1", 10.0, "2021-07-18", "12"),
+              Seq(2, "a2", 10.0, "2021-07-18", "12")
+            )
           }
-          checkAnswer(s"select id, name, price, dt from $tableName order by id")(
-            Seq(1, "a1", 10.0, "2021-07-18"),
-            Seq(2, "a2", 10.0, "2021-07-18")
-          )
+        }
+      })
+    }
+  }
 
-          // Test bulk insert for multi-level partition
-          val tableMultiPartition = generateTableName
-          spark.sql(
-            s"""
-               |create table $tableMultiPartition (
-               |  id int,
-               |  name string,
-               |  price double,
-               |  dt string,
-               |  hh string
-               |) using hudi
-               | tblproperties (
-               |  type = '$tableType',
-               |  primaryKey = 'id'
-               | )
-               | partitioned by (dt, hh)
-               | location '${tmp.getCanonicalPath}/$tableMultiPartition'
-       """.stripMargin)
+  test("Test bulk insert with insert into for non partitioned table") {

Review Comment:
   `Test bulk insert with insert overwrite table` test `INSERT_OVERWRITE_TABLE`,
   
   `Test bulk insert with insert overwrite partition` test `INSERT_OVERWRITE`
   
   These two tests test default values of `BULKINSERT_OVERWRITE_OPERATION_TYPE`



-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 5914b1a1fedea0ae708e5e2f96130c2a73dd5b66 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791) 
   * 1fadedfb975375bba6571e7ecf51de55d7e8dae2 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 #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * be2128221d76e0d0f6c26db5c87122c6efdc726c 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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   Gentle ping @alexeykudinkin @xushiyan @danny0405 @yihua 


-- 
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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   > Thanks @boneanxs for extracting out the breaking change. Left one minor comment for the config. Can you also squash all commits to one?
   
   @codope Thanks for reviewing, address the comment and squashed all commits.


-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * be0af151295a90e2a8878b02f2dec13d6e1b6bbf Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017) 
   
   <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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * be0af151295a90e2a8878b02f2dec13d6e1b6bbf Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017) 
   * 4d8ba7c81f123af23dca05642c5562ac79eed4ff 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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101",
       "triggerID" : "02b3132bf72d0d456d495a06501fd051d50b381e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 02b3132bf72d0d456d495a06501fd051d50b381e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18101) 
   
   <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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   @yihua Gentle ping... could you pls help to review it?


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

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

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


[GitHub] [hudi] hudi-bot commented on pull request #8076: Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * f384bbc843028360687903b3b6de835685235b68 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570) 
   * 9fa7e9923c01a635744d23314bd7bafce83aee76 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591) 
   
   <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] boneanxs commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   >  I am yet to review fully, but have taken one pass. Can you break it down into two PRs - a) don't delete the table location if using SaveMode.Overwrite for bulk_insert, insert_overwrite, b) add support for bulk_insert for insert_overwrite and insert_overwrite_table.
   
   Yea, sure, will do so
   
   > Also, I want to understand the use case when we need this. If you can elaborate a bit more on why we need this, that would be great.
   
   Currently, we want to migrate all existing hive tables to HUDI table, given many hive tables
      1) usually perform `insert_overwrite` operation to overwrite the partition 
      2) written by batch jobs, could contains TB level data one day 
      3) doesn't need to perform the `tag`, `drop duplicates`
   
   `bulk_insert` mode fit such scenario well, we can use `bulk_insert` mode to boost the write performance and make users easier to migrate existing hive table to hudi table.


-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298) 
   
   <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] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala:
##########
@@ -106,8 +106,14 @@ private class HoodieV1WriteBuilder(writeOptions: CaseInsensitiveStringMap,
     override def toInsertableRelation: InsertableRelation = {
       new InsertableRelation {
         override def insert(data: DataFrame, overwrite: Boolean): Unit = {
+          val mode = if (overwriteTable || overwritePartition) {

Review Comment:
   WIth this pr, it won't delete the basePath.
   
   ```scala
   // HoodieSparkSqlWriter#handleSaveModes
   // won't delete the path if it's Overwrite mode and INSERT_OVERWRITE_TABLE, INSERT_OVERWRITE
   
   else if (mode == SaveMode.Overwrite && tableExists &&
           (operation != WriteOperationType.INSERT_OVERWRITE_TABLE
             && operation != WriteOperationType.INSERT_OVERWRITE
             && operation != WriteOperationType.BULK_INSERT)) {
           // For INSERT_OVERWRITE_TABLE, INSERT_OVERWRITE and BULK_INSERT with Overwrite mode,
           // we'll use replacecommit to overwrite the old data.
           log.warn(s"hoodie table at $tablePath already exists. Deleting existing data & overwriting with new data.")
           fs.delete(tablePath, true)
           tableExists = false
         }
   ```



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

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

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


[GitHub] [hudi] boneanxs commented on a diff in pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSourceDAGExecution.scala:
##########
@@ -89,7 +89,7 @@ class TestSparkDataSourceDAGExecution extends HoodieSparkClientTestBase with Sca
   @CsvSource(Array(
     "upsert,org.apache.hudi.client.SparkRDDWriteClient.commit",
     "insert,org.apache.hudi.client.SparkRDDWriteClient.commit",
-    "bulk_insert,org.apache.hudi.HoodieSparkSqlWriter$.bulkInsertAsRow"))
+    "bulk_insert,org.apache.hudi.HoodieDatasetBulkInsertHelper.bulkInsert"))

Review Comment:
   `bulk_insert` with `Overwrite` mode will change to use `DatasetBulkInsertOverwriteTableCommitActionExecutor` to avoid deleting basePath first. So it would call `FSUtils.getAllPartitionPaths` to get all existing partition paths, which would introduce extra stages containing `HoodieSparkSqlWriter$.bulkInsertAsRow`, but it won't call write operation multi times, can see from the screenshot.
   
   ![Screenshot 2023-05-23 at 14 45 58](https://github.com/apache/hudi/assets/10115332/d2e0cf3b-65ed-45a3-b786-0778f9b8e395)
   
   So I change here to check the write method `HoodieDatasetBulkInsertHelper.bulkInsert`.
   



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

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

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


[GitHub] [hudi] yihua commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   @boneanxs meanwhile, could you rebase the PR on the latest master?


-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 80d97a0abaf04b69d0b0887463794396f0ace49f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479) 
   * 84f88aaf71b5e7e2040059ac567cf255002cfcb5 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013) 
   * be0af151295a90e2a8878b02f2dec13d6e1b6bbf 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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16674",
       "triggerID" : "1fadedfb975375bba6571e7ecf51de55d7e8dae2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17290",
       "triggerID" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e1ba1b4dc8eaa3dcce229a6e9f47fc57e363775b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17298",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "851a1c3d1bbb317a01822b67ac14ea16ba49fa5e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17284",
       "triggerID" : "1559060170",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17300",
       "triggerID" : "03ad0c018d1a929b55d30934d74c9ba84509e88b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17479",
       "triggerID" : "80d97a0abaf04b69d0b0887463794396f0ace49f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18013",
       "triggerID" : "84f88aaf71b5e7e2040059ac567cf255002cfcb5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18017",
       "triggerID" : "be0af151295a90e2a8878b02f2dec13d6e1b6bbf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089",
       "triggerID" : "4d8ba7c81f123af23dca05642c5562ac79eed4ff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094",
       "triggerID" : "f4cf8eb1001906a9f93677f37c2cb028dd049106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 4d8ba7c81f123af23dca05642c5562ac79eed4ff Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18089) 
   * f4cf8eb1001906a9f93677f37c2cb028dd049106 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=18094) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

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

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


[GitHub] [hudi] codope commented on pull request #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   Looks good to me. @yihua @nsivabalan If you can take one pass, that would be great.


-- 
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 #8076: [HUDI-5884] Support bulk_insert for insert_overwrite and insert_overwrite_table

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15479",
       "triggerID" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be2128221d76e0d0f6c26db5c87122c6efdc726c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15481",
       "triggerID" : "1447986557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15503",
       "triggerID" : "6b1fd158b5e0a0102cc5a5312a31a34828b7a06c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15506",
       "triggerID" : "8432800aa63cc5e4d4384f2ade7747aff96bc1c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a239ada8998fd440f19c0082b26d206ed589870",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f384bbc843028360687903b3b6de835685235b68",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15570",
       "triggerID" : "f384bbc843028360687903b3b6de835685235b68",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15591",
       "triggerID" : "9fa7e9923c01a635744d23314bd7bafce83aee76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8becde27e89beaa60930f6ddff5d681971092836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15733",
       "triggerID" : "8becde27e89beaa60930f6ddff5d681971092836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791",
       "triggerID" : "5914b1a1fedea0ae708e5e2f96130c2a73dd5b66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a239ada8998fd440f19c0082b26d206ed589870 UNKNOWN
   * 5914b1a1fedea0ae708e5e2f96130c2a73dd5b66 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15791) 
   
   <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