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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]