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

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

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