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

2023-06-28 Thread via GitHub


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 
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-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@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

2023-05-30 Thread via GitHub


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 doExecute(Dataset records, boolean 
arePartitionRecordsSorted) {
+Map opts = 
writeConfig.getProps().entrySet().stream().collect(Collectors.toMap(
+e -> String.valueOf(e.getKey()),
+e -> String.valueOf(e.getValue(;
+Map optsOverrides = Collections.singletonMap(
+HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, 
String.valueOf(arePartitionRecordsSorted));
+
+String targetFormat;
+Map 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>` 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-unsubscr...@hudi.apache.org

For queries about this service, please contact 

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

2023-05-30 Thread via GitHub


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-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@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

2023-05-17 Thread via GitHub


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 

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

2023-04-01 Thread via GitHub


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 
performClusteringWithRecordsAsRow(Dataset 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> partitioner = 
getRowPartitioner(strategyParams, schema);
+Dataset 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 
performClusteringWithRecordsAsRow(Dataset 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> partitioner =