alexeykudinkin commented on code in PR #5470:
URL: https://github.com/apache/hudi/pull/5470#discussion_r929165914


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java:
##########
@@ -66,12 +66,26 @@ protected BuiltinKeyGenerator(TypedProperties config) {
   @Override
   @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
   public String getRecordKey(Row row) {
+    // TODO avoid conversion to avro
+    //      since converterFn is transient this will be repeatedly initialized 
over and over again
     if (null == converterFn) {
       converterFn = AvroConversionUtils.createConverterToAvro(row.schema(), 
STRUCT_NAME, NAMESPACE);
     }
     return getKey(converterFn.apply(row)).getRecordKey();
   }
 
+  @Override
+  @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
+  public String getRecordKey(InternalRow internalRow, StructType schema) {
+    try {

Review Comment:
   Correct. These are revisited



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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
+
+import org.apache.hudi.client.model.HoodieInternalRow
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.util.ReflectionUtils
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.keygen.BuiltinKeyGenerator
+import org.apache.hudi.table.BulkInsertPartitioner
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.HoodieUnsafeRDDUtils.createDataFrame
+import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, 
getNestedInternalRowValue}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types.{StringType, StructField, StructType}
+import org.apache.spark.sql.{DataFrame, Dataset, HoodieUnsafeRDDUtils, Row}
+import org.apache.spark.unsafe.types.UTF8String
+
+import scala.collection.JavaConverters.asScalaBufferConverter
+
+object HoodieDatasetBulkInsertHelper extends Logging {

Review Comment:
   To avoid back-n-forth Java/Scala conversions



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java:
##########
@@ -73,18 +75,15 @@ public WriteSupport.FinalizedWriteContext finalizeWrite() {
     return new WriteSupport.FinalizedWriteContext(extraMetaData);
   }
 
-  public void add(String recordKey) {
-    this.bloomFilter.add(recordKey);
-    if (minRecordKey != null) {
-      minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : 
recordKey;
-    } else {
-      minRecordKey = recordKey;
+  public void add(UTF8String recordKey) {
+    this.bloomFilter.add(recordKey.getBytes());

Review Comment:
   Correct



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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
+
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.util.ReflectionUtils
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.keygen.BuiltinKeyGenerator
+import org.apache.hudi.table.BulkInsertPartitioner
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.HoodieUnsafeRDDUtils.createDataFrame
+import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, 
getNestedInternalRowValue}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.types.{StringType, StructField, StructType}
+import org.apache.spark.sql.{DataFrame, Dataset, HoodieUnsafeRDDUtils, Row}
+import org.apache.spark.unsafe.types.UTF8String
+
+import scala.collection.JavaConverters.asScalaBufferConverter
+
+object HoodieDatasetBulkInsertHelper extends Logging {
+
+  /**
+   * Prepares [[DataFrame]] for bulk-insert into Hudi table, taking following 
steps:
+   *
+   * <ol>
+   *   <li>Invoking configured [[KeyGenerator]] to produce record key, alas 
partition-path value</li>
+   *   <li>Prepends Hudi meta-fields to every row in the dataset</li>
+   *   <li>Dedupes rows (if necessary)</li>
+   *   <li>Partitions dataset using provided [[partitioner]]</li>
+   * </ol>
+   */
+  def prepareForBulkInsert(df: DataFrame,
+                           config: HoodieWriteConfig,
+                           partitioner: BulkInsertPartitioner[Dataset[Row]],
+                           isGlobalIndex: Boolean,
+                           dropPartitionColumns: Boolean): Dataset[Row] = {
+    val populateMetaFields = config.populateMetaFields()
+    val schema = df.schema
+
+    val keyGeneratorClassName = 
config.getStringOrThrow(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME,
+      "Key-generator class name is required")
+
+    val prependedRdd: RDD[InternalRow] =
+      df.queryExecution.toRdd.mapPartitions { iter =>

Review Comment:
   I think issue @nsivabalan is referring to is the fact that this PR shouldn't 
be measured in isolation but only together with #5523 (which is landed as well)



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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
+
+import org.apache.hudi.client.model.HoodieInternalRow
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.util.ReflectionUtils
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.keygen.BuiltinKeyGenerator
+import org.apache.hudi.table.BulkInsertPartitioner
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.HoodieUnsafeRDDUtils.createDataFrame
+import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, 
getNestedInternalRowValue}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types.{StringType, StructField, StructType}
+import org.apache.spark.sql.{DataFrame, Dataset, HoodieUnsafeRDDUtils, Row}
+import org.apache.spark.unsafe.types.UTF8String
+
+import scala.collection.JavaConverters.asScalaBufferConverter
+
+object HoodieDatasetBulkInsertHelper extends Logging {
+
+  /**
+   * Prepares [[DataFrame]] for bulk-insert into Hudi table, taking following 
steps:
+   *
+   * <ol>
+   *   <li>Invoking configured [[KeyGenerator]] to produce record key, alas 
partition-path value</li>
+   *   <li>Prepends Hudi meta-fields to every row in the dataset</li>
+   *   <li>Dedupes rows (if necessary)</li>
+   *   <li>Partitions dataset using provided [[partitioner]]</li>
+   * </ol>
+   */
+  def prepareForBulkInsert(df: DataFrame,
+                           config: HoodieWriteConfig,
+                           partitioner: BulkInsertPartitioner[Dataset[Row]],
+                           isGlobalIndex: Boolean,
+                           dropPartitionColumns: Boolean): Dataset[Row] = {
+    val populateMetaFields = config.populateMetaFields()
+    val schema = df.schema
+
+    val keyGeneratorClassName = 
config.getStringOrThrow(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME,
+      "Key-generator class name is required")
+
+    val prependedRdd: RDD[InternalRow] =
+      df.queryExecution.toRdd.mapPartitions { iter =>
+        val keyGenerator =
+          ReflectionUtils.loadClass(keyGeneratorClassName, new 
TypedProperties(config.getProps))
+            .asInstanceOf[BuiltinKeyGenerator]
+
+        iter.map { row =>
+          val (recordKey, partitionPath) =
+            if (populateMetaFields) {
+              (UTF8String.fromString(keyGenerator.getRecordKey(row, schema)),
+                UTF8String.fromString(keyGenerator.getPartitionPath(row, 
schema)))
+            } else {
+              (UTF8String.EMPTY_UTF8, UTF8String.EMPTY_UTF8)
+            }
+          val commitTimestamp = UTF8String.EMPTY_UTF8
+          val commitSeqNo = UTF8String.EMPTY_UTF8
+          val filename = UTF8String.EMPTY_UTF8
+
+          // TODO use mutable row, avoid re-allocating
+          new HoodieInternalRow(commitTimestamp, commitSeqNo, recordKey, 
partitionPath, filename, row, false)
+        }
+      }
+
+    val metaFields = Seq(
+      StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, StringType),
+      StructField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, StringType),
+      StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, StringType),
+      StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, StringType),
+      StructField(HoodieRecord.FILENAME_METADATA_FIELD, StringType))
+
+    val updatedSchema = StructType(metaFields ++ schema.fields)
+    val updatedDF = HoodieUnsafeRDDUtils.createDataFrame(df.sparkSession, 
prependedRdd, updatedSchema)
+
+    if (!populateMetaFields) {
+      updatedDF
+    } else {
+      val trimmedDF = if (dropPartitionColumns) {
+        val keyGenerator = ReflectionUtils.loadClass(keyGeneratorClassName, 
new TypedProperties(config.getProps)).asInstanceOf[BuiltinKeyGenerator]
+        val partitionPathFields = keyGenerator.getPartitionPathFields.asScala
+        val nestedPartitionPathFields = partitionPathFields.filter(f => 
f.contains('.'))
+        if (nestedPartitionPathFields.nonEmpty) {
+          logWarning(s"Can not drop nested partition path fields: 
$nestedPartitionPathFields")
+        }
+
+        val partitionPathCols = partitionPathFields -- 
nestedPartitionPathFields
+        updatedDF.drop(partitionPathCols: _*)
+      } else {
+        updatedDF
+      }
+
+      val dedupedDF = if (config.shouldCombineBeforeInsert) {
+        dedupeRows(trimmedDF, config.getPreCombineField, isGlobalIndex)
+      } else {
+        trimmedDF
+      }
+
+      partitioner.repartitionRecords(dedupedDF, 
config.getBulkInsertShuffleParallelism)
+    }
+  }
+
+  private def dedupeRows(df: DataFrame, preCombineFieldRef: String, 
isGlobalIndex: Boolean): DataFrame = {
+    val recordKeyMetaFieldOrd = 
df.schema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD)
+    val partitionPathMetaFieldOrd = 
df.schema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD)
+    // NOTE: Pre-combine field could be a nested field
+    val preCombineFieldPath = composeNestedFieldPath(df.schema, 
preCombineFieldRef)
+
+    val dedupedRdd = df.queryExecution.toRdd
+      .map { row =>
+        val rowKey = if (isGlobalIndex) {
+          row.getString(recordKeyMetaFieldOrd)
+        } else {
+          val partitionPath = row.getString(partitionPathMetaFieldOrd)
+          val recordKey = row.getString(recordKeyMetaFieldOrd)
+          s"$partitionPath:$recordKey"
+        }
+        // NOTE: It's critical whenever we keep the reference to the row, to 
make a copy
+        //       since Spark might be providing us with a mutable copy 
(updated during the iteration)
+        (rowKey, row.copy())

Review Comment:
   This exact code will fail if we remove copy, b/c often `InternalRow` is a 
mutable copy that Spark changes during iteration, which is safe when we access 
just the one under the pointer, but in the subsequent `reduceByKey` we access 2 
rows at the same time



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java:
##########
@@ -18,26 +18,24 @@
 
 package org.apache.hudi.keygen;
 
+import org.apache.avro.generic.GenericRecord;
 import org.apache.hudi.ApiMaturityLevel;
 import org.apache.hudi.AvroConversionUtils;
 import org.apache.hudi.PublicAPIMethod;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.exception.HoodieIOException;
-
-import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.exception.HoodieException;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.types.DataType;
 import org.apache.spark.sql.types.StructType;
+import scala.Function1;

Review Comment:
   This is removed in #5523



-- 
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