yihua commented on code in PR #13656: URL: https://github.com/apache/hudi/pull/13656#discussion_r2268595217
########## hudi-common/src/test/java/org/apache/hudi/keygen/constant/TestKeyGeneratorType.java: ########## @@ -0,0 +1,76 @@ +/* + * 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.constant; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class TestKeyGeneratorType { + + private static Stream<Arguments> testFromClassNameParams() { + return Stream.of( + Arguments.of("org.apache.hudi.keygen.SimpleKeyGenerator", KeyGeneratorType.SIMPLE), + Arguments.of("org.apache.hudi.keygen.SimpleAvroKeyGenerator", KeyGeneratorType.SIMPLE_AVRO), + Arguments.of("org.apache.hudi.keygen.ComplexKeyGenerator", KeyGeneratorType.COMPLEX), + Arguments.of("org.apache.hudi.keygen.ComplexAvroKeyGenerator", KeyGeneratorType.COMPLEX_AVRO), + Arguments.of("org.apache.hudi.keygen.TimestampBasedKeyGenerator", KeyGeneratorType.TIMESTAMP), + Arguments.of("org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator", KeyGeneratorType.TIMESTAMP_AVRO), + Arguments.of("org.apache.hudi.keygen.CustomKeyGenerator", KeyGeneratorType.CUSTOM), + Arguments.of("org.apache.hudi.keygen.CustomAvroKeyGenerator", KeyGeneratorType.CUSTOM_AVRO), + Arguments.of("org.apache.hudi.keygen.NonpartitionedKeyGenerator", KeyGeneratorType.NON_PARTITION), + Arguments.of("org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator", KeyGeneratorType.NON_PARTITION_AVRO), + Arguments.of("org.apache.hudi.keygen.GlobalDeleteKeyGenerator", KeyGeneratorType.GLOBAL_DELETE), + Arguments.of("org.apache.hudi.keygen.GlobalAvroDeleteKeyGenerator", KeyGeneratorType.GLOBAL_DELETE_AVRO), + Arguments.of("org.apache.hudi.keygen.AutoRecordGenWrapperKeyGenerator", KeyGeneratorType.AUTO_RECORD), + Arguments.of("org.apache.hudi.keygen.AutoRecordGenWrapperAvroKeyGenerator", KeyGeneratorType.AUTO_RECORD_AVRO), + Arguments.of("org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator", KeyGeneratorType.HOODIE_TABLE_METADATA), + Arguments.of("org.apache.spark.sql.hudi.command.SqlKeyGenerator", KeyGeneratorType.SPARK_SQL), + Arguments.of("org.apache.spark.sql.hudi.command.UuidKeyGenerator", KeyGeneratorType.SPARK_SQL_UUID), + Arguments.of("org.apache.spark.sql.hudi.command.MergeIntoKeyGenerator", KeyGeneratorType.SPARK_SQL_MERGE_INTO), + Arguments.of("org.apache.hudi.utilities.deltastreamer.TestHoodieDeltaStreamer$TestGenerator", KeyGeneratorType.STREAMER_TEST), Review Comment: This should be `USER_PROVIDED` ########## hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala: ########## @@ -52,7 +52,8 @@ object SparkKeyGenUtils { * @return partition column names only, concatenated by "," */ def getPartitionColumns(KeyGenClassNameOption: Option[String], props: TypedProperties): String = { - val keyGenerator = if (KeyGenClassNameOption.isEmpty) { + val keyGenerator = if (KeyGenClassNameOption.isEmpty + || KeyGeneratorType.USER_PROVIDED.getClassName.equals(KeyGenClassNameOption.get)) { Review Comment: Why is this change needed? Should `HoodieSparkKeyGeneratorFactory.createKeyGenerator(KeyGenClassNameOption.get, props)` work if `KeyGenClassNameOption` is present? ########## hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/MockUserProvidedKeyGenerator.java: ########## @@ -0,0 +1,127 @@ +/* + * 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.Option; +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.Collections; + +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; + +/** + * This class is used for test purpose, and should never be used + * in other ways. + */ +public class MockUserProvidedKeyGenerator extends BuiltinKeyGenerator { + + private final SimpleAvroKeyGenerator simpleAvroKeyGenerator; + + public MockUserProvidedKeyGenerator(TypedProperties props) { + this(props, Option.ofNullable(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), null)), + props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())); + } + + public MockUserProvidedKeyGenerator(TypedProperties props, String partitionPathField) { + this(props, Option.empty(), partitionPathField); + } + + public MockUserProvidedKeyGenerator(TypedProperties props, Option<String> recordKeyField, String partitionPathField) { + super(props); + // Make sure key-generator is configured properly + validateRecordKey(recordKeyField); + validatePartitionPath(partitionPathField); + + this.recordKeyFields = !recordKeyField.isPresent() ? Collections.emptyList() : Collections.singletonList(recordKeyField.get()); + this.partitionPathFields = partitionPathField == null ? Collections.emptyList() : Collections.singletonList(partitionPathField); + this.simpleAvroKeyGenerator = new SimpleAvroKeyGenerator(props, recordKeyField, partitionPathField); + } + + @Override + public String getRecordKey(GenericRecord record) { + return simpleAvroKeyGenerator.getRecordKey(record); + } + + @Override + public String getPartitionPath(GenericRecord record) { + return simpleAvroKeyGenerator.getPartitionPath(record); + } + Review Comment: Change the logic to be special, e.g., adding a prefix, so that it is different from any built-in key generator and the validation in tests follow the same to enhance the test? ########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala: ########## @@ -2065,6 +2065,66 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals(count, 0) } + @ParameterizedTest + @MethodSource(Array("provideParamsForKeyGenTest")) + def testUserProvidedKeyGeneratorClass(keyGenClass: String, + keyGenType: String): Unit = { + val recordType = HoodieRecordType.AVRO + val (writeOpts, readOpts) = getWriterReaderOpts(recordType, + CommonOptionUtils.commonOpts ++ Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> keyGenClass, + HoodieWriteConfig.KEYGENERATOR_TYPE.key -> keyGenType, Review Comment: Could we also test the cases where only one of `KEYGENERATOR_CLASS_NAME` and `KEYGENERATOR_TYPE` is set? Is there already a test like that? ########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala: ########## @@ -2065,6 +2065,66 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals(count, 0) } + @ParameterizedTest + @MethodSource(Array("provideParamsForKeyGenTest")) + def testUserProvidedKeyGeneratorClass(keyGenClass: String, + keyGenType: String): Unit = { Review Comment: nit: add `expectedKeyGenType` for better readability of tests? ########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala: ########## @@ -2065,6 +2065,66 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals(count, 0) } + @ParameterizedTest + @MethodSource(Array("provideParamsForKeyGenTest")) + def testUserProvidedKeyGeneratorClass(keyGenClass: String, + keyGenType: String): Unit = { + val recordType = HoodieRecordType.AVRO + val (writeOpts, readOpts) = getWriterReaderOpts(recordType, + CommonOptionUtils.commonOpts ++ Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> keyGenClass, + HoodieWriteConfig.KEYGENERATOR_TYPE.key -> keyGenType, + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition")) + val expectedKeyGenType = if (StringUtils.isNullOrEmpty(keyGenClass)) { + KeyGeneratorType.USER_PROVIDED.name + } else { + KeyGeneratorType.fromClassName(keyGenClass).name + } + + // Insert. + val records = recordsToStrings(dataGen.generateInserts("000", 10)).asScala.toList + val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) + + // Make sure the initial configurations set properly for USER_PROVIDED type. + if (StringUtils.isNullOrEmpty(keyGenClass) && keyGenType.equals(KeyGeneratorType.USER_PROVIDED.name)) { + assertThrows(classOf[IllegalArgumentException])({ + writeToHudi(writeOpts, inputDF) + }) + // scalastyle:off return + return + // scalastyle:on return Review Comment: Fix the scalastyle check? ########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala: ########## @@ -2065,6 +2065,66 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals(count, 0) } + @ParameterizedTest + @MethodSource(Array("provideParamsForKeyGenTest")) + def testUserProvidedKeyGeneratorClass(keyGenClass: String, + keyGenType: String): Unit = { + val recordType = HoodieRecordType.AVRO + val (writeOpts, readOpts) = getWriterReaderOpts(recordType, + CommonOptionUtils.commonOpts ++ Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> keyGenClass, + HoodieWriteConfig.KEYGENERATOR_TYPE.key -> keyGenType, + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition")) + val expectedKeyGenType = if (StringUtils.isNullOrEmpty(keyGenClass)) { + KeyGeneratorType.USER_PROVIDED.name + } else { + KeyGeneratorType.fromClassName(keyGenClass).name + } + + // Insert. + val records = recordsToStrings(dataGen.generateInserts("000", 10)).asScala.toList + val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) + + // Make sure the initial configurations set properly for USER_PROVIDED type. + if (StringUtils.isNullOrEmpty(keyGenClass) && keyGenType.equals(KeyGeneratorType.USER_PROVIDED.name)) { + assertThrows(classOf[IllegalArgumentException])({ + writeToHudi(writeOpts, inputDF) + }) + // scalastyle:off return + return + // scalastyle:on return + } else { // Otherwise, the data is properly ingested. + writeToHudi(writeOpts, inputDF) + } + var actualDF = spark.read.format("hudi").options(readOpts).load(basePath) + val inputKeyDF = inputDF.select("_row_key").sort("_row_key") + var actualKeyDF = actualDF.select("_row_key").sort("_row_key") + assertTrue(inputKeyDF.except(actualKeyDF).isEmpty && actualKeyDF.except(inputKeyDF).isEmpty) + val metaClient = getHoodieMetaClient(storageConf, basePath) + val actualKeyGenType = metaClient.getTableConfig.getKeyGeneratorType + assertEquals(expectedKeyGenType, actualKeyGenType) + // For USER_PROVIDED type, the class should exist in table config. + if (KeyGeneratorType.USER_PROVIDED.name == actualKeyGenType) { + assertEquals(keyGenClass, metaClient.getTableConfig.getKeyGeneratorClassName) + } + + // First update. + val firstUpdate = recordsToStrings(dataGen.generateUpdatesForAllRecords("001")).asScala.toList + val firstUpdateDF = spark.read.json(spark.sparkContext.parallelize(firstUpdate, 2)) + writeToHudi(writeOpts, firstUpdateDF) + val newReadOpts = readOpts ++ Map(HoodieTableConfig.POPULATE_META_FIELDS.key -> "false") Review Comment: Key generator generates the keys and partition paths in the meta fields. Does `HoodieTableConfig.POPULATE_META_FIELDS.key -> "false"` fail the test? Could you investigate the root cause? ########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala: ########## @@ -2083,4 +2143,24 @@ object TestCOWDataSource { (autoUpgrade: String) => targetVersions.map( (targetVersion: String) => Arguments.of(autoUpgrade, targetVersion))) } + + def provideParamsForKeyGenTest(): java.util.List[Arguments] = { + java.util.Arrays.asList( + Arguments.of( + "org.apache.hudi.keygen.MockUserProvidedKeyGenerator", + KeyGeneratorType.USER_PROVIDED.name()), + Arguments.of( + "", + KeyGeneratorType.USER_PROVIDED.name()), + Arguments.of( + "org.apache.hudi.keygen.SimpleKeyGenerator", + KeyGeneratorType.SIMPLE.name()), + Arguments.of( + "org.apache.hudi.keygen.SimpleAvroKeyGenerator", + KeyGeneratorType.SIMPLE.name()), + Arguments.of( + "org.apache.hudi.keygen.ComplexKeyGenerator", + "") Review Comment: Use `Option<String>` so that if it's `Option.empty()`, do not set the write config, instead of using the empty string. -- 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]
