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

[GitHub] [hudi] danny0405 commented on a diff in pull request #8107: [HUDI-5514] Adding auto generation of record keys support to Hudi/Spark

danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1155456183


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +86,19 @@ object HoodieDatasetBulkInsertHelper
           val keyGenerator =
             ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps))
               .asInstanceOf[SparkKeyGeneratorInterface]
+          val partitionId = TaskContext.getPartitionId()
+          var rowId = 0
 
           iter.map { row =>
-            val recordKey = keyGenerator.getRecordKey(row, schema)
+            // auto generate record keys if needed
+            val recordKey = if (autoGenerateRecordKeys) {
+              val recKey = HoodieRecord.generateSequenceId(instantTime, partitionId, rowId)
+              rowId += 1
+              UTF8String.fromString(recKey)
+            }
+            else { // else use key generator to fetch record key
+              keyGenerator.getRecordKey(row, schema)

Review Comment:
   Why we generate a globally unique key for bulk_insert while for normal ingestion we use an empty string?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java:
##########
@@ -32,7 +34,9 @@ public class ComplexAvroKeyGenerator extends BaseKeyGenerator {
 
   public ComplexAvroKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(","))
+    this.setAutoGenerateRecordKeys(!props.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()));
+    this.recordKeyFields = autoGenerateRecordKeys() ? Collections.emptyList() :

Review Comment:
   Can we move all the logic
   
   ```java
   setAutoGenerateRecordKeys(!props.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()));
   ```
   into the base class `BaseKeyGenerator` constructor.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java:
##########
@@ -44,6 +48,9 @@ public ComplexAvroKeyGenerator(TypedProperties props) {
 
   @Override
   public String getRecordKey(GenericRecord record) {
+    if (autoGenerateRecordKeys()) {
+      return StringUtils.EMPTY_STRING;
+    }

Review Comment:
   Is it safe to use the empty string, the payloads can be merged while going through the merging procedure.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -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
+
+import org.apache.avro.generic.GenericRecord
+import org.apache.hudi.DataSourceWriteOptions.INSERT_DROP_DUPS
+import org.apache.hudi.common.config.HoodieConfig
+import org.apache.hudi.common.model.{HoodieRecord, WriteOperationType}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.exception.HoodieKeyGeneratorException
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions
+
+object AutoRecordKeyGenerationUtils {
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String], hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys = !parameters.contains(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) // if record key is not configured,
+    // hudi will auto generate.
+
+    if (autoGenerateRecordKeys) {
+      // de-dup is not supported with auto generation of record keys
+      if (parameters.getOrElse(HoodieWriteConfig.COMBINE_BEFORE_INSERT.key(),
+        HoodieWriteConfig.COMBINE_BEFORE_INSERT.defaultValue()).toBoolean) {
+        throw new HoodieKeyGeneratorException("Enabling " + HoodieWriteConfig.COMBINE_BEFORE_INSERT.key() + " is not supported with auto generation of record keys ")
+      }
+      // drop dupes is not supported
+      if (hoodieConfig.getBoolean(INSERT_DROP_DUPS)) {
+        throw new HoodieKeyGeneratorException("Enabling " + INSERT_DROP_DUPS.key() + " is not supported with auto generation of record keys ")
+      }
+      // virtual keys are not supported with auto generation of record keys.
+      if (!parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue().toString).toBoolean) {
+        throw new HoodieKeyGeneratorException("Disabling " + HoodieTableConfig.POPULATE_META_FIELDS.key() + " is not supported with auto generation of record keys")
+      }
+    }
+  }
+
+  /**
+   * Auto Generate record keys when auto generation config is enabled.
+   * <ol>
+   *   <li>Generated keys will be unique not only w/in provided [[org.apache.spark.sql.DataFrame]], but
+   *   globally unique w/in the target table</li>
+   *   <li>Generated keys have minimal overhead (to compute, persist and read)</li>
+   * </ol>
+   *
+   * Keys adhere to the following format:
+   *
+   * [instantTime]_[PartitionId]_[RowId]
+   *
+   * where
+   * instantTime refers to the commit time of the batch being ingested.
+   * PartitionId refers to spark's partition Id.
+   * RowId refers to the row index within the spark partition.
+   *
+   * @param autoGenerateKeys true if auto generation of record keys is enabled. false otherwise.
+   * @param genRecsItr Iterator of GenericRecords.
+   * @param instantTime commit time of the batch.
+   * @param sparkPartitionId spark partition Id of interest.
+   * @return Iterator of Pair of GenericRecord and Optionally generated record key.
+   */
+  def mayBeAutoGenerateRecordKeys(autoGenerateKeys : Boolean, genRecsItr: Iterator[GenericRecord], instantTime: String,
+                                  sparkPartitionId: Integer): Iterator[(GenericRecord, Option[String])] = {
+    var rowId = 0
+    // we will override record keys if auto generation if keys is enabled.
+    genRecsItr.map(avroRecord =>
+      if (autoGenerateKeys) {
+        val recordKey : String = HoodieRecord.generateSequenceId(instantTime, sparkPartitionId, rowId)
+        rowId += 1
+        (avroRecord, Some(recordKey))
+      } else {
+        (avroRecord, Option.empty)

Review Comment:
   Move the `autoGenerateKeys` out of the map loop to make it more efficient.



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