codope commented on code in PR #7726:
URL: https://github.com/apache/hudi/pull/7726#discussion_r1083729980
##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestAutoRecordKeyGenerator.java:
##########
@@ -112,8 +112,9 @@ protected GenericRecord createRecord(String partitionField,
String stringValue,
protected TypedProperties getKeyGenProperties(String partitionPathField, int
numFieldsInKeyGen) {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(),
partitionPathField);
- properties.put(KeyGeneratorOptions.NUM_FIELDS_IN_KEYLESS_GENERATOR.key(),
numFieldsInKeyGen);
+
properties.put(KeyGeneratorOptions.NUM_FIELDS_IN_AUTO_RECORDKEY_GENERATION.key(),
numFieldsInKeyGen);
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "");
Review Comment:
Do we still need to set this prop?
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAutoRecordKeyGeneration.scala:
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi
+
+
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.keygen.TimestampBasedKeyGenerator
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions
+import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
+import org.apache.spark.sql._
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.CsvSource
+
+import scala.collection.JavaConversions._
+
+/**
+ * Tests auto generation of record keys.
+ */
+class TestAutoRecordKeyGeneration extends SparkClientFunctionalTestHarness {
Review Comment:
+1 for end to end test.
Can we write a similar test for spark-sql path as well?
##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestAutoRecordKeyGenerator.java:
##########
@@ -29,45 +29,45 @@
import java.io.IOException;
-public class TestKeylessKeyGenerator {
+public class TestAutoRecordKeyGenerator {
Review Comment:
Should we also assert different partition paths in this test?
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java:
##########
@@ -64,14 +66,22 @@ public String getPartitionPath(GenericRecord record) {
@Override
public String getRecordKey(Row row) {
- tryInitRowAccessor(row.schema());
- return combineCompositeRecordKey(rowAccessor.getRecordKeyParts(row));
+ if (autoGenerateRecordKeys) {
+ return super.getRecordKey(row);
+ } else {
+ tryInitRowAccessor(row.schema());
Review Comment:
let's file a ticket for handling row-writer path.
##########
hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java:
##########
@@ -81,4 +88,10 @@ public List<String> getPartitionPathFields() {
public boolean isConsistentLogicalTimestampEnabled() {
return consistentLogicalTimestampEnabled;
}
+
+ protected void instantiateAutoRecordKeyGenerator() {
+ if (autoGenerateRecordKeys) {
+ autoRecordKeyGenerator = new AutoRecordKeyGenerator(config,
getPartitionPathFields());
Review Comment:
Why not move it to the constructor itself? Is there some keygen for which we
don't want to instantiate `autoRecordKeyGenerator`? Moving it to the
constructor ensures that no one can miss it in future keygens.
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java:
##########
@@ -32,19 +33,25 @@ 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 =
props.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
+ ?
Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(","))
.map(String::trim)
.filter(s -> !s.isEmpty())
- .collect(Collectors.toList());
+ .collect(Collectors.toList()) : Collections.EMPTY_LIST;
this.partitionPathFields =
Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(","))
.map(String::trim)
.filter(s -> !s.isEmpty())
.collect(Collectors.toList());
+ instantiateAutoRecordKeyGenerator();
}
@Override
public String getRecordKey(GenericRecord record) {
- return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(),
isConsistentLogicalTimestampEnabled());
+ if (autoGenerateRecordKeys) {
Review Comment:
I think we should move this conditional inside `KeyGenUtils.getRecordKey`
otherwise it's a burden on the developer to handle in each keygen class. Wdyt?
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java:
##########
@@ -65,6 +65,7 @@ public enum TimestampType implements Serializable {
private final DateTimeZone outputDateTimeZone;
protected final boolean encodePartitionPath;
+ private AutoRecordKeyGenerator autoRecordKeyGenerator;
Review Comment:
don't really need this.
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java:
##########
@@ -85,9 +85,13 @@ public String getPartitionPath(GenericRecord record) {
@Override
public String getRecordKey(Row row) {
- return getRecordKeyFieldNames().size() == 1
- ? new SimpleKeyGenerator(config,
config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()),
null).getRecordKey(row)
- : new ComplexKeyGenerator(config).getRecordKey(row);
+ if (autoGenerateRecordKeys) {
Review Comment:
Also, need to handle empty/null recordkey field config in the constructor on
line 63.
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java:
##########
@@ -32,19 +33,25 @@ 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 =
props.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
Review Comment:
Let's make sure that this is handled in every keygen's 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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]