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

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

Reply via email to