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

[GitHub] [hudi] nsivabalan opened a new pull request, #8107: [HUDI-5514] Adding auto generation of record keys support to Hudi

nsivabalan opened a new pull request, #8107:
URL: https://github.com/apache/hudi/pull/8107

   ### Change Logs
   
   This PR is introducing capability for Record Key Auto Generation
   
   ### Background
   
   At present ingesting data into Hudi has a few unavoidable prerequisites one of which is specifying record key configuration (with record key serving as primary key). Necessity to specify primary key is one of the core assumptions built into Hudi model centered around being able to update the target table efficiently.
   
   However, some payloads actually don't have a naturally present record key: for ex, when ingesting some kind of "logs" into Hudi there might be no unique identifier held in every record that could serve the purpose of being record key, while meeting global uniqueness requirements of the primary key. Nevertheless, we want to make sure that Hudi is able to support such payloads while still providing for Hudi's core strength
   
   #### Requirements
   1. Auto-generated record keys have to provide for global uniqueness w/in the table, not just w/in the batch. 
   This is necessary to make sure we're able to support updating such tables.
   2. Keys should be generated in a way that would allow for their efficient compression
   This is necessary to make sure that auto-generated keys are not bringing substantial overhead (on storage and in handling)
   3. Auto generation of record keys should be robust against partial failures and retries like task and stage failures and retries. In other words, such events should not result in data duplication or data loss. 
   
   #### Implementation
   To support payloads with no naturally present record key, here we're proposing to enable new mode of operation for Hudi where synthetic, globally unique (w/in the table) record key will be injected upon persistence of the dataset as Hudi table.
   
   For achieving our goal of providing globally unique keys we're planning on relying on the following synthetic key format comprised of 2 components
   
    - (Reserved) Commit timestamp: Use reserved commit timestamp as prefix (to provide for global uniqueness of rows)
    - Engine's task partitionId or parallelizable unit for the engine of interest. (Spark PartitionId incase of spark engine)
    - Row id: unique identifier of the row (record) w/in the provided task partition. 
   
   Combining them in a single string key as below
   
   ```"${commit_timestamp}_${partition_id}_${row_id}"```
   
   For row-id generation we're planning on using generator very similar in spirit to `monotonically_increasing_id()` expression from Spark to generate unique identity value for every row w/in batch (could be easily implemented for any parallel execution framework like Flink, etc)
   
   Please note, that this setup is very similar to how currently `_hoodie_commit_seqno` is produced.
   
   - Added support to all 4 different writes in spark(spark ds, spark streaming, spark-ds and deltastreamer). 
   - Added/Fixed all built in key generators to support auto generation of record keys. 
   - Added guard rails around some of the configs when used along side auto generation of record keys (for eg, de-dup within same batch is not allowed, "upsert" operation type is not allowed etc). 
   
   ### Impact
   
   - Eases the usability of Hudi among users who are looking to ingest immutable datasets. 
   
   ### Risk level (write none, low medium or high below)
   
   Low. 
   
   ### Documentation Update
   
   Will be updating the configurations page on this. Also, might need to update our website to call out this feature separately. may be under KeyGenerators page. 
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1151448306


##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "

Review Comment:
   yes. @yihua had some tracking ticket for this. Ethan: I could not locate one. can you share the jira link. 



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())

Review Comment:
   yeah. we can't generate for an upsert. its only insert or bulk_insert for spark-datasource writes. but w/ spark-sql, we might want to support MIT, updates, deletes. so, will fix this accordingly. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java:
##########
@@ -29,24 +31,29 @@
 public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   public SimpleAvroKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(props, Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null)),
         props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()));
   }
 
   SimpleAvroKeyGenerator(TypedProperties props, String partitionPathField) {
-    this(props, null, partitionPathField);
+    this(props, Option.empty(), partitionPathField);
   }
 
-  SimpleAvroKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
+  SimpleAvroKeyGenerator(TypedProperties props, Option<String> recordKeyField, String partitionPathField) {

Review Comment:
   we are good. everywhere we use the constructor w/ just TypedProps



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,19 @@ object HoodieDatasetBulkInsertHelper
           val keyGenerator =
             ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps))
               .asInstanceOf[SparkKeyGeneratorInterface]
+          val partitionId = TaskContext.getPartitionId()

Review Comment:
   yeah. if entire source RDD is not re-computed, we should be ok. 



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.{HoodieAutoRecordKeyException, HoodieException}
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =

Review Comment:
   so, when a user does not configure record key explicitly, we can turn on auto key generation automatically? 
   yes, we could do that. 



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "
+          + "no natural record key satisfying requirements of being the primary-key in the Hudi table. "
+          + "Record key auto-gen is generally recommended for 'append-only' workloads, ie ones leveraging 'insert' or "

Review Comment:
   just so we are on same page. your suggestion is to make this a writer config, but still make it immutable? i.e. once set, use can never unset? 
   I feel this bit contradicting to how write configs behaves which can keep changing from commit to another. let me know wdyt. I don't have a strong opinion. 
   



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,19 @@ object HoodieDatasetBulkInsertHelper
           val keyGenerator =
             ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps))
               .asInstanceOf[SparkKeyGeneratorInterface]
+          val partitionId = TaskContext.getPartitionId()
+          var rowId = 0

Review Comment:
   yes. you are right. 



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "
+          + "no natural record key satisfying requirements of being the primary-key in the Hudi table. "
+          + "Record key auto-gen is generally recommended for 'append-only' workloads, ie ones leveraging 'insert' or "

Review Comment:
   sure. will take a stab at removing as a table property. 
   just that table properties are immutable, where as writer properties are mutable. Since for a table where auto generation is enabled, it does not makes sense to go back to other key gens, I thought it makes sense to go w/ table property. 
   



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java:
##########
@@ -41,7 +42,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
   private final TimestampBasedAvroKeyGenerator timestampBasedAvroKeyGenerator;
 
   public TimestampBasedKeyGenerator(TypedProperties config) throws IOException {
-    this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null),

Review Comment:
   nope. have responded else where



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -608,14 +617,31 @@ private Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchFromSourc
     SerializableSchema avroSchema = new SerializableSchema(schemaProvider.getTargetSchema());
     SerializableSchema processedAvroSchema = new SerializableSchema(isDropPartitionColumns() ? HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get());
     if (recordType == HoodieRecordType.AVRO) {
-      records = avroRDD.map(record -> {
-        GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, partitionColumns) : record;
+      // if auto generation of keys is enabled, lets generate one
+      JavaRDD<Tuple2<GenericRecord, Option<String>>> recordRecordKeyOverrideRdd = avroRDD.mapPartitions(
+          (FlatMapFunction<Iterator<GenericRecord>, Tuple2<GenericRecord, Option<String>>>) genericRecordIterator -> {
+            int rowId = 0;

Review Comment:
   just in 2 places right. deltastreamer and other spark writers.
   yeah, its not easy to avoid de-dup here



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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1488271779

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0d733095ce4e9342901a03f0530c94c373593d15 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1492028726

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0d733095ce4e9342901a03f0530c94c373593d15 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968) 
   * 2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1495222373

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 09d9feab5048d47a149f4088c23af9b5072250fa Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077) 
   * d3e3d9ffd1bf60dabfb36d37133493683ea56a4c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100) 
   * 572189472623065f460bd18436fb3b21602449af UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1495306511

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 572189472623065f460bd18436fb3b21602449af Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1495348583

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 572189472623065f460bd18436fb3b21602449af Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101) 
   * 711df161776bfbe4f66cb04310eb15ccc0069716 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537296681

   Only one test failure: https://dev.azure.com/apache-hudi-ci-org/apache-hudi-ci/_build/results?buildId=16905&view=logs&j=b1544eb9-7ff1-5db9-0187-3e05abf459bc&t=e0ae894b-41c9-5f4b-7ed2-bdf5243b02e7&l=268132


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1186790831


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala:
##########
@@ -109,16 +109,6 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
         StructField("dt", StringType, true))
     )
 
-    // miss primaryKey parameter
-    val sqlOptions1 = baseSqlOptions ++ Map(
-      "type" -> "mor"
-    )
-
-    val e1 = intercept[IllegalArgumentException] {
-      HoodieOptionConfig.validateTable(spark, schema, sqlOptions1)

Review Comment:
   we can't add this back. after this patch, record key field is not a mandatory field in general. 
   



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131847074


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestAutoGenerationOfRecordKeys.scala:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.functional
+
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hudi.HoodieConversionUtils.toJavaOption
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType, WriteOperationType}
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.common.util
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.exception.ExceptionUtil.getRootCause
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.functional.CommonOptionUtils._
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions
+import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator, TimestampBasedKeyGenerator}
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
+import org.apache.hudi.testutils.HoodieSparkClientTestBase
+import org.apache.hudi.util.JFunction
+import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers, ScalaAssertionSupport}
+import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
+import org.apache.spark.sql.{SaveMode, SparkSession, SparkSessionExtensions}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{CsvSource, EnumSource}
+
+import java.util.function.Consumer
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+class TestAutoGenerationOfRecordKeys extends HoodieSparkClientTestBase with ScalaAssertionSupport {
+  var spark: SparkSession = null

Review Comment:
   this will be set in BeforeEach method. we don't have any code paths were this might be null. I don't think we need to add Option here. 



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514] Adding auto generation of record keys support to Hudi

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1463434776

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9dfbe3e6135456e7f8c79513270eb5e7e4ed123d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131844741


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("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 HoodieException("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 HoodieException("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.
+   * @return Iterator of Pair of GenericRecord and Optionally generated record key.
+   */
+  def mayBeAutoGenerateRecordKeys(autoGenerateKeys : Boolean, genRecsItr: Iterator[GenericRecord], instantTime: String): Iterator[(GenericRecord, Option[String])] = {
+    var rowId = 0
+    val sparkPartitionId = TaskContext.getPartitionId()
+
+    // 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

Review Comment:
   yes. from our testing, ordering is guaranteed. 



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131853630


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,

Review Comment:
   nope. its feasible via spark-sql. will tackle this in phase 2
   



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1534445001

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd5d81ac4fc008709c4e34438c3242dcb18c6161 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1144114597


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java:
##########
@@ -36,8 +39,9 @@ public class NonpartitionedAvroKeyGenerator extends BaseKeyGenerator {
 
   public NonpartitionedAvroKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
-        .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
+    this.recordKeyFields = autoGenerateRecordKeys() ? Collections.emptyList() :

Review Comment:
   should this code be pushed to the super class?



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

Review Comment:
   may be a simple if instead of ternary is more readable? ternary is best used IMO for simple expressions. 
   
   



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java:
##########
@@ -29,24 +31,29 @@
 public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   public SimpleAvroKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(props, Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null)),

Review Comment:
   won't it return `null` if the key is not found in `props` anyways?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java:
##########
@@ -29,24 +31,29 @@
 public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   public SimpleAvroKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(props, Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null)),
         props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()));
   }
 
   SimpleAvroKeyGenerator(TypedProperties props, String partitionPathField) {
-    this(props, null, partitionPathField);
+    this(props, Option.empty(), partitionPathField);
   }
 
-  SimpleAvroKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
+  SimpleAvroKeyGenerator(TypedProperties props, Option<String> recordKeyField, String partitionPathField) {
     super(props);
-    this.recordKeyFields = recordKeyField == null
+    this.recordKeyFields = !recordKeyField.isPresent()
         ? Collections.emptyList()
-        : Collections.singletonList(recordKeyField);
+        : Collections.singletonList(recordKeyField.get());
     this.partitionPathFields = Collections.singletonList(partitionPathField);
   }
 
   @Override
   public String getRecordKey(GenericRecord record) {
+    if (autoGenerateRecordKeys()) {
+      // To fetch partition path, caller will have to call getKey() on KeyGenerator and call .getPartitionPath. Hence we have to pass empty field to support

Review Comment:
   avoid this comment everywhere?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java:
##########
@@ -29,24 +31,29 @@
 public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   public SimpleAvroKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(props, Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null)),
         props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()));
   }
 
   SimpleAvroKeyGenerator(TypedProperties props, String partitionPathField) {
-    this(props, null, partitionPathField);
+    this(props, Option.empty(), partitionPathField);
   }
 
-  SimpleAvroKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
+  SimpleAvroKeyGenerator(TypedProperties props, Option<String> recordKeyField, String partitionPathField) {

Review Comment:
   This does not break any reflection based instantiation?



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,19 @@ object HoodieDatasetBulkInsertHelper
           val keyGenerator =
             ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps))
               .asInstanceOf[SparkKeyGeneratorInterface]
+          val partitionId = TaskContext.getPartitionId()
+          var rowId = 0

Review Comment:
   nts: this var gets copied over to each executed and we generate the `rowId` independently for each partition?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java:
##########
@@ -41,7 +42,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
   private final TimestampBasedAvroKeyGenerator timestampBasedAvroKeyGenerator;
 
   public TimestampBasedKeyGenerator(TypedProperties config) throws IOException {
-    this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null),

Review Comment:
   same. will `getString()` not return null if not found



##########
hudi-common/src/main/java/org/apache/hudi/exception/HoodieAutoRecordKeyException.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.exception;
+
+/**
+ * Used for exceptions related to Auto generation of record keys.
+ */
+public class HoodieAutoRecordKeyException extends HoodieException {

Review Comment:
   rename: HoodieKeyGenerationException



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.{HoodieAutoRecordKeyException, HoodieException}
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =

Review Comment:
   Can we simplify this more for the user. automatically turn on the config, if the user does `bulk_insert` from all Spark write paths. If they upsert the table later, then we generate keys for inserted records.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java:
##########
@@ -29,24 +31,29 @@
 public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   public SimpleAvroKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(props, Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null)),

Review Comment:
   https://docs.oracle.com/javase/8/docs/api/java/util/Properties.html#getProperty-java.lang.String-



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())

Review Comment:
   idk how we would autogenerate keys for inserted records within an upsert for eg. without implementing this at the HoodieMerge/Create handle layers? Could you sketch that impl?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.{HoodieAutoRecordKeyException, HoodieException}
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =

Review Comment:
   I think making this a top level choice, introduces the need to care about a new config auto generation of key, to not configure an existing config - key generator.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java:
##########
@@ -29,24 +31,29 @@
 public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   public SimpleAvroKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(props, Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null)),
         props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()));
   }
 
   SimpleAvroKeyGenerator(TypedProperties props, String partitionPathField) {
-    this(props, null, partitionPathField);
+    this(props, Option.empty(), partitionPathField);
   }
 
-  SimpleAvroKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
+  SimpleAvroKeyGenerator(TypedProperties props, Option<String> recordKeyField, String partitionPathField) {
     super(props);
-    this.recordKeyFields = recordKeyField == null
+    this.recordKeyFields = !recordKeyField.isPresent()

Review Comment:
   use `Option.ifPresent`? or `Option.map.orElse` ?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "

Review Comment:
   +1 we should track this in the separate usability improvement track



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,19 @@ object HoodieDatasetBulkInsertHelper
           val keyGenerator =
             ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps))
               .asInstanceOf[SparkKeyGeneratorInterface]
+          val partitionId = TaskContext.getPartitionId()

Review Comment:
   we should be resilient to drive failures as well? the entire Hudi commit will abort if the driver is restarted? I am wondering if the RDD recomputation introduces any non-determinism here in this here. In general, as long as we don't recompute the source RDD, it should be resilient



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java:
##########
@@ -64,12 +67,22 @@ public String getPartitionPath(GenericRecord record) {
 
   @Override
   public String getRecordKey(Row row) {
+    if (autoGenerateRecordKeys()) {
+      // To fetch partition path, caller will have to call getKey() on KeyGenerator and call .getPartitionPath. Hence we have to pass empty field to support
+      // returning partition path for the callers.
+      return StringUtils.EMPTY_STRING;
+    }
     tryInitRowAccessor(row.schema());
     return combineCompositeRecordKey(rowAccessor.getRecordKeyParts(row));
   }
 
   @Override
   public UTF8String getRecordKey(InternalRow internalRow, StructType schema) {
+    if (autoGenerateRecordKeys()) {
+      // To fetch partition path, caller will have to call getKey() on KeyGenerator and call .getPartitionPath. Hence we have to pass empty field to support

Review Comment:
   remove comment



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -608,14 +617,31 @@ private Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchFromSourc
     SerializableSchema avroSchema = new SerializableSchema(schemaProvider.getTargetSchema());
     SerializableSchema processedAvroSchema = new SerializableSchema(isDropPartitionColumns() ? HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get());
     if (recordType == HoodieRecordType.AVRO) {
-      records = avroRDD.map(record -> {
-        GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, partitionColumns) : record;
+      // if auto generation of keys is enabled, lets generate one
+      JavaRDD<Tuple2<GenericRecord, Option<String>>> recordRecordKeyOverrideRdd = avroRDD.mapPartitions(
+          (FlatMapFunction<Iterator<GenericRecord>, Tuple2<GenericRecord, Option<String>>>) genericRecordIterator -> {
+            int rowId = 0;

Review Comment:
   we are repeating this in 3 places now? I see that its hard to restructure though. hmmm.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "
+          + "no natural record key satisfying requirements of being the primary-key in the Hudi table. "
+          + "Record key auto-gen is generally recommended for 'append-only' workloads, ie ones leveraging 'insert' or "

Review Comment:
   IMO auto-generation should happen at the writer level. Do we need a table property? if there is no key generator, then the keys were auto generated?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1116,31 +1124,47 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys is auto generation is enabled.
+          val recordsWithRecordKeyOverride = mayBeAutoGenerateRecordKeys(autoGenerateRecordKeys, it, instantTime, sparkPartitionId)

Review Comment:
   nts: need to dig deeper here.



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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1488087455

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9dfbe3e6135456e7f8c79513270eb5e7e4ed123d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648) 
   * 0d733095ce4e9342901a03f0530c94c373593d15 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1156418648


##########
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:
   this is kind of unavoidable as of current structure. For eg, even to fetch partition path, our KeyGenerator interface, only exposes 
   ```
   HoodieKey getKey(GenericRecord record)
   ```
   
   So, to fetch partition path, we have to call getKey(genRec).getPartitionPath and hence I had to return empty string here. we don't want to add a new api to the interface just for this purpose. 
   
   Incase of auto key gen flows, we generate the record keys explicitly (not via key gen class) and add it to HoodieKey that we materialize in memory for all records.
   
   I can sync up w/ you f2f to clarify this. 
   Ideally, we need to have 2 different interfaces. one to generate partition path and one to generate record key. and so some of these workarounds may not be required. but w/ current structure, we use a single key gen class to generate both record keys and partition paths as well. 
   
   



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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1492040654

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0d733095ce4e9342901a03f0530c94c373593d15 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968) 
   * 2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1495748013

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 711df161776bfbe4f66cb04310eb15ccc0069716 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 merged pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 merged PR #8107:
URL: https://github.com/apache/hudi/pull/8107


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


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

Posted by "codope (via GitHub)" <gi...@apache.org>.
codope commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1127709935


##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "
+          + "no natural record key satisfying requirements of being the primary-key in the Hudi table. "
+          + "Record key auto-gen is only recommended for 'append-only' payloads, ie ones leveraging 'insert' or "
+          + "'bulk_insert' operations, not requiring de-duplication or updating existing (already persisted) "
+          + "records. But users should be able to levergage UPDATE construct with spark-sql to updated these records if need be. "
+          + "Note, that record keys produced by auto-gen are globally unique, therefore permitting "
+          + "records to be subsequently deleted (or updates via spark-sql) if necessary"

Review Comment:
   The two statements do not sound in line with each other. The first one says it's recommended for append-only, not requiring de-dup or update. The other says keys produced are globally unique to support subsequent deletes or updates. Can you clarify? 



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -671,6 +682,10 @@ public Boolean shouldDropPartitionColumns() {
     return getBooleanOrDefault(DROP_PARTITION_COLUMNS);
   }
 
+  public Boolean shouldAutoGenerateRecordKeys() {

Review Comment:
   this is unused?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "
+          + "no natural record key satisfying requirements of being the primary-key in the Hudi table. "
+          + "Record key auto-gen is only recommended for 'append-only' payloads, ie ones leveraging 'insert' or "
+          + "'bulk_insert' operations, not requiring de-duplication or updating existing (already persisted) "
+          + "records. But users should be able to levergage UPDATE construct with spark-sql to updated these records if need be. "

Review Comment:
   ```suggestion
             + "records. But users should be able to leverage UPDATE construct with spark-sql to update these records if need be. "
   ```



##########
hudi-common/src/main/java/org/apache/hudi/common/engine/TaskContextSupplier.java:
##########
@@ -34,5 +34,13 @@ public abstract class TaskContextSupplier implements Serializable {
 
   public abstract Supplier<Long> getAttemptIdSupplier();
 
+  public  Supplier<Integer> getAttemptNo() {
+    return () -> -1;
+  }
+
+  public Supplier<Integer> getStageAttemptNo() {

Review Comment:
   let's rename to `getStageAttemptNumber`?
   Where are these methods being used?



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,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)

Review Comment:
   What happens in retries? I think partitionId will be same but are the order of records guaranteed to be the same (or are they sorted)?



##########
hudi-common/src/main/java/org/apache/hudi/common/engine/TaskContextSupplier.java:
##########
@@ -34,5 +34,13 @@ public abstract class TaskContextSupplier implements Serializable {
 
   public abstract Supplier<Long> getAttemptIdSupplier();
 
+  public  Supplier<Integer> getAttemptNo() {

Review Comment:
   let's rename to `getAttemptNumber`?



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,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)
+            }
             val partitionPath = keyGenerator.getPartitionPath(row, schema)
             val commitTimestamp = UTF8String.EMPTY_UTF8
             val commitSeqNo = UTF8String.EMPTY_UTF8

Review Comment:
   Can you remind me why do we keep these as empty here? Is it populated in write handles? 
   Should we at least assign the commitSeqNo at this stage itself to avoid recomputing again?



##########
hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java:
##########
@@ -37,6 +38,7 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
   protected final boolean encodePartitionPath;
   protected final boolean hiveStylePartitioning;
   protected final boolean consistentLogicalTimestampEnabled;
+  private boolean autoGenerateRecordKeys;

Review Comment:
   let's keep the naming consistent throughout - i see three different names for the same purpose. Let's pick one of `autoGenerateRecordKeys`, or `doAutoGenerateRecordKeys`, or `shouldAutoGenerateRecordKeys`. I like the latter but any one throughout the code will do.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "

Review Comment:
   Will users still need to specify the keygen class if this is set to true?



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

Review Comment:
   we have to be a bit careful about setting this to empty list. This list is passed as-is to the `KeyGenUtils#getRecordKey` taht could fail for empty list.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java:
##########
@@ -44,6 +47,11 @@ public ComplexAvroKeyGenerator(TypedProperties props) {
 
   @Override
   public String getRecordKey(GenericRecord record) {
+    if (doAutoGenerateRecordKeys()) {
+      // To fetch partition path, caller will have to call getKey() on KeyGenerator and call .getPartitionPath. Hence we have to pass empty field to support
+      // returning partition path for the callers.

Review Comment:
   why calling `getKey()` is necessary for fetching partition path? shouldn't the implementation of `getPartitionPath` be necessary and sufficient?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("Enabling " + HoodieWriteConfig.COMBINE_BEFORE_INSERT.key() + " is not supported with auto generation of record keys ");

Review Comment:
   ```suggestion
           throw new HoodieException("Enabling " + HoodieWriteConfig.COMBINE_BEFORE_INSERT.key() + " is not supported with auto generation of record keys ")
   ```



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("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 HoodieException("Enabling " + INSERT_DROP_DUPS.key() + " is not supported with auto generation of record keys ");

Review Comment:
   ```suggestion
           throw new HoodieException("Enabling " + INSERT_DROP_DUPS.key() + " is not supported with auto generation of record keys ")
   ```



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("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 HoodieException("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 HoodieException("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.
+   * @return Iterator of Pair of GenericRecord and Optionally generated record key.
+   */
+  def mayBeAutoGenerateRecordKeys(autoGenerateKeys : Boolean, genRecsItr: Iterator[GenericRecord], instantTime: String): Iterator[(GenericRecord, Option[String])] = {
+    var rowId = 0
+    val sparkPartitionId = TaskContext.getPartitionId()
+
+    // 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

Review Comment:
   same question here too - are the order of records guaranteed to be same across retries?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java:
##########
@@ -809,6 +809,7 @@ public static class PropertyBuilder {
     private String metadataPartitions;
     private String inflightMetadataPartitions;
     private String secondaryIndexesMetadata;
+    private String autoGenerateRecordKeys;

Review Comment:
   How about keeping it boolean?



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,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

Review Comment:
   Did you also consider using `monotonically_increasing_id()`?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("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 HoodieException("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 HoodieException("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.
+   * @return Iterator of Pair of GenericRecord and Optionally generated record key.
+   */
+  def mayBeAutoGenerateRecordKeys(autoGenerateKeys : Boolean, genRecsItr: Iterator[GenericRecord], instantTime: String): Iterator[(GenericRecord, Option[String])] = {
+    var rowId = 0
+    val sparkPartitionId = TaskContext.getPartitionId()
+
+    // 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

Review Comment:
   Also, does it need to be atomic counter like we do for commit seq number in create/append handles? 



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestAutoGenerationOfRecordKeys.scala:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.functional
+
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hudi.HoodieConversionUtils.toJavaOption
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType, WriteOperationType}
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.common.util
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.exception.ExceptionUtil.getRootCause
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.functional.CommonOptionUtils._
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions
+import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator, TimestampBasedKeyGenerator}
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
+import org.apache.hudi.testutils.HoodieSparkClientTestBase
+import org.apache.hudi.util.JFunction
+import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers, ScalaAssertionSupport}
+import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
+import org.apache.spark.sql.{SaveMode, SparkSession, SparkSessionExtensions}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{CsvSource, EnumSource}
+
+import java.util.function.Consumer
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+class TestAutoGenerationOfRecordKeys extends HoodieSparkClientTestBase with ScalaAssertionSupport {
+  var spark: SparkSession = null

Review Comment:
   let's use `Option`?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())

Review Comment:
   Is there a plan to support upsert in future? If so, is the current design extensible enough for that?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java:
##########
@@ -38,7 +39,7 @@ public class SimpleKeyGenerator extends BuiltinKeyGenerator {
   private final SimpleAvroKeyGenerator simpleAvroKeyGenerator;
 
   public SimpleKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null),

Review Comment:
   let's use `Option`?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestAutoGenerationOfRecordKeys.scala:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.functional
+
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hudi.HoodieConversionUtils.toJavaOption
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType, WriteOperationType}
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.common.util
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.exception.ExceptionUtil.getRootCause
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.functional.CommonOptionUtils._
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions
+import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator, TimestampBasedKeyGenerator}
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
+import org.apache.hudi.testutils.HoodieSparkClientTestBase
+import org.apache.hudi.util.JFunction
+import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers, ScalaAssertionSupport}
+import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
+import org.apache.spark.sql.{SaveMode, SparkSession, SparkSessionExtensions}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{CsvSource, EnumSource}
+
+import java.util.function.Consumer
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+class TestAutoGenerationOfRecordKeys extends HoodieSparkClientTestBase with ScalaAssertionSupport {
+  var spark: SparkSession = null
+  val verificationCol: String = "driver"
+  val updatedVerificationVal: String = "driver_update"
+
+  override def getSparkSessionExtensionsInjector: util.Option[Consumer[SparkSessionExtensions]] =
+    toJavaOption(
+      Some(
+        JFunction.toJavaConsumer((receiver: SparkSessionExtensions) => new HoodieSparkSessionExtension().apply(receiver)))
+    )
+
+  @BeforeEach override def setUp() {
+    initPath()
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+    initTestDataGenerator()
+    initFileSystem()
+  }
+
+  @AfterEach override def tearDown() = {
+    cleanupSparkContexts()
+    cleanupTestDataGenerator()
+    cleanupFileSystem()
+    FileSystem.closeAll()
+    System.gc()
+  }
+
+  @ParameterizedTest
+  @CsvSource(value = Array(
+    "AVRO,insert,COPY_ON_WRITE", "AVRO,bulk_insert,COPY_ON_WRITE", "AVRO,insert,MERGE_ON_READ", "AVRO,bulk_insert,MERGE_ON_READ"
+  ))
+  def testRecordKeysAutoGen(recordType: HoodieRecordType, op: String, tableType: HoodieTableType): Unit = {
+    testRecordKeysAutoGenInternal(recordType, op, tableType)
+  }
+
+  @Test
+  def testRecordKeyAutoGenWithTimestampBasedKeyGen(): Unit = {
+    testRecordKeysAutoGenInternal(HoodieRecordType.AVRO, "insert", HoodieTableType.COPY_ON_WRITE,
+      classOf[TimestampBasedKeyGenerator].getName)
+  }
+
+  @Test
+  def testRecordKeyAutoGenWithComplexKeyGen(): Unit = {
+    testRecordKeysAutoGenInternal(HoodieRecordType.AVRO, "insert", HoodieTableType.COPY_ON_WRITE,
+      classOf[ComplexKeyGenerator].getName,
+      complexPartitionPath = true)
+  }
+
+  @Test
+  def testRecordKeyAutoGenWithNonPartitionedKeyGen(): Unit = {
+    testRecordKeysAutoGenInternal(HoodieRecordType.AVRO, "insert", HoodieTableType.COPY_ON_WRITE,
+      classOf[NonpartitionedKeyGenerator].getName, complexPartitionPath = false, nonPartitionedDataset = true)
+  }
+
+  def testRecordKeysAutoGenInternal(recordType: HoodieRecordType, op: String = "insert", tableType: HoodieTableType = HoodieTableType.COPY_ON_WRITE,
+                                    keyGenClass: String = classOf[SimpleKeyGenerator].getCanonicalName,
+                                    complexPartitionPath: Boolean = false, nonPartitionedDataset: Boolean = false): Unit = {
+    val (vanillaWriteOpts, readOpts) = getWriterReaderOpts(recordType)
+
+    var options: Map[String, String] = vanillaWriteOpts ++ Map(
+      DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> keyGenClass)
+
+    val isTimestampBasedKeyGen: Boolean = classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)
+    if (isTimestampBasedKeyGen) {
+      options += Config.TIMESTAMP_TYPE_FIELD_PROP -> "DATE_STRING"
+      options += Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP -> "yyyy/MM/dd"
+      options += Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyyMMdd"
+    }
+
+    if (complexPartitionPath) {
+      options += KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key() -> "rider,_hoodie_is_deleted"
+    }
+    if (nonPartitionedDataset) {
+      options = options -- Seq(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())
+    }
+
+    // NOTE: In this test we deliberately removing record-key configuration
+    //       to validate Hudi is handling this case appropriately
+    val writeOpts = options -- Seq(DataSourceWriteOptions.RECORDKEY_FIELD.key) ++ Map(
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key() -> "true"
+    )
+
+    // Insert Operation
+    val records = recordsToStrings(dataGen.generateInserts("000", 5)).toList
+    val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
+    inputDF.cache
+
+    //
+    // Step #1: Persist first batch with auto-gen'd record-keys
+    //
+
+    inputDF.write.format("hudi")
+      .options(writeOpts)
+      .option(DataSourceWriteOptions.OPERATION.key, op)
+      .option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType.name())
+      .mode(SaveMode.Overwrite)
+      .save(basePath)
+
+    assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
+
+    //
+    // Step #2: Persist *same* batch with auto-gen'd record-keys (new record keys should
+    //          be generated this time)
+    //
+    val inputDF2 = inputDF
+    inputDF2.write.format("hudi")
+      .options(writeOpts)
+      .option(DataSourceWriteOptions.OPERATION.key, op)
+      .option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType.name())
+      .mode(SaveMode.Append)
+      .save(basePath)
+
+    val readDF = spark.read.format("hudi")
+      .options(readOpts)
+      .load(basePath)
+    readDF.cache
+
+    val recordKeys = readDF.select(HoodieRecord.RECORD_KEY_METADATA_FIELD)
+      .distinct()
+      .collectAsList()
+      .map(_.getString(0))
+
+    // Validate auto-gen'd keys are globally unique
+    assertEquals(10, recordKeys.size)
+
+    // validate entire batch is present in snapshot read
+    val expectedInputDf = inputDF.union(inputDF2).drop("partition","rider","_hoodie_is_deleted")
+    val actualDf = readDF.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala: _*).drop("partition","rider","_hoodie_is_deleted")
+    assertEquals(expectedInputDf.except(actualDf).count, 0)
+  }
+
+  @Test
+  def testRecordKeysAutoGenEnableToDisable(): Unit = {
+    val (vanillaWriteOpts, readOpts) = getWriterReaderOpts(HoodieRecordType.AVRO)
+
+    var options: Map[String, String] = vanillaWriteOpts ++ Map(
+      DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> classOf[SimpleKeyGenerator].getCanonicalName)
+
+    // NOTE: In this test we deliberately removing record-key configuration
+    //       to validate Hudi is handling this case appropriately
+    val writeOpts = options -- Seq(DataSourceWriteOptions.RECORDKEY_FIELD.key) ++ Map(
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key() -> "true"
+    )
+
+    // Insert Operation
+    val records = recordsToStrings(dataGen.generateInserts("000", 5)).toList
+    val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
+    inputDF.cache
+
+    //
+    // Step #1: Persist first batch with auto-gen'd record-keys
+    //
+
+    inputDF.write.format("hudi")
+      .options(writeOpts)
+      .option(DataSourceWriteOptions.OPERATION.key, "insert")
+      .mode(SaveMode.Overwrite)
+      .save(basePath)
+
+    assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
+
+    //
+    // Step #2: Insert w/ explicit record key config and removing auto generate config. Should fail since we can't modify
+    // this property.
+    //
+    val e = assertThrows(classOf[HoodieException]) {

Review Comment:
   Should we create a separate Exception for this feature?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,

Review Comment:
   How are deletes being handled? Is it soft delete or hard delete?



##########
hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java:
##########
@@ -37,6 +38,7 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
   protected final boolean encodePartitionPath;
   protected final boolean hiveStylePartitioning;
   protected final boolean consistentLogicalTimestampEnabled;
+  private boolean autoGenerateRecordKeys;

Review Comment:
   make it `final`?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java:
##########
@@ -44,6 +47,11 @@ public ComplexAvroKeyGenerator(TypedProperties props) {
 
   @Override
   public String getRecordKey(GenericRecord record) {
+    if (doAutoGenerateRecordKeys()) {

Review Comment:
   how about `if (doAutoGenerateRecordKeys() || getRecordKeyFields.isEmpty())`?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java:
##########
@@ -49,7 +50,9 @@ public SimpleKeyGenerator(TypedProperties props) {
   SimpleKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
     super(props);
     // Make sure key-generator is configured properly
-    validateRecordKey(recordKeyField);
+    if (!doAutoGenerateRecordKeys()) {
+      validateRecordKey(recordKeyField);
+    }

Review Comment:
   move it inside `validateRecordKey`? that method should be self-contained, return true if autoGenerateRecordKeys is enabled.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1096,31 +1104,47 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys is auto generation is enabled.
+          val recordsWithRecordKeyOverride = mayBeAutoGenerateRecordKeys(autoGenerateRecordKeys, it, instantTime)
+
+          // handle dropping partition columns
+          recordsWithRecordKeyOverride.map { avroRecordRecordKeyOverRide =>
             val processedRecord = if (shouldDropPartitionColumns) {
-              HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema)
+              HoodieAvroUtils.rewriteRecord(avroRecordRecordKeyOverRide._1, dataFileSchema)
+            } else {
+              avroRecordRecordKeyOverRide._1
+            }
+
+            // Generate HoodieKey for records
+            val hoodieKey = if (autoGenerateRecordKeys) {
+              // fetch record key from the recordKeyOverride if auto generation is enabled.
+              new HoodieKey(avroRecordRecordKeyOverRide._2.get, keyGenerator.getKey(avroRecordRecordKeyOverRide._1).getPartitionPath)

Review Comment:
   I see why you need to handle the autoGenerateRecordKeys as a special case in the keygens. This is a bit troubling that to get the partition path we need to call getKey first. I vaguely recall our discussion around decoupling partition path from keygen. For now, can we not add a method `getPartitionPath` in `KeyGenerator` and reuse the underlying implementation instead of going through `HoodieKey`? 



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("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 HoodieException("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 HoodieException("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.
+   * @return Iterator of Pair of GenericRecord and Optionally generated record key.
+   */
+  def mayBeAutoGenerateRecordKeys(autoGenerateKeys : Boolean, genRecsItr: Iterator[GenericRecord], instantTime: String): Iterator[(GenericRecord, Option[String])] = {
+    var rowId = 0
+    val sparkPartitionId = TaskContext.getPartitionId()

Review Comment:
   how about passing it from the call site?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("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 HoodieException("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 HoodieException("Disabling " + HoodieTableConfig.POPULATE_META_FIELDS.key() + " is not supported with auto generation of record keys");

Review Comment:
   ```suggestion
           throw new HoodieException("Disabling " + HoodieTableConfig.POPULATE_META_FIELDS.key() + " is not supported with auto generation of record keys")
   ```



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131842281


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())

Review Comment:
   as called out in the docs, UPDATE and DELETE via spark-sql should be supported. That will be phase 2.
   



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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1495342734

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 572189472623065f460bd18436fb3b21602449af Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101) 
   * 711df161776bfbe4f66cb04310eb15ccc0069716 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1495226882

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d3e3d9ffd1bf60dabfb36d37133493683ea56a4c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100) 
   * 572189472623065f460bd18436fb3b21602449af Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185740539


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperKeyGenerator.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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ */
+public class AutoRecordGenWrapperKeyGenerator extends BuiltinKeyGenerator {
+
+  private final BuiltinKeyGenerator builtinKeyGenerator;
+  private final boolean autoGenerateRecordKeys;
+  private final AtomicBoolean initializeAutoKeyGenProps = new AtomicBoolean(false);
+  private int partitionId;
+  private String instantTime;
+  private int rowId;
+
+  public AutoRecordGenWrapperKeyGenerator(TypedProperties config, BuiltinKeyGenerator builtinKeyGenerator) {
+    super(config);
+    this.builtinKeyGenerator = builtinKeyGenerator;
+    this.autoGenerateRecordKeys = !config.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key());

Review Comment:
   he he.yes. my bad. 



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185756525


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -1267,4 +1268,41 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
       })
     }
   }
+
+  test("Test Insert Into with auto generate record keys") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  dt string,
+           |  name string,
+           |  price double,
+           |  ts long
+           |) using hudi
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}'
+       """.stripMargin)

Review Comment:
   We have tested all key generators in TestAutoGenerationOfRecordKeys with diff operation types as well. So, did not want to repeat them again



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1536168943

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce3d13748cd488df7e055392f2e9db4ac2bfc18b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185060720


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/HoodieWrapperKeyGenerator.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.StringUtils;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.List;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ */
+public class HoodieWrapperKeyGenerator extends BuiltinKeyGenerator {
+
+  private final BuiltinKeyGenerator builtinKeyGenerator;
+
+  public HoodieWrapperKeyGenerator(TypedProperties config, BuiltinKeyGenerator builtinKeyGenerator) {
+    super(config);
+    this.builtinKeyGenerator = builtinKeyGenerator;
+  }
+
+  @Override
+  public String getRecordKey(GenericRecord record) {
+    if (autoGenerateRecordKeys()) {
+      return StringUtils.EMPTY_STRING;
+    } else {

Review Comment:
   Initially I tried moving it out, but we have some validations on record key field present in some of the key gen classes including base class. except for those conditionals, no additional processing is based on autoGenerateRecordKeys().



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/HoodieWrapperAvroKeyGenerator.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.StringUtils;
+
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.List;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ */
+public class HoodieWrapperAvroKeyGenerator extends BaseKeyGenerator {
+
+  private BaseKeyGenerator keyGenerator;
+
+  public HoodieWrapperAvroKeyGenerator(TypedProperties config, BaseKeyGenerator keyGenerator) {
+    super(config);
+    this.keyGenerator = keyGenerator;
+  }
+
+  @Override
+  public String getRecordKey(GenericRecord record) {
+    if (autoGenerateRecordKeys()) {
+      return StringUtils.EMPTY_STRING;

Review Comment:
   same response as above.



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185675443


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -83,8 +87,19 @@ object HoodieDatasetBulkInsertHelper
             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

Review Comment:
   yet to fix this 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java:
##########
@@ -61,10 +61,11 @@ public class KeyGenUtils {
    */
   public static KeyGeneratorType inferKeyGeneratorType(
       String recordsKeyFields, String partitionFields) {
+    boolean autoGenerateRecordKeys = recordsKeyFields == null;
     if (!StringUtils.isNullOrEmpty(partitionFields)) {
       int numPartFields = partitionFields.split(",").length;
-      int numRecordKeyFields = recordsKeyFields.split(",").length;
-      if (numPartFields == 1 && numRecordKeyFields == 1) {
+      int numRecordKeyFields = autoGenerateRecordKeys ? 0 : recordsKeyFields.split(",").length;

Review Comment:
   yet to address this.



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537297965

   And this one: `org.apache.hudi.TestDataSourceDefaults`


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


[GitHub] [hudi] nsivabalan commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1538386415

   yes, its a known flaky test. Going ahead w/ landing. 
   


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537278113

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e49fef0e72149160ffe124636e7c89d1ebe97e18 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1536894048

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce3d13748cd488df7e055392f2e9db4ac2bfc18b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861) 
   * b5b72b6043cc2a110d175b8f2b69b9e38902359d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1493528241

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023) 
   * 96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1156776979


##########
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:
   We already have `getRecordKey` and `getPartitionPath` as the public API, if you want to fix the `HoodieKey`, shouldn't the `HoodieKey getKey()` be fixed instead?



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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1499813805

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 711df161776bfbe4f66cb04310eb15ccc0069716 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110) 
   * e03d1c8712b0ae5dce7743a2588ee6b606cc562d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1493559301

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023) 
   * 96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1493606668

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 101db3163460b64d02c519d75ac75e04e85a3305 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073) 
   * 09d9feab5048d47a149f4088c23af9b5072250fa Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1493602283

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070) 
   * 101db3163460b64d02c519d75ac75e04e85a3305 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073) 
   * 09d9feab5048d47a149f4088c23af9b5072250fa UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1534258433

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e03d1c8712b0ae5dce7743a2588ee6b606cc562d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177) 
   * dd5d81ac4fc008709c4e34438c3242dcb18c6161 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1187567556


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java:
##########
@@ -55,7 +57,11 @@ public enum PartitionKeyType {
 
   public CustomAvroKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")).map(String::trim).collect(Collectors.toList());
+    this.recordKeyFields = Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null))

Review Comment:
   sure.



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1151437700


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java:
##########
@@ -36,8 +39,9 @@ public class NonpartitionedAvroKeyGenerator extends BaseKeyGenerator {
 
   public NonpartitionedAvroKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
-        .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
+    this.recordKeyFields = autoGenerateRecordKeys() ? Collections.emptyList() :

Review Comment:
   we have 10+ classes to fix on this. wondering if we can make it in a separate patch https://issues.apache.org/jira/browse/HUDI-5995 
   dont want to add more changes to this patch.



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514] Adding auto generation of record keys support to Hudi

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1457397182

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 35aed635391309c3c6c4b3794044bba53b3468ef Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131832606


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,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)
+            }
             val partitionPath = keyGenerator.getPartitionPath(row, schema)
             val commitTimestamp = UTF8String.EMPTY_UTF8
             val commitSeqNo = UTF8String.EMPTY_UTF8

Review Comment:
   guess, we only focussed on record key and partition path since it has to be generated upfront. rest of them can be filled or generated from within write handle. Lets keep other optimization or unrelated changes out of this patch. Already its a large patch. 
   



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131845834


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1096,31 +1104,47 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys is auto generation is enabled.
+          val recordsWithRecordKeyOverride = mayBeAutoGenerateRecordKeys(autoGenerateRecordKeys, it, instantTime)
+
+          // handle dropping partition columns
+          recordsWithRecordKeyOverride.map { avroRecordRecordKeyOverRide =>
             val processedRecord = if (shouldDropPartitionColumns) {
-              HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema)
+              HoodieAvroUtils.rewriteRecord(avroRecordRecordKeyOverRide._1, dataFileSchema)
+            } else {
+              avroRecordRecordKeyOverRide._1
+            }
+
+            // Generate HoodieKey for records
+            val hoodieKey = if (autoGenerateRecordKeys) {
+              // fetch record key from the recordKeyOverride if auto generation is enabled.
+              new HoodieKey(avroRecordRecordKeyOverRide._2.get, keyGenerator.getKey(avroRecordRecordKeyOverRide._1).getPartitionPath)

Review Comment:
   yes. 
   https://github.com/apache/hudi/pull/7699
   HUDI-5535



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


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

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1490815859

   > it may not be easy for us to deduce whether the table had previous auto key gen enabled and if the user switches later?
   Isn't that user's responsibility though? Do we store the key generator today in table properties. if not, this PR should keep that behavior IMO


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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1153980994


##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "

Review Comment:
   -1 for this option, let's sync up the semantics like RDBMS does, if the table has no definition of any primary keys, it become a non-deduplication table.



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185716426


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ */
+public class AutoRecordGenWrapperAvroKeyGenerator extends BaseKeyGenerator {
+
+  private final BaseKeyGenerator keyGenerator;
+  private final boolean autoGenerateRecordKeys;
+  private final AtomicBoolean initializeAutoKeyGenProps = new AtomicBoolean(false);
+  private int partitionId;
+  private String instantTime;
+  private int rowId;
+
+  public AutoRecordGenWrapperAvroKeyGenerator(TypedProperties config, BaseKeyGenerator keyGenerator) {
+    super(config);
+    this.keyGenerator = keyGenerator;
+    this.autoGenerateRecordKeys = !config.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key());
+  }

Review Comment:
   Isn't the attribute `autoGenerateRecordKeys` always true?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperKeyGenerator.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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ */
+public class AutoRecordGenWrapperKeyGenerator extends BuiltinKeyGenerator {
+
+  private final BuiltinKeyGenerator builtinKeyGenerator;
+  private final boolean autoGenerateRecordKeys;
+  private final AtomicBoolean initializeAutoKeyGenProps = new AtomicBoolean(false);
+  private int partitionId;
+  private String instantTime;
+  private int rowId;
+
+  public AutoRecordGenWrapperKeyGenerator(TypedProperties config, BuiltinKeyGenerator builtinKeyGenerator) {
+    super(config);
+    this.builtinKeyGenerator = builtinKeyGenerator;
+    this.autoGenerateRecordKeys = !config.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key());

Review Comment:
   The `autoGenerateRecordKeys` is always true, the factory already ensures that.



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -79,11 +83,20 @@ object HoodieDatasetBulkInsertHelper
 
       val prependedRdd: RDD[InternalRow] =
         df.queryExecution.toRdd.mapPartitions { iter =>
-          val keyGenerator =
+          val sparkKeyGenerator =
             ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps))
-              .asInstanceOf[SparkKeyGeneratorInterface]
+              .asInstanceOf[BuiltinKeyGenerator]
+              val keyGenerator: BuiltinKeyGenerator = if (autoGenerateRecordKeys) {
+                val typedProps = new TypedProperties(config.getProps)
+                typedProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(TaskContext.getPartitionId()))
+                typedProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime)
+                new AutoRecordGenWrapperKeyGenerator(typedProps, sparkKeyGenerator).asInstanceOf[BuiltinKeyGenerator]

Review Comment:
   Curious why we didn't have any factory class there?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.InternalRow
+
+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])] = {

Review Comment:
   Can be removed.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala:
##########
@@ -844,8 +844,8 @@ object DataSourceOptionsHelper {
    */
   def fetchMissingWriteConfigsFromTableConfig(tableConfig: HoodieTableConfig, params: Map[String, String]) : Map[String, String] = {
     val missingWriteConfigs = scala.collection.mutable.Map[String, String]()
-    if (!params.contains(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) && tableConfig.getRecordKeyFieldProp != null) {
-      missingWriteConfigs ++= Map(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key() -> tableConfig.getRecordKeyFieldProp)
+    if (!params.contains(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) && tableConfig.getRawRecordKeyFieldProp != null) {
+      missingWriteConfigs ++= Map(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key() -> tableConfig.getRawRecordKeyFieldProp)

Review Comment:
   We need to dig out why the `_hoodie_record_key` is used as a backend field name before, there it actually can take effect?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala:
##########
@@ -259,17 +249,6 @@ class TestDataSourceDefaults extends ScalaAssertionSupport {
       keyGen.getRecordKey(internalRow, structType)
     }
 
-    // Record's key field not specified
-    assertThrows(classOf[IllegalArgumentException]) {
-      val props = new TypedProperties()
-      props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField")
-      val keyGen = new ComplexKeyGenerator(props)
-
-      keyGen.getKey(baseRecord)

Review Comment:
   add it back to make the test pass.



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java:
##########
@@ -75,11 +75,6 @@ public void testNullPartitionPathFields() {
     Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutPartitionPathProp()));
   }
 
-  @Test
-  public void testNullRecordKeyFields() {
-    Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutRecordKeyProp()));
-  }

Review Comment:
   Have not addressed?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java:
##########
@@ -109,17 +110,17 @@ public UTF8String getPartitionPath(InternalRow row, StructType schema) {
     return combinePartitionPathUnsafe(rowAccessor.getRecordPartitionPathValues(row));
   }
 
-  private static void validatePartitionPath(String partitionPathField) {
+  private void validatePartitionPath(String partitionPathField) {

Review Comment:
   It is a good convention to alway make utilities `static`!



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.InternalRow
+
+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])] = {
+
+    genRecsItr.map(avroRecord => {
+        (avroRecord, Option.empty)
+      })
+  }
+
+  def mayBeAutoGenerateRecordKeysForSparkRow(autoGenerateKeys : Boolean, internalRowsItr: Iterator[InternalRow], instantTime: String,
+                                  sparkPartitionId: Integer): Iterator[(InternalRow, Option[String])] = {

Review Comment:
   Can be removed.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1118,47 +1124,74 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+          val keyGenProps = new TypedProperties(config.getProps)
+          if (autoGenerateRecordKeys) {
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(sparkPartitionId))

Review Comment:
   Guess the initialization of `keyGenProps` can be moved out of the `mapPartition` call?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala:
##########
@@ -89,16 +89,6 @@ class TestDataSourceDefaults extends ScalaAssertionSupport {
       }
     }
 
-    {
-      // Record's key field not specified
-      val props = new TypedProperties()
-      props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "partitionField")
-
-      assertThrows(classOf[IllegalArgumentException]) {
-        new SimpleKeyGenerator(props)

Review Comment:
   add it back to make the test pass.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1118,47 +1124,74 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+          val keyGenProps = new TypedProperties(config.getProps)
+          if (autoGenerateRecordKeys) {
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(sparkPartitionId))
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime)
+          }
+          val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(keyGenProps)
+            .asInstanceOf[BaseKeyGenerator]
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          val recordRecordKeyPairItr = it.map(avroRecord => {
+            (avroRecord, keyGenerator.getRecordKey(avroRecord))
+          })

Review Comment:
   Why we eagerly materialize the record key first?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala:
##########
@@ -109,16 +109,6 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
         StructField("dt", StringType, true))
     )
 
-    // miss primaryKey parameter
-    val sqlOptions1 = baseSqlOptions ++ Map(
-      "type" -> "mor"
-    )
-
-    val e1 = intercept[IllegalArgumentException] {
-      HoodieOptionConfig.validateTable(spark, schema, sqlOptions1)

Review Comment:
   add it back to make the test pass.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java:
##########
@@ -61,10 +64,27 @@ public class KeyGenUtils {
    */
   public static KeyGeneratorType inferKeyGeneratorType(
       String recordsKeyFields, String partitionFields) {
+    boolean autoGenerateRecordKeys = recordsKeyFields == null;
+    if (autoGenerateRecordKeys) {

Review Comment:
   Make `recordsKeyFields` param optional ?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -1267,4 +1268,41 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
       })
     }
   }
+
+  test("Test Insert Into with auto generate record keys") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  dt string,
+           |  name string,
+           |  price double,
+           |  ts long
+           |) using hudi
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}'
+       """.stripMargin)

Review Comment:
   test all cases including: simple keygen/complex key gen and non-partitioned key gen



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -867,18 +894,20 @@ private Pair<Option<String>, JavaRDD<WriteStatus>> writeToSink(JavaRDD<HoodieRec
    *
    * @return Instant time of the commit
    */
-  private String startCommit() {
+  private String startCommit(String instantTime, boolean retryEnabled) {

Review Comment:
   Is this change related with the PR?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -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
+
+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) {

Review Comment:
   Can we move the check `if (autoGenerateRecordKeys) {` out of the method?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala:
##########
@@ -487,16 +466,6 @@ class TestDataSourceDefaults extends ScalaAssertionSupport {
       assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType))
     }
 
-    {
-      // Record's key field not specified
-      val props = new TypedProperties()
-      props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField")

Review Comment:
   add it back to make the test pass.



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185755227


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1118,47 +1124,74 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+          val keyGenProps = new TypedProperties(config.getProps)
+          if (autoGenerateRecordKeys) {
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(sparkPartitionId))
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime)
+          }
+          val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(keyGenProps)
+            .asInstanceOf[BaseKeyGenerator]
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          val recordRecordKeyPairItr = it.map(avroRecord => {
+            (avroRecord, keyGenerator.getRecordKey(avroRecord))
+          })

Review Comment:
   not required anymore. It was a residue from last clean up.



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185818843


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1120,47 +1126,69 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+          val keyGenProps = new TypedProperties(config.getProps)
+          if (autoGenerateRecordKeys) {
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(sparkPartitionId))
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime)
+          }
+          val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(keyGenProps)
+            .asInstanceOf[BaseKeyGenerator]
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // handle dropping partition columns
+          it.map { avroRec =>
             val processedRecord = if (shouldDropPartitionColumns) {
-              HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema)
+              HoodieAvroUtils.rewriteRecord(avroRec, dataFileSchema)
             } else {
-              avroRecord
+              avroRec
             }
+
+            val hoodieKey = new HoodieKey(keyGenerator.getRecordKey(avroRec), keyGenerator.getPartitionPath(avroRec))
             val hoodieRecord = if (shouldCombine) {
-              val orderingVal = HoodieAvroUtils.getNestedFieldVal(avroRecord, config.getString(PRECOMBINE_FIELD),
+              val orderingVal = HoodieAvroUtils.getNestedFieldVal(avroRec, config.getString(PRECOMBINE_FIELD),
                 false, consistentLogicalTimestampEnabled).asInstanceOf[Comparable[_]]
-              DataSourceUtils.createHoodieRecord(processedRecord, orderingVal, keyGenerator.getKey(avroRecord),
+              DataSourceUtils.createHoodieRecord(processedRecord, orderingVal, hoodieKey,
                 config.getString(PAYLOAD_CLASS_NAME))
             } else {
-              DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(avroRecord),
+              DataSourceUtils.createHoodieRecord(processedRecord, hoodieKey,
                 config.getString(PAYLOAD_CLASS_NAME))
             }
             hoodieRecord
           }
         }).toJavaRDD()
 
       case HoodieRecord.HoodieRecordType.SPARK =>
-        val sparkKeyGenerator = keyGenerator.asInstanceOf[SparkKeyGeneratorInterface]
         val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
         val dataFileStructType = HoodieInternalRowUtils.getCachedSchema(dataFileSchema)
         val writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema)
         val sourceStructType = df.schema
 
         df.queryExecution.toRdd.mapPartitions { it =>
+          val sparkPartitionId = TaskContext.getPartitionId()
+          val keyGenProps = new TypedProperties(config.getProps)
+          if (autoGenerateRecordKeys) {
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(sparkPartitionId))
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime)
+          }
+          val sparkKeyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(keyGenProps).asInstanceOf[SparkKeyGeneratorInterface]
           val targetStructType = if (shouldDropPartitionColumns) dataFileStructType else writerStructType
           // NOTE: To make sure we properly transform records
           val targetStructTypeRowWriter = getCachedUnsafeRowWriter(sourceStructType, targetStructType)
 
-          it.map { sourceRow =>
-            val recordKey = sparkKeyGenerator.getRecordKey(sourceRow, sourceStructType)
+          val recordRecordKeyPairItr = it.map(avroRecord => {
+            (avroRecord, sparkKeyGenerator.getRecordKey(avroRecord, sourceStructType))
+          })
+

Review Comment:
   Guess this is not necessary change?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperKeyGenerator.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ * <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.
+ */
+public class AutoRecordGenWrapperKeyGenerator extends BuiltinKeyGenerator {
+
+  private final BuiltinKeyGenerator builtinKeyGenerator;
+  private final AtomicBoolean initializeAutoKeyGenProps = new AtomicBoolean(false);
+  private int partitionId;
+  private String instantTime;
+  private int rowId;
+
+  public AutoRecordGenWrapperKeyGenerator(TypedProperties config, BuiltinKeyGenerator builtinKeyGenerator) {
+    super(config);
+    this.builtinKeyGenerator = builtinKeyGenerator;
+  }
+
+  @Override
+  public String getRecordKey(GenericRecord record) {
+    initializeAutoKeyGenProps();
+    return HoodieRecord.generateSequenceId(instantTime, partitionId, rowId++);
+  }
+
+  private void initializeAutoKeyGenProps() {
+    if (!initializeAutoKeyGenProps.getAndSet(true)) {
+      this.rowId = 0;
+      this.partitionId = config.getInteger(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG);

Review Comment:
   Can we move it to the constructor?



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -609,28 +616,49 @@ private Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchFromSourc
     SerializableSchema avroSchema = new SerializableSchema(schemaProvider.getTargetSchema());
     SerializableSchema processedAvroSchema = new SerializableSchema(isDropPartitionColumns() ? HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get());
     if (recordType == HoodieRecordType.AVRO) {
-      records = avroRDD.map(record -> {
-        GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, partitionColumns) : record;
+      JavaRDD<Tuple2<GenericRecord, HoodieKey>> recordHoodieKeyPairRdd = avroRDD.mapPartitions(
+          (FlatMapFunction<Iterator<GenericRecord>, Tuple2<GenericRecord, HoodieKey>>) genericRecordIterator -> {
+            if (autoGenerateRecordKeys) {
+              props.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(TaskContext.getPartitionId()));
+              props.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime);
+            }
+            BuiltinKeyGenerator builtinKeyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
+            List<Tuple2<GenericRecord, HoodieKey>> recordWithRecordKeyOverride = new ArrayList<>();
+            while (genericRecordIterator.hasNext()) {
+              GenericRecord genRec = genericRecordIterator.next();
+              recordWithRecordKeyOverride.add(new Tuple2(genRec, new HoodieKey(builtinKeyGenerator.getRecordKey(genRec),
+                  builtinKeyGenerator.getPartitionPath(genRec))));
+            }
+            return recordWithRecordKeyOverride.iterator();
+          });
+
+      records = recordHoodieKeyPairRdd.map(recordHoodieKeyPair -> {

Review Comment:
   Still didn't quite get why we must get a (record, key) pair iterator first?



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537552167

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 31e6dc5d1d1d60a8d6465abbfb6549790861a476 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912) 
   * f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] nsivabalan commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1539183331

   I did ran some benchmarks. Confirmed that write latency is on par w/ 0.13.0 and with this patch w/o auto record key generation. and auto record key generation numbers are also on par. 


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1539325235

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "780318c5f048c4bf69980ac47d10d5e23994a21b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16954",
       "triggerID" : "780318c5f048c4bf69980ac47d10d5e23994a21b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 780318c5f048c4bf69980ac47d10d5e23994a21b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16954) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514] Adding auto generation of record keys support to Hudi

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1463234314

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 35aed635391309c3c6c4b3794044bba53b3468ef Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603) 
   * 9dfbe3e6135456e7f8c79513270eb5e7e4ed123d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1162074712


##########
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:
   synced up directly. we will take it up in a follow up patch where we will de-couple partition path generation and record key generation 
   Refer to https://github.com/apache/hudi/pull/7699
   [HUDI-5535](https://issues.apache.org/jira/browse/HUDI-5535).
   



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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1493566173

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070) 
   * 101db3163460b64d02c519d75ac75e04e85a3305 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1156413723


##########
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:
   for normal ingestion, we don't use empty string. I will respond to your question elsewhere (where we return empty string). its not very apparent. 
   



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131833317


##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "

Review Comment:
   yes and no.  We do infer key gen class. So, for simple, Complex and NonPartitioned, users don't need to explicitly set any key gen class. but for timestamp based or custom, they need to. 
   



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131845254


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("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 HoodieException("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 HoodieException("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.
+   * @return Iterator of Pair of GenericRecord and Optionally generated record key.
+   */
+  def mayBeAutoGenerateRecordKeys(autoGenerateKeys : Boolean, genRecsItr: Iterator[GenericRecord], instantTime: String): Iterator[(GenericRecord, Option[String])] = {
+    var rowId = 0
+    val sparkPartitionId = TaskContext.getPartitionId()
+
+    // 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

Review Comment:
   since these are executed from within a single executor. I didn't find any need. might confuse developers later if we change to Atomic. so, would prefer to keep it this way.



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


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

Posted by "codope (via GitHub)" <gi...@apache.org>.
codope commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1143727808


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1116,31 +1124,47 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys is auto generation is enabled.

Review Comment:
   ```suggestion
             // generate record keys if auto generation is enabled.
   ```



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.HoodieException
+import org.apache.spark.TaskContext
+
+object AutoRecordKeyGenerationUtils {
+
+   // supported operation types when auto generation of record keys is enabled.
+   val supportedOperations: Set[String] =
+    Set(WriteOperationType.INSERT, WriteOperationType.BULK_INSERT, WriteOperationType.DELETE,
+      WriteOperationType.INSERT_OVERWRITE, WriteOperationType.INSERT_OVERWRITE_TABLE,
+      WriteOperationType.DELETE_PARTITION).map(_.name())
+
+  def validateParamsForAutoGenerationOfRecordKeys(parameters: Map[String, String],
+                                                  operation: WriteOperationType, hoodieConfig: HoodieConfig): Unit = {
+    val autoGenerateRecordKeys: Boolean = parameters.getOrElse(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.key(),
+      HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS.defaultValue()).toBoolean
+
+    if (autoGenerateRecordKeys) {
+      // check for supported operations.
+      if (!supportedOperations.contains(operation.name())) {
+        throw new HoodieException(operation.name() + " is not supported with Auto generation of record keys. "
+          + "Supported operations are : " + supportedOperations)
+      }
+      // 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 HoodieException("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 HoodieException("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 HoodieException("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.
+   * @return Iterator of Pair of GenericRecord and Optionally generated record key.
+   */
+  def mayBeAutoGenerateRecordKeys(autoGenerateKeys : Boolean, genRecsItr: Iterator[GenericRecord], instantTime: String): Iterator[(GenericRecord, Option[String])] = {
+    var rowId = 0
+    val sparkPartitionId = TaskContext.getPartitionId()
+
+    // 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

Review Comment:
   We should check why it is Atomic in some of the write handles, eg. HoodieAppendHandle. If it's not needed then we can do away with this overhead.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -260,6 +260,18 @@ public class HoodieTableConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("The metadata of secondary indexes");
 
+  public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS = ConfigProperty
+      .key("hoodie.table.auto.generate.record.keys")
+      .defaultValue("false")
+      .withDocumentation("Enables automatic generation of the record-keys in cases when dataset bears "

Review Comment:
   Got it. Somehow, feels counter-intuitive that I enabled auto record keys and then also need to set keygen. But, I guess we can't do much here. We should think of a way to infer timestamp/custom keygen class based on how user set partitionpath field or other configs in `KeyGeneratorOptions.Config`.



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1488053579

   hey @vinothchandar : I thought about not making it a table property, but finding it hard. can you help me if there is a way out. 
   User still has to configure the right key gen class or we auto infer it. So, the table config will hold a key gen class (implr or timestmp, etc) explicitly irrespective of whether auto key gen is enabled or not. As I mentioned elsewhere in this patch(in response to sagar), hudis' key gen class is for both record key generation and partition path generation. So, atleast for partition path generation, users have to configure(we do auto infer in most of the cases, but we still make an entry to our table config)
   
   So, if we make the auto key gen a write config, it may not be easy for us to deduce whether the table had previous auto key gen enabled and if the user switches later? only way we can deduce this is to make it a table property. 
   
   
   


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1493920143

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 09d9feab5048d47a149f4088c23af9b5072250fa Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1494967724

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 09d9feab5048d47a149f4088c23af9b5072250fa Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077) 
   * d3e3d9ffd1bf60dabfb36d37133493683ea56a4c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1182499972


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -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
+
+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) {

Review Comment:
   In this case, the dupes will never be found, that's all right?  I don't see a reason to fail the writes per se. In general, we should avoid these interdependencies between configs. Can we WARN for these cases and let the write go through.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1118,31 +1124,47 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys if auto generation is enabled.
+          val recordsWithRecordKeyOverride = mayBeAutoGenerateRecordKeys(autoGenerateRecordKeys, it, instantTime, sparkPartitionId)
+
+          // handle dropping partition columns
+          recordsWithRecordKeyOverride.map { avroRecordRecordKeyOverRide =>
             val processedRecord = if (shouldDropPartitionColumns) {
-              HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema)
+              HoodieAvroUtils.rewriteRecord(avroRecordRecordKeyOverRide._1, dataFileSchema)
+            } else {
+              avroRecordRecordKeyOverRide._1
+            }
+
+            // Generate HoodieKey for records
+            val hoodieKey = if (autoGenerateRecordKeys) {
+              // fetch record key from the recordKeyOverride if auto generation is enabled.
+              new HoodieKey(avroRecordRecordKeyOverRide._2.get, keyGenerator.getKey(avroRecordRecordKeyOverRide._1).getPartitionPath)
             } else {
-              avroRecord
+              keyGenerator.getKey(avroRecordRecordKeyOverRide._1)
             }
+
             val hoodieRecord = if (shouldCombine) {
-              val orderingVal = HoodieAvroUtils.getNestedFieldVal(avroRecord, config.getString(PRECOMBINE_FIELD),
+              val orderingVal = HoodieAvroUtils.getNestedFieldVal(avroRecordRecordKeyOverRide._1, config.getString(PRECOMBINE_FIELD),
                 false, consistentLogicalTimestampEnabled).asInstanceOf[Comparable[_]]
-              DataSourceUtils.createHoodieRecord(processedRecord, orderingVal, keyGenerator.getKey(avroRecord),
+              DataSourceUtils.createHoodieRecord(processedRecord, orderingVal, hoodieKey,
                 config.getString(PAYLOAD_CLASS_NAME))
             } else {
-              DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(avroRecord),
+              DataSourceUtils.createHoodieRecord(processedRecord, hoodieKey,
                 config.getString(PAYLOAD_CLASS_NAME))
             }
             hoodieRecord
           }
         }).toJavaRDD()
 
       case HoodieRecord.HoodieRecordType.SPARK =>
+        // TODO: fix auto generation of record keys

Review Comment:
   should we do this for this path in the same PR? I am scared of introducing more divergences. 



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -1267,4 +1267,39 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
       })
     }
   }
+
+
+  test("Test Insert Into with auto generate record keys") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  dt string,
+           |  name string,
+           |  price double,
+           |  ts long
+           |) using hudi
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}'
+       """.stripMargin)
+
+      // Note: Do not write the field alias, the partition field must be placed last.
+      spark.sql(
+        s"""
+           | insert into $tableName values
+           | (1, 'a1', 10, 1000, "2021-01-05"),
+           | (2, 'a2', 20, 2000, "2021-01-06"),
+           | (3, 'a3', 30, 3000, "2021-01-07")
+              """.stripMargin)
+
+      checkAnswer(s"select id, name, price, ts, dt from $tableName")(

Review Comment:
   don't we have to read the keys out?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala:
##########
@@ -196,10 +196,12 @@ object HoodieOptionConfig {
     // validate primary key
     val primaryKeys = sqlOptions.get(SQL_KEY_TABLE_PRIMARY_KEY.sqlKeyName)
       .map(_.split(",").filter(_.length > 0))
-    ValidationUtils.checkArgument(primaryKeys.nonEmpty, "No `primaryKey` is specified.")
-    primaryKeys.get.foreach { primaryKey =>
-      ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, getRootLevelFieldName(primaryKey))),
-        s"Can't find primaryKey `$primaryKey` in ${schema.treeString}.")
+    // ValidationUtils.checkArgument(primaryKeys.nonEmpty, "No `primaryKey` is specified.")

Review Comment:
   remove



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -1267,4 +1267,39 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
       })
     }
   }
+
+
+  test("Test Insert Into with auto generate record keys") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      // Create a partitioned table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  dt string,
+           |  name string,
+           |  price double,
+           |  ts long
+           |) using hudi
+           | partitioned by (dt)
+           | location '${tmp.getCanonicalPath}'
+       """.stripMargin)
+
+      // Note: Do not write the field alias, the partition field must be placed last.
+      spark.sql(
+        s"""
+           | insert into $tableName values
+           | (1, 'a1', 10, 1000, "2021-01-05"),
+           | (2, 'a2', 20, 2000, "2021-01-06"),
+           | (3, 'a3', 30, 3000, "2021-01-07")
+              """.stripMargin)
+
+      checkAnswer(s"select id, name, price, ts, dt from $tableName")(

Review Comment:
   this test is not testing that the keys were actually generated and is unique. Love to see more comprehensive SQL tests, that include the entire set of DMLs. 



##########
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:
   Do we have alignment on the wrapping KeyGenerator? @nsivabalan what @danny0405  is saying is - have a `HoodieInternalKeyGenerator` class and do all these checks in that one file, while delegating to the inner key generator. Makes sense to me, will avoid touching all key generators. no?



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1184700259


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java:
##########
@@ -71,22 +73,31 @@ public static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) t
       throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
 
+    BaseKeyGenerator keyGenerator = null;
+
     switch (keyGeneratorTypeEnum) {
       case SIMPLE:
-        return new SimpleAvroKeyGenerator(props);
+        keyGenerator = new SimpleAvroKeyGenerator(props);
+        break;
       case COMPLEX:
-        return new ComplexAvroKeyGenerator(props);
+        keyGenerator = new ComplexAvroKeyGenerator(props);
+        break;
       case TIMESTAMP:
-        return new TimestampBasedAvroKeyGenerator(props);
+        keyGenerator = new TimestampBasedAvroKeyGenerator(props);
+        break;
       case CUSTOM:
-        return new CustomAvroKeyGenerator(props);
+        keyGenerator = new CustomAvroKeyGenerator(props);
+        break;
       case NON_PARTITION:
-        return new NonpartitionedAvroKeyGenerator(props);
+        keyGenerator = new NonpartitionedAvroKeyGenerator(props);
+        break;
       case GLOBAL_DELETE:
-        return new GlobalAvroDeleteKeyGenerator(props);
+        keyGenerator = new GlobalAvroDeleteKeyGenerator(props);
+        break;
       default:
         throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
+    return new HoodieWrapperAvroKeyGenerator(props, keyGenerator);
   }

Review Comment:
   Can we return the wrapper generator only when there is necessity, like when we are sure it is the key-less use case.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/HoodieWrapperKeyGenerator.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.StringUtils;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.List;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ */
+public class HoodieWrapperKeyGenerator extends BuiltinKeyGenerator {
+
+  private final BuiltinKeyGenerator builtinKeyGenerator;
+
+  public HoodieWrapperKeyGenerator(TypedProperties config, BuiltinKeyGenerator builtinKeyGenerator) {
+    super(config);
+    this.builtinKeyGenerator = builtinKeyGenerator;
+  }
+
+  @Override
+  public String getRecordKey(GenericRecord record) {
+    if (autoGenerateRecordKeys()) {
+      return StringUtils.EMPTY_STRING;
+    } else {

Review Comment:
   The `autoGenerateRecordKeys()` attribute should only restrict to this class: move it out from the base class.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1118,44 +1124,70 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys if auto generation is enabled.
+          val recordsWithRecordKeyOverride = mayBeAutoGenerateRecordKeys(autoGenerateRecordKeys, it, instantTime, sparkPartitionId)
+
+          // handle dropping partition columns
+          recordsWithRecordKeyOverride.map { avroRecordRecordKeyOverRide =>
             val processedRecord = if (shouldDropPartitionColumns) {
-              HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema)
+              HoodieAvroUtils.rewriteRecord(avroRecordRecordKeyOverRide._1, dataFileSchema)
+            } else {
+              avroRecordRecordKeyOverRide._1
+            }
+
+            // Generate HoodieKey for records

Review Comment:
   Can we conceal all the auto ken gen logic/changes just into the wrapper key generator, if you need the `TaskContext.getPartitionId()` as an auxiliary, just pass it into the wrapper key generator(may need some code refactoring here).



##########
hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java:
##########
@@ -63,10 +67,14 @@ protected BaseKeyGenerator(TypedProperties config) {
    */
   @Override
   public final HoodieKey getKey(GenericRecord record) {
-    if (getRecordKeyFieldNames() == null || getPartitionPathFields() == null) {
+    if (!autoGenerateRecordKeys() && (getRecordKeyFieldNames() == null || getPartitionPathFields() == null)) {
       throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
     }
-    return new HoodieKey(getRecordKey(record), getPartitionPath(record));
+    String recordKey = StringUtils.EMPTY_STRING;
+    if (!autoGenerateRecordKeys()) {

Review Comment:
   The base class should never introduce any key-less related changes.



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java:
##########
@@ -75,11 +75,6 @@ public void testNullPartitionPathFields() {
     Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutPartitionPathProp()));
   }
 
-  @Test
-  public void testNullRecordKeyFields() {
-    Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutRecordKeyProp()));
-  }

Review Comment:
   We need to keep these tests, just set the operation as `upsert` to make the tests pass.



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

Review Comment:
   Why we still keep the changes in these key generators? Shouldn't all the keyless realted changes only happen in the wrapper key generator?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/HoodieWrapperAvroKeyGenerator.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.StringUtils;
+
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.List;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ */
+public class HoodieWrapperAvroKeyGenerator extends BaseKeyGenerator {
+
+  private BaseKeyGenerator keyGenerator;
+
+  public HoodieWrapperAvroKeyGenerator(TypedProperties config, BaseKeyGenerator keyGenerator) {
+    super(config);
+    this.keyGenerator = keyGenerator;
+  }
+
+  @Override
+  public String getRecordKey(GenericRecord record) {
+    if (autoGenerateRecordKeys()) {
+      return StringUtils.EMPTY_STRING;

Review Comment:
   The `autoGenerateRecordKeys()` attribute should only restrict to this class: move it out from the base class.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala:
##########
@@ -150,8 +150,9 @@ object HoodieWriterUtils {
       if (null != tableConfig) {
         val datasourceRecordKey = params.getOrElse(RECORDKEY_FIELD.key(), null)
         val tableConfigRecordKey = tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)
-        if (null != datasourceRecordKey && null != tableConfigRecordKey
-          && datasourceRecordKey != tableConfigRecordKey) {
+        if ((null != datasourceRecordKey && null != tableConfigRecordKey
+          && datasourceRecordKey != tableConfigRecordKey) || (null != datasourceRecordKey && !datasourceRecordKey.isEmpty
+          && tableConfigRecordKey == null)) {
           diffConfigs.append(s"RecordKey:\t$datasourceRecordKey\t$tableConfigRecordKey\n")

Review Comment:
   Can we add some doc what the new condition represents?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -528,9 +527,13 @@ public String getPreCombineField() {
   }
 
   public Option<String[]> getRecordKeyFields() {
-    String keyFieldsValue = getStringOrDefault(RECORDKEY_FIELDS, HoodieRecord.RECORD_KEY_METADATA_FIELD);
-    return Option.of(Arrays.stream(keyFieldsValue.split(","))
-        .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {}));
+    String keyFieldsValue = getStringOrDefault(RECORDKEY_FIELDS, null);
+    if (keyFieldsValue == null) {
+      return Option.empty();

Review Comment:
   Should be cautious with this change, please make sure the invoker does not explicitly relies on the fetchiing of metadata field: `HoodieRecord.RECORD_KEY_METADATA_FIELD`.



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -610,14 +616,31 @@ private Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchFromSourc
     SerializableSchema avroSchema = new SerializableSchema(schemaProvider.getTargetSchema());
     SerializableSchema processedAvroSchema = new SerializableSchema(isDropPartitionColumns() ? HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get());
     if (recordType == HoodieRecordType.AVRO) {
-      records = avroRDD.map(record -> {
-        GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, partitionColumns) : record;
+      // if auto generation of keys is enabled, lets generate one
+      JavaRDD<Tuple2<GenericRecord, Option<String>>> recordRecordKeyOverrideRdd = avroRDD.mapPartitions(
+          (FlatMapFunction<Iterator<GenericRecord>, Tuple2<GenericRecord, Option<String>>>) genericRecordIterator -> {
+            int rowId = 0;
+            int sparkPartitionId = TaskContext.getPartitionId();
+            List<Tuple2<GenericRecord, Option<String>>> recordWithRecordKeyOverride = new ArrayList<>();
+            while (genericRecordIterator.hasNext()) {
+              GenericRecord genRec = genericRecordIterator.next();
+              Option<String> recordKeyOpt = autoGenerateRecordKeys ? Option.of(HoodieRecord.generateSequenceId(instantTime, sparkPartitionId, rowId++))
+                  : Option.empty();
+              recordWithRecordKeyOverride.add(new Tuple2(genRec, recordKeyOpt));
+            }

Review Comment:
   Ditto with `HoodieSparkSqlWriter`.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java:
##########
@@ -61,10 +61,11 @@ public class KeyGenUtils {
    */
   public static KeyGeneratorType inferKeyGeneratorType(
       String recordsKeyFields, String partitionFields) {
+    boolean autoGenerateRecordKeys = recordsKeyFields == null;
     if (!StringUtils.isNullOrEmpty(partitionFields)) {
       int numPartFields = partitionFields.split(",").length;
-      int numRecordKeyFields = recordsKeyFields.split(",").length;
-      if (numPartFields == 1 && numRecordKeyFields == 1) {
+      int numRecordKeyFields = autoGenerateRecordKeys ? 0 : recordsKeyFields.split(",").length;

Review Comment:
   Not sure why we need this change.



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537601087

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925",
       "triggerID" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185814578


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java:
##########
@@ -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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ * <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.
+ */
+public class AutoRecordGenWrapperAvroKeyGenerator extends BaseKeyGenerator {
+
+  private final BaseKeyGenerator keyGenerator;
+  private final AtomicBoolean initializeAutoKeyGenProps = new AtomicBoolean(false);
+  private int partitionId;
+  private String instantTime;
+  private int rowId;
+
+  public AutoRecordGenWrapperAvroKeyGenerator(TypedProperties config, BaseKeyGenerator keyGenerator) {
+    super(config);
+    this.keyGenerator = keyGenerator;
+  }
+
+  @Override
+  public String getRecordKey(GenericRecord record) {
+    initializeAutoKeyGenProps();
+    return HoodieRecord.generateSequenceId(instantTime, partitionId, rowId++);
+  }
+
+  private void initializeAutoKeyGenProps() {
+    if (!initializeAutoKeyGenProps.getAndSet(true)) {
+      this.rowId = 0;
+      this.partitionId = config.getInteger(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG);

Review Comment:
   Can the init code be moved to the constructor?



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537553376

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925",
       "triggerID" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 31e6dc5d1d1d60a8d6465abbfb6549790861a476 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912) 
   * f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1187565407


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java:
##########
@@ -71,22 +73,36 @@ public static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) t
       throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
 
+    BaseKeyGenerator keyGenerator = null;
+
     switch (keyGeneratorTypeEnum) {
       case SIMPLE:
-        return new SimpleAvroKeyGenerator(props);
+        keyGenerator = new SimpleAvroKeyGenerator(props);
+        break;
       case COMPLEX:
-        return new ComplexAvroKeyGenerator(props);
+        keyGenerator = new ComplexAvroKeyGenerator(props);
+        break;
       case TIMESTAMP:
-        return new TimestampBasedAvroKeyGenerator(props);
+        keyGenerator = new TimestampBasedAvroKeyGenerator(props);
+        break;
       case CUSTOM:
-        return new CustomAvroKeyGenerator(props);
+        keyGenerator = new CustomAvroKeyGenerator(props);
+        break;
       case NON_PARTITION:
-        return new NonpartitionedAvroKeyGenerator(props);
+        keyGenerator = new NonpartitionedAvroKeyGenerator(props);
+        break;
       case GLOBAL_DELETE:
-        return new GlobalAvroDeleteKeyGenerator(props);
+        keyGenerator = new GlobalAvroDeleteKeyGenerator(props);
+        break;
       default:
         throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
+
+    if (KeyGenUtils.enableAutoGenerateRecordKeys(props)) {

Review Comment:
   thats what my preference was, but Danny preferred to keep it out. just instantiate the wrapper only if required. so that we don't make any changes to the non auto record key gen code path. 
   



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1538996828

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925",
       "triggerID" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fcd81e9fda3e3b6fa7e813d6c56b880042b90bed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16934",
       "triggerID" : "fcd81e9fda3e3b6fa7e813d6c56b880042b90bed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "780318c5f048c4bf69980ac47d10d5e23994a21b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "780318c5f048c4bf69980ac47d10d5e23994a21b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fcd81e9fda3e3b6fa7e813d6c56b880042b90bed Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16934) 
   * 780318c5f048c4bf69980ac47d10d5e23994a21b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514] Adding auto generation of record keys support to Hudi

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1457390768

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 35aed635391309c3c6c4b3794044bba53b3468ef UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537871642

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925",
       "triggerID" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fcd81e9fda3e3b6fa7e813d6c56b880042b90bed",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fcd81e9fda3e3b6fa7e813d6c56b880042b90bed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925) 
   * fcd81e9fda3e3b6fa7e813d6c56b880042b90bed UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1538141494

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925",
       "triggerID" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fcd81e9fda3e3b6fa7e813d6c56b880042b90bed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16934",
       "triggerID" : "fcd81e9fda3e3b6fa7e813d6c56b880042b90bed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fcd81e9fda3e3b6fa7e813d6c56b880042b90bed Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16934) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537881352

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925",
       "triggerID" : "f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fcd81e9fda3e3b6fa7e813d6c56b880042b90bed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16934",
       "triggerID" : "fcd81e9fda3e3b6fa7e813d6c56b880042b90bed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6b8d65052236818a43eb9a81efe7b6bdd9ff1b8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16925) 
   * fcd81e9fda3e3b6fa7e813d6c56b880042b90bed Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16934) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1187547491


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.List;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ * <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.
+ */
+public class AutoRecordGenWrapperAvroKeyGenerator extends BaseKeyGenerator {

Review Comment:
   rename `WrappingKeyGenerator` to make it easier to read? its best to decouple names for broad implementations like these from specific features for which they are added.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java:
##########
@@ -55,7 +57,11 @@ public enum PartitionKeyType {
 
   public CustomAvroKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")).map(String::trim).collect(Collectors.toList());
+    this.recordKeyFields = Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null))

Review Comment:
   please move this logic into an util method and not repeat 5 lines in each file?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java:
##########
@@ -71,22 +73,36 @@ public static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) t
       throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
 
+    BaseKeyGenerator keyGenerator = null;
+
     switch (keyGeneratorTypeEnum) {
       case SIMPLE:
-        return new SimpleAvroKeyGenerator(props);
+        keyGenerator = new SimpleAvroKeyGenerator(props);
+        break;
       case COMPLEX:
-        return new ComplexAvroKeyGenerator(props);
+        keyGenerator = new ComplexAvroKeyGenerator(props);
+        break;
       case TIMESTAMP:
-        return new TimestampBasedAvroKeyGenerator(props);
+        keyGenerator = new TimestampBasedAvroKeyGenerator(props);
+        break;
       case CUSTOM:
-        return new CustomAvroKeyGenerator(props);
+        keyGenerator = new CustomAvroKeyGenerator(props);
+        break;
       case NON_PARTITION:
-        return new NonpartitionedAvroKeyGenerator(props);
+        keyGenerator = new NonpartitionedAvroKeyGenerator(props);
+        break;
       case GLOBAL_DELETE:
-        return new GlobalAvroDeleteKeyGenerator(props);
+        keyGenerator = new GlobalAvroDeleteKeyGenerator(props);
+        break;
       default:
         throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
+
+    if (KeyGenUtils.enableAutoGenerateRecordKeys(props)) {

Review Comment:
   should we always use the wrapper key generator and move all this logic into that class?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala:
##########
@@ -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
+
+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) {

Review Comment:
   This needs resolution?



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1538168229

   Should be a flaky test: https://dev.azure.com/apache-hudi-ci-org/apache-hudi-ci/_build/results?buildId=16934&view=logs&j=600e7de6-e133-5e69-e615-50ee129b3c08&t=bbbd7bcc-ae73-56b8-887a-cd2d6deaafc7&l=15297


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1493562355

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023) 
   * 96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070) 
   * 101db3163460b64d02c519d75ac75e04e85a3305 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1494975080

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 09d9feab5048d47a149f4088c23af9b5072250fa Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077) 
   * d3e3d9ffd1bf60dabfb36d37133493683ea56a4c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1498252269

   Follow up patch : https://github.com/apache/hudi/pull/8389
   


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1535707257

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd5d81ac4fc008709c4e34438c3242dcb18c6161 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819) 
   * 1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1536026321

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 83166a873b87ed702f83fb91aed89eba4d322f51 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856) 
   * 50651d141172716f7605c19cdf69ce0fdda07942 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859) 
   * ce3d13748cd488df7e055392f2e9db4ac2bfc18b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] nsivabalan commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1536870362

   @danny0405 : all feedback has been addressed


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537085829

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b5b72b6043cc2a110d175b8f2b69b9e38902359d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876) 
   * fd6092c26ef67021d43a4b7b663d744933a20e06 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185065540


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1118,44 +1124,70 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys if auto generation is enabled.
+          val recordsWithRecordKeyOverride = mayBeAutoGenerateRecordKeys(autoGenerateRecordKeys, it, instantTime, sparkPartitionId)
+
+          // handle dropping partition columns
+          recordsWithRecordKeyOverride.map { avroRecordRecordKeyOverRide =>
             val processedRecord = if (shouldDropPartitionColumns) {
-              HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema)
+              HoodieAvroUtils.rewriteRecord(avroRecordRecordKeyOverRide._1, dataFileSchema)
+            } else {
+              avroRecordRecordKeyOverRide._1
+            }
+
+            // Generate HoodieKey for records

Review Comment:
   nope. as I called out, keyGen class is meant to be called for each record separately. But here we call it within mapPartitions call. and we generate it based on rowId. 
   Without touching the apis for key gen class, nor the constructor of key gen instantiation, it might be tough. 
   but lets chat to see if we can do it. 
   I would also love to conceal or keep it in one place if feasible. 
   



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1535895915

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 83166a873b87ed702f83fb91aed89eba4d322f51 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856) 
   * 50651d141172716f7605c19cdf69ce0fdda07942 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1162232618


##########
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:
   Agree, we can take a refactoring in another patch for separating record key and partition path generation. What I'm suggesting is orthogonal to the refactoring and it minimize the changes, can we do that first in this patch?



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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1488096549

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9dfbe3e6135456e7f8c79513270eb5e7e4ed123d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648) 
   * 0d733095ce4e9342901a03f0530c94c373593d15 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185056314


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

Review Comment:
   yes, unfortunately, the way we parse record keys is specific to each key gen class and is not generalized in any base class. Even then, the base class needs some fixes since record key was deemed mandatory before and now w/ this patch, it is not. 
   I will work on a follow up patch after landing this (do not want to drag this further) to fix the parsing or record keys. 
   https://issues.apache.org/jira/browse/HUDI-6172
   



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537269501

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd6092c26ef67021d43a4b7b663d744933a20e06 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893) 
   * e49fef0e72149160ffe124636e7c89d1ebe97e18 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537340763

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e49fef0e72149160ffe124636e7c89d1ebe97e18 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905) 
   * 31e6dc5d1d1d60a8d6465abbfb6549790861a476 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131840399


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java:
##########
@@ -44,6 +47,11 @@ public ComplexAvroKeyGenerator(TypedProperties props) {
 
   @Override
   public String getRecordKey(GenericRecord record) {
+    if (doAutoGenerateRecordKeys()) {

Review Comment:
   nope. if not for auto record key generation, record key field cannot be empty. even if its, we should let KeyGenUtils.getRecordKey() handle 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-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514] Adding auto generation of record keys support to Hudi

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1463230758

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 35aed635391309c3c6c4b3794044bba53b3468ef Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603) 
   * 9dfbe3e6135456e7f8c79513270eb5e7e4ed123d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1534269582

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e03d1c8712b0ae5dce7743a2588ee6b606cc562d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177) 
   * dd5d81ac4fc008709c4e34438c3242dcb18c6161 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1535763170

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852) 
   * 83166a873b87ed702f83fb91aed89eba4d322f51 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185753812


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala:
##########
@@ -844,8 +844,8 @@ object DataSourceOptionsHelper {
    */
   def fetchMissingWriteConfigsFromTableConfig(tableConfig: HoodieTableConfig, params: Map[String, String]) : Map[String, String] = {
     val missingWriteConfigs = scala.collection.mutable.Map[String, String]()
-    if (!params.contains(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) && tableConfig.getRecordKeyFieldProp != null) {
-      missingWriteConfigs ++= Map(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key() -> tableConfig.getRecordKeyFieldProp)
+    if (!params.contains(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) && tableConfig.getRawRecordKeyFieldProp != null) {
+      missingWriteConfigs ++= Map(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key() -> tableConfig.getRawRecordKeyFieldProp)

Review Comment:
   not sure of your comment. lets sync up



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1159253894


##########
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:
   I'm wondering whether we can add a new specific key generator for the keyless scenario, this new key generator is a wrapper for the existing key generator, and it masks out some invocations like:
   
   1. `#getRecordKey`, throws exception directly
   2. `#getKey`, returns a empty string for the HoodieKey record key
   
   This could avoid to introduce potential bugs and is very straight-forward.



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537087402

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b5b72b6043cc2a110d175b8f2b69b9e38902359d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876) 
   * fd6092c26ef67021d43a4b7b663d744933a20e06 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1536035998

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 50651d141172716f7605c19cdf69ce0fdda07942 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859) 
   * ce3d13748cd488df7e055392f2e9db4ac2bfc18b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537182352

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd6092c26ef67021d43a4b7b663d744933a20e06 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] nsivabalan commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1538513241

   have addressed 
   ```
   Can we move the check if (autoGenerateRecordKeys) { out of the method?
   ```
   renamed the method to mayBeValidateParamsForAutoGenerationOfRecordKeys. and Danny is ok with 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-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1539067675

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "780318c5f048c4bf69980ac47d10d5e23994a21b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16954",
       "triggerID" : "780318c5f048c4bf69980ac47d10d5e23994a21b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 780318c5f048c4bf69980ac47d10d5e23994a21b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16954) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1499806398

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 711df161776bfbe4f66cb04310eb15ccc0069716 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110) 
   * e03d1c8712b0ae5dce7743a2588ee6b606cc562d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131831680


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,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)

Review Comment:
   from my testing, the ordering within spark partition does not change.



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131831306


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -82,9 +85,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

Review Comment:
   nope. that may not guarantee uniqueness.



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1151439496


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java:
##########
@@ -29,24 +31,29 @@
 public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   public SimpleAvroKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
+    this(props, Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null)),

Review Comment:
   we use TypedProperties and got getString, we call checkKey() which might throw exception if key is not found
   https://github.com/apache/hudi/blob/7243393c6881802803c0233cbac42daf1271afb3/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java#L72
   
   and hence. 



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1536886929

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce3d13748cd488df7e055392f2e9db4ac2bfc18b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861) 
   * b5b72b6043cc2a110d175b8f2b69b9e38902359d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1536956006

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b5b72b6043cc2a110d175b8f2b69b9e38902359d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1535750349

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd5d81ac4fc008709c4e34438c3242dcb18c6161 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819) 
   * 1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852) 
   * 83166a873b87ed702f83fb91aed89eba4d322f51 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1535886997

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852) 
   * 83166a873b87ed702f83fb91aed89eba4d322f51 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856) 
   * 50651d141172716f7605c19cdf69ce0fdda07942 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537270553

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd6092c26ef67021d43a4b7b663d744933a20e06 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893) 
   * e49fef0e72149160ffe124636e7c89d1ebe97e18 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185674744


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java:
##########
@@ -61,10 +61,11 @@ public class KeyGenUtils {
    */
   public static KeyGeneratorType inferKeyGeneratorType(
       String recordsKeyFields, String partitionFields) {
+    boolean autoGenerateRecordKeys = recordsKeyFields == null;
     if (!StringUtils.isNullOrEmpty(partitionFields)) {
       int numPartFields = partitionFields.split(",").length;
-      int numRecordKeyFields = recordsKeyFields.split(",").length;
-      if (numPartFields == 1 && numRecordKeyFields == 1) {
+      int numRecordKeyFields = autoGenerateRecordKeys ? 0 : recordsKeyFields.split(",").length;

Review Comment:
   yet to address this.



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -83,8 +87,19 @@ object HoodieDatasetBulkInsertHelper
             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

Review Comment:
   yet to fix this 



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


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

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1521854219

   @danny0405 how easy would this be to do if for the Flink code paths?


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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537657119

   Looks good! It is great if we can move the decision `
   !props.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
   into a tool method in KeyGenUtils.autoGenerateRecordKeys(TypedProperties props) , the benefit is if someday we wanna change the keyless precondition, only one modification can do the trick.


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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537342752

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e49fef0e72149160ffe124636e7c89d1ebe97e18 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905) 
   * 31e6dc5d1d1d60a8d6465abbfb6549790861a476 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1187818121


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.List;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ * <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.
+ */
+public class AutoRecordGenWrapperAvroKeyGenerator extends BaseKeyGenerator {

Review Comment:
   the rational behind this naming is, we use this key gen only when auto generation of record keys are enabled. if not, we do go go through this key gen class. 
   will sync up w/ Danny. We had consensus to keep the non auto record key gen not go via this class. 
   



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1540027454

   Tests have passed: https://dev.azure.com/apache-hudi-ci-org/apache-hudi-ci/_build/results?buildId=16954&view=results


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185062811


##########
hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java:
##########
@@ -63,10 +67,14 @@ protected BaseKeyGenerator(TypedProperties config) {
    */
   @Override
   public final HoodieKey getKey(GenericRecord record) {
-    if (getRecordKeyFieldNames() == null || getPartitionPathFields() == null) {
+    if (!autoGenerateRecordKeys() && (getRecordKeyFieldNames() == null || getPartitionPathFields() == null)) {
       throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
     }
-    return new HoodieKey(getRecordKey(record), getPartitionPath(record));
+    String recordKey = StringUtils.EMPTY_STRING;
+    if (!autoGenerateRecordKeys()) {

Review Comment:
   responded for another comment



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1539058174

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "780318c5f048c4bf69980ac47d10d5e23994a21b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "780318c5f048c4bf69980ac47d10d5e23994a21b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 780318c5f048c4bf69980ac47d10d5e23994a21b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185065540


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1118,44 +1124,70 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys if auto generation is enabled.
+          val recordsWithRecordKeyOverride = mayBeAutoGenerateRecordKeys(autoGenerateRecordKeys, it, instantTime, sparkPartitionId)
+
+          // handle dropping partition columns
+          recordsWithRecordKeyOverride.map { avroRecordRecordKeyOverRide =>
             val processedRecord = if (shouldDropPartitionColumns) {
-              HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema)
+              HoodieAvroUtils.rewriteRecord(avroRecordRecordKeyOverRide._1, dataFileSchema)
+            } else {
+              avroRecordRecordKeyOverRide._1
+            }
+
+            // Generate HoodieKey for records

Review Comment:
   nope. as I called out, keyGen class is meant to be called for each record separately. But here we call it within mapPartitions call. and we generate it based on rowId. 
   Without touching the apis for key gen class, nor the constructor of key gen instantiation, it might be tough. 
   but lets chat to see if we can do 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-unsubscribe@hudi.apache.org

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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185062182


##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -528,9 +527,13 @@ public String getPreCombineField() {
   }
 
   public Option<String[]> getRecordKeyFields() {
-    String keyFieldsValue = getStringOrDefault(RECORDKEY_FIELDS, HoodieRecord.RECORD_KEY_METADATA_FIELD);
-    return Option.of(Arrays.stream(keyFieldsValue.split(","))
-        .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {}));
+    String keyFieldsValue = getStringOrDefault(RECORDKEY_FIELDS, null);
+    if (keyFieldsValue == null) {
+      return Option.empty();

Review Comment:
   yes. agreed. 



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185051024


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java:
##########
@@ -71,22 +73,31 @@ public static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) t
       throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
 
+    BaseKeyGenerator keyGenerator = null;
+
     switch (keyGeneratorTypeEnum) {
       case SIMPLE:
-        return new SimpleAvroKeyGenerator(props);
+        keyGenerator = new SimpleAvroKeyGenerator(props);
+        break;
       case COMPLEX:
-        return new ComplexAvroKeyGenerator(props);
+        keyGenerator = new ComplexAvroKeyGenerator(props);
+        break;
       case TIMESTAMP:
-        return new TimestampBasedAvroKeyGenerator(props);
+        keyGenerator = new TimestampBasedAvroKeyGenerator(props);
+        break;
       case CUSTOM:
-        return new CustomAvroKeyGenerator(props);
+        keyGenerator = new CustomAvroKeyGenerator(props);
+        break;
       case NON_PARTITION:
-        return new NonpartitionedAvroKeyGenerator(props);
+        keyGenerator = new NonpartitionedAvroKeyGenerator(props);
+        break;
       case GLOBAL_DELETE:
-        return new GlobalAvroDeleteKeyGenerator(props);
+        keyGenerator = new GlobalAvroDeleteKeyGenerator(props);
+        break;
       default:
         throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
+    return new HoodieWrapperAvroKeyGenerator(props, keyGenerator);
   }

Review Comment:
   then, wouldn't it defeat the purpose of using the delegator pattern. the wrapper key gen should take care of either delegating or serving it by itself (get record keys for auto key gen use-case). 
   



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1532483205

   > Need closure on wrapping the key generator impl as @danny0405 was suggesting, among other things.
   > 
   > @danny0405 we need to do this for Flink as well. thoughts?
   
   Flink already impl the keyless use case for a whole diggerent code path, it is already wrapping the key generator in current code base.


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185756163


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1118,47 +1124,74 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+          val keyGenProps = new TypedProperties(config.getProps)
+          if (autoGenerateRecordKeys) {
+            keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(sparkPartitionId))

Review Comment:
   we can only move the declaration. 
   for eg, adding spark partition Id has to be done from within map partition call. 



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1186672625


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala:
##########
@@ -41,14 +41,31 @@ object SparkKeyGenUtils {
    * @return partition columns
    */
   def getPartitionColumns(keyGenClass: String, typedProperties: TypedProperties): String = {
-
-    if (keyGenClass.equals(classOf[CustomKeyGenerator].getCanonicalName) || keyGenClass.equals(classOf[CustomAvroKeyGenerator])) {
+    // For CustomKeyGenerator and CustomAvroKeyGenerator, the partition path filed format
+    // is: "field_name: field_type", we extract the field_name from the partition path field.
+    if (keyGenClass.equals(classOf[CustomKeyGenerator].getCanonicalName) || keyGenClass.equals(classOf[CustomAvroKeyGenerator].getCanonicalName)) {
       typedProperties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())
         .split(",").map(pathField => {
         pathField.split(CustomAvroKeyGenerator.SPLIT_REGEX)
           .headOption.getOrElse(s"Illegal partition path field format: '$pathField' for ${keyGenClass}")}).mkString(",")
     } else {
-      typedProperties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())
+      typedProperties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "")
+    }
+  }
+
+  def getPartitionColumns(keyGen: KeyGenerator, typedProperties: TypedProperties): String = {
+    keyGen match {
+      // For CustomKeyGenerator and CustomAvroKeyGenerator, the partition path filed format

Review Comment:
   Do we need this? Also fix the useless class imported.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java:
##########
@@ -157,12 +155,6 @@ private String getPartitionPath(Option<GenericRecord> record, Option<Row> row, O
     return partitionPath.toString();
   }
 
-  private void validateRecordKeyFields() {
-    if (getRecordKeyFieldNames() == null || getRecordKeyFieldNames().isEmpty()) {
-      throw new HoodieKeyException("Unable to find field names for record key in cfg");
-    }

Review Comment:
   If we know the user set up `UPSERT` operation explicitly, we can keep this validation. Or maybe we just move the validation to the kengen factory class.



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java:
##########
@@ -240,62 +240,6 @@ public void testInvalidPartitionKeyType(TypedProperties props) {
     }
   }
 
-  @Test
-  public void testNoRecordKeyFieldPropWithKeyGeneratorClass() {
-    testNoRecordKeyFieldProp(true);
-  }
-
-  @Test
-  public void testNoRecordKeyFieldPropWithKeyGeneratorType() {
-    testNoRecordKeyFieldProp(false);
-  }
-
-  public void testNoRecordKeyFieldProp(boolean useKeyGeneratorClassName) {
-    TypedProperties propsWithoutRecordKeyFieldProps = getPropsWithoutRecordKeyFieldProps(useKeyGeneratorClassName);
-    try {
-      BuiltinKeyGenerator keyGenerator =
-          (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(propsWithoutRecordKeyFieldProps);
-
-      keyGenerator.getKey(getRecord());
-      Assertions.fail("should fail when record key field is not provided!");

Review Comment:
   We can not remove all the non-key validation for the code, we still need to keep tests for `UPSERT` and null keys validation.



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1535699791

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd5d81ac4fc008709c4e34438c3242dcb18c6161 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819) 
   * 1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185773877


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -867,18 +894,20 @@ private Pair<Option<String>, JavaRDD<WriteStatus>> writeToSink(JavaRDD<HoodieRec
    *
    * @return Instant time of the commit
    */
-  private String startCommit() {
+  private String startCommit(String instantTime, boolean retryEnabled) {

Review Comment:
   yes, this is required since we need instant time upfront. 



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1185751769


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala:
##########
@@ -79,11 +83,20 @@ object HoodieDatasetBulkInsertHelper
 
       val prependedRdd: RDD[InternalRow] =
         df.queryExecution.toRdd.mapPartitions { iter =>
-          val keyGenerator =
+          val sparkKeyGenerator =
             ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps))
-              .asInstanceOf[SparkKeyGeneratorInterface]
+              .asInstanceOf[BuiltinKeyGenerator]
+              val keyGenerator: BuiltinKeyGenerator = if (autoGenerateRecordKeys) {
+                val typedProps = new TypedProperties(config.getProps)
+                typedProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(TaskContext.getPartitionId()))
+                typedProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime)
+                new AutoRecordGenWrapperKeyGenerator(typedProps, sparkKeyGenerator).asInstanceOf[BuiltinKeyGenerator]

Review Comment:
   yeah. I could not recollect the reason as well. 



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514][HUDI-5574][HUDI-5604][HUDI-5535] Adding auto generation of record keys support to Hudi/Spark

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1537376890

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16819",
       "triggerID" : "dd5d81ac4fc008709c4e34438c3242dcb18c6161",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16852",
       "triggerID" : "1e7e4f039fccbfc87a4dfab0cf4628b8a3adaf12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16856",
       "triggerID" : "83166a873b87ed702f83fb91aed89eba4d322f51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16859",
       "triggerID" : "50651d141172716f7605c19cdf69ce0fdda07942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16861",
       "triggerID" : "ce3d13748cd488df7e055392f2e9db4ac2bfc18b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16876",
       "triggerID" : "b5b72b6043cc2a110d175b8f2b69b9e38902359d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16893",
       "triggerID" : "fd6092c26ef67021d43a4b7b663d744933a20e06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16905",
       "triggerID" : "e49fef0e72149160ffe124636e7c89d1ebe97e18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912",
       "triggerID" : "31e6dc5d1d1d60a8d6465abbfb6549790861a476",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 31e6dc5d1d1d60a8d6465abbfb6549790861a476 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16912) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1187566953


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java:
##########
@@ -71,22 +73,36 @@ public static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) t
       throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
 
+    BaseKeyGenerator keyGenerator = null;
+
     switch (keyGeneratorTypeEnum) {
       case SIMPLE:
-        return new SimpleAvroKeyGenerator(props);
+        keyGenerator = new SimpleAvroKeyGenerator(props);
+        break;
       case COMPLEX:
-        return new ComplexAvroKeyGenerator(props);
+        keyGenerator = new ComplexAvroKeyGenerator(props);
+        break;
       case TIMESTAMP:
-        return new TimestampBasedAvroKeyGenerator(props);
+        keyGenerator = new TimestampBasedAvroKeyGenerator(props);
+        break;
       case CUSTOM:
-        return new CustomAvroKeyGenerator(props);
+        keyGenerator = new CustomAvroKeyGenerator(props);
+        break;
       case NON_PARTITION:
-        return new NonpartitionedAvroKeyGenerator(props);
+        keyGenerator = new NonpartitionedAvroKeyGenerator(props);
+        break;
       case GLOBAL_DELETE:
-        return new GlobalAvroDeleteKeyGenerator(props);
+        keyGenerator = new GlobalAvroDeleteKeyGenerator(props);
+        break;
       default:
         throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
+
+    if (KeyGenUtils.enableAutoGenerateRecordKeys(props)) {

Review Comment:
   We discussed this and think it is not necessary, we should not add any overhead for the UPSERT code path, like deciding the primary key and also has less risk to introduce new bugs.



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1187567855


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java:
##########
@@ -71,22 +73,36 @@ public static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) t
       throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
 
+    BaseKeyGenerator keyGenerator = null;
+
     switch (keyGeneratorTypeEnum) {
       case SIMPLE:
-        return new SimpleAvroKeyGenerator(props);
+        keyGenerator = new SimpleAvroKeyGenerator(props);
+        break;
       case COMPLEX:
-        return new ComplexAvroKeyGenerator(props);
+        keyGenerator = new ComplexAvroKeyGenerator(props);
+        break;
       case TIMESTAMP:
-        return new TimestampBasedAvroKeyGenerator(props);
+        keyGenerator = new TimestampBasedAvroKeyGenerator(props);
+        break;
       case CUSTOM:
-        return new CustomAvroKeyGenerator(props);
+        keyGenerator = new CustomAvroKeyGenerator(props);
+        break;
       case NON_PARTITION:
-        return new NonpartitionedAvroKeyGenerator(props);
+        keyGenerator = new NonpartitionedAvroKeyGenerator(props);
+        break;
       case GLOBAL_DELETE:
-        return new GlobalAvroDeleteKeyGenerator(props);
+        keyGenerator = new GlobalAvroDeleteKeyGenerator(props);
+        break;
       default:
         throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
     }
+
+    if (KeyGenUtils.enableAutoGenerateRecordKeys(props)) {

Review Comment:
   @danny0405 : can u chime in



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1186019885


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java:
##########
@@ -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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A wrapper key generator to intercept getRecordKey calls for auto record key generator.
+ * <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.
+ */
+public class AutoRecordGenWrapperAvroKeyGenerator extends BaseKeyGenerator {
+
+  private final BaseKeyGenerator keyGenerator;
+  private final AtomicBoolean initializeAutoKeyGenProps = new AtomicBoolean(false);
+  private int partitionId;
+  private String instantTime;
+  private int rowId;
+
+  public AutoRecordGenWrapperAvroKeyGenerator(TypedProperties config, BaseKeyGenerator keyGenerator) {
+    super(config);
+    this.keyGenerator = keyGenerator;
+  }
+
+  @Override
+  public String getRecordKey(GenericRecord record) {
+    initializeAutoKeyGenProps();
+    return HoodieRecord.generateSequenceId(instantTime, partitionId, rowId++);
+  }
+
+  private void initializeAutoKeyGenProps() {
+    if (!initializeAutoKeyGenProps.getAndSet(true)) {
+      this.rowId = 0;
+      this.partitionId = config.getInteger(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG);

Review Comment:
   This comment is not addressed.



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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1154469420


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -1145,6 +1145,10 @@ public String getKeyGeneratorClass() {
     return getString(KEYGENERATOR_CLASS_NAME);
   }
 
+  public boolean autoGenerateRecordKeys() {
+    return getBooleanOrDefault(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS);
+  }

Review Comment:
   yes, I had a jam w/ vinoth yesterday. we are going to rely on user configuring record key. If its configured we will honor it. if not, we will auto generate it. wrt operation type, its little involved. We have to decide the default operation type based on whether use has configured other props or not. 
   
   my high level goal here is:
   for someone using plain parquet (writing to parquet table), when they switch to hudi, we want to give them smoother transition. 
   
   so, df.wirte.format("parquet").save(location)
   should be easily translatable to 
   df.wirte.format("hudi").save(newLocation).
   
   along w/ it, we can infer partitions if 
   df.write.partitionBy(colA).format("hudi").save(newLocation). 
   
   So, in these cases(no precombine), we should make insert as the default operation type. In other cases, we will make upsert as default operation type (only when user has not overridden the operation type). 
   
   
   
   
   
   
   
   



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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1492423459

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1499907177

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15648",
       "triggerID" : "9dfbe3e6135456e7f8c79513270eb5e7e4ed123d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15968",
       "triggerID" : "0d733095ce4e9342901a03f0530c94c373593d15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16023",
       "triggerID" : "2cc8e7a0ef019790e5d7fb8890ac73ec6ffc1054",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16070",
       "triggerID" : "96adaab5efe3a7b2dca2cf2356d5b6bb89d9829a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16073",
       "triggerID" : "101db3163460b64d02c519d75ac75e04e85a3305",
       "triggerType" : "PUSH"
     }, {
       "hash" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16077",
       "triggerID" : "09d9feab5048d47a149f4088c23af9b5072250fa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16100",
       "triggerID" : "d3e3d9ffd1bf60dabfb36d37133493683ea56a4c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "572189472623065f460bd18436fb3b21602449af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16101",
       "triggerID" : "572189472623065f460bd18436fb3b21602449af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16110",
       "triggerID" : "711df161776bfbe4f66cb04310eb15ccc0069716",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177",
       "triggerID" : "e03d1c8712b0ae5dce7743a2588ee6b606cc562d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e03d1c8712b0ae5dce7743a2588ee6b606cc562d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16177) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


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

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1153979756


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -1145,6 +1145,10 @@ public String getKeyGeneratorClass() {
     return getString(KEYGENERATOR_CLASS_NAME);
   }
 
+  public boolean autoGenerateRecordKeys() {
+    return getBooleanOrDefault(HoodieTableConfig.AUTO_GENERATE_RECORD_KEYS);
+  }

Review Comment:
   Can we eliminate this option? We can infter keyless ingestion from 2 other options:
   
   1. the write operation is defined as `INSERT`;
   2. or user does not declare any `record.key` fields.



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131837703


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java:
##########
@@ -44,6 +47,11 @@ public ComplexAvroKeyGenerator(TypedProperties props) {
 
   @Override
   public String getRecordKey(GenericRecord record) {
+    if (doAutoGenerateRecordKeys()) {
+      // To fetch partition path, caller will have to call getKey() on KeyGenerator and call .getPartitionPath. Hence we have to pass empty field to support
+      // returning partition path for the callers.

Review Comment:
   yes, unfortunately thats how it is. bcoz, oru KeyGenerator abstract class exposes only getter of HoodieKey. and hence to fetch record keys, callers call getKey(GenRec).getPartitionPath. 
   We have a ticket to fix some code refactoring around record key generation and partition path. 
   For eg refer to https://github.com/apache/hudi/pull/7699
   HUDI-5535. 
   



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131839536


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

Review Comment:
   when auto generation of record key is enabled, we should not be invoking KeyGenUtils#getRecordKey since keys are auto generated. so, I guess we should be ok. 



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


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

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8107:
URL: https://github.com/apache/hudi/pull/8107#discussion_r1131845834


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1096,31 +1104,47 @@ object HoodieSparkSqlWriter {
           Some(writerSchema))
 
         avroRecords.mapPartitions(it => {
+          val sparkPartitionId = TaskContext.getPartitionId()
+
           val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr)
           val consistentLogicalTimestampEnabled = parameters.getOrElse(
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
             DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean
 
-          it.map { avroRecord =>
+          // generate record keys is auto generation is enabled.
+          val recordsWithRecordKeyOverride = mayBeAutoGenerateRecordKeys(autoGenerateRecordKeys, it, instantTime)
+
+          // handle dropping partition columns
+          recordsWithRecordKeyOverride.map { avroRecordRecordKeyOverRide =>
             val processedRecord = if (shouldDropPartitionColumns) {
-              HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema)
+              HoodieAvroUtils.rewriteRecord(avroRecordRecordKeyOverRide._1, dataFileSchema)
+            } else {
+              avroRecordRecordKeyOverRide._1
+            }
+
+            // Generate HoodieKey for records
+            val hoodieKey = if (autoGenerateRecordKeys) {
+              // fetch record key from the recordKeyOverride if auto generation is enabled.
+              new HoodieKey(avroRecordRecordKeyOverRide._2.get, keyGenerator.getKey(avroRecordRecordKeyOverRide._1).getPartitionPath)

Review Comment:
   Since we have plans to fix this w/ 
   https://github.com/apache/hudi/pull/7699
   HUDI-5535, I don't want to add additional apis to the base interface/abstract class for now. 
   lets revisit holistically. 
   



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


[GitHub] [hudi] hudi-bot commented on pull request #8107: [HUDI-5514] Adding auto generation of record keys support to Hudi

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #8107:
URL: https://github.com/apache/hudi/pull/8107#issuecomment-1457531903

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603",
       "triggerID" : "35aed635391309c3c6c4b3794044bba53b3468ef",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 35aed635391309c3c6c4b3794044bba53b3468ef Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15603) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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