alexeykudinkin commented on code in PR #7726:
URL: https://github.com/apache/hudi/pull/7726#discussion_r1085760200
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java:
##########
@@ -39,6 +39,7 @@ public NonpartitionedAvroKeyGenerator(TypedProperties props) {
this.recordKeyFields =
Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
.split(",")).map(String::trim).filter(s ->
!s.isEmpty()).collect(Collectors.toList());
this.partitionPathFields = EMPTY_PARTITION_FIELD_LIST;
+ instantiateAutoRecordKeyGenerator();
Review Comment:
Please assign var inside the ctor, make it final
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java:
##########
@@ -64,14 +67,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());
+ return combineCompositeRecordKey(rowAccessor.getRecordKeyParts(row));
+ }
}
@Override
public UTF8String getRecordKey(InternalRow internalRow, StructType schema) {
- tryInitRowAccessor(schema);
- return
combineCompositeRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow));
+ if (autoGenerateRecordKeys) {
+ return super.getRecordKey(internalRow, schema);
Review Comment:
Why are we redirecting to BuiltinKeyGenerator here?
On top of that i don't see that we've changed it either, so how is this
supposed to work?
##########
hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java:
##########
@@ -32,11 +32,16 @@
*/
public abstract class BaseKeyGenerator extends KeyGenerator {
+ protected static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+ public static final String SPLIT_REGEX = ":";
+
protected List<String> recordKeyFields;
protected List<String> partitionPathFields;
protected final boolean encodePartitionPath;
protected final boolean hiveStylePartitioning;
protected final boolean consistentLogicalTimestampEnabled;
+ protected final boolean autoGenerateRecordKeys;
+ protected AutoRecordKeyGenerator autoRecordKeyGenerator;
Review Comment:
Please wrap it into Option
##########
hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java:
##########
@@ -69,11 +69,21 @@ public class KeyGeneratorOptions extends HoodieConfig {
+ "`2016-12-29 09:54:00.0` in row-writer path, while it will be
written as long value `1483023240000000` in non row-writer path. "
+ "If enabled, then the timestamp value will be written in both the
cases.");
- public static final ConfigProperty<Integer> NUM_FIELDS_IN_KEYLESS_GENERATOR
= ConfigProperty
- .key("hoodie.datasource.write.recordkey.keyless.field.count")
+ public static final ConfigProperty<String> AUTO_GENERATE_RECORD_KEYS =
ConfigProperty
+ .key("hoodie.auto.generate.record.keys")
+ .defaultValue("false")
+ .sinceVersion("0.13.0")
+ .withDocumentation("When enabled, hudi will auto generate a
deterministic key for a record based on the contents of the field. "
+ + "The keys are guaranteed to be deterministic but not unique, so
they can only be used for insert workflows with deduplication disabled."
+ + "The class attempts to get sufficient uniqueness for keys to
prevent frequent collisions by choosing the fields it uses in order of
decreasing "
+ + "likelihood for uniqueness.");
+
+ public static final ConfigProperty<Integer>
NUM_FIELDS_IN_AUTO_RECORDKEY_GENERATION = ConfigProperty
Review Comment:
I don't think we should expose this kind of configuration -- it's very
confusing in terms of what exactly it affects, and how setting it will
translate into desired outcome
##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestAutoRecordKeyGenerator.java:
##########
@@ -29,66 +29,76 @@
import java.io.IOException;
-public class TestKeylessKeyGenerator {
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestAutoRecordKeyGenerator {
private static final long TIME = 1672265446090L;
private static final Schema SCHEMA;
+ private static final String PARTITION_PATH_STR = "partition1";
static {
try {
- SCHEMA = new
Schema.Parser().parse(TestKeylessKeyGenerator.class.getClassLoader().getResourceAsStream("keyless_schema.avsc"));
+ SCHEMA = new
Schema.Parser().parse(TestAutoRecordKeyGenerator.class.getClassLoader().getResourceAsStream("keyless_schema.avsc"));
} catch (IOException ex) {
throw new RuntimeException(ex);
}
}
@Test
public void createKeyWithoutPartitionColumn() {
- KeylessKeyGenerator keyGenerator = new
KeylessKeyGenerator(getKeyGenProperties("", 3));
- GenericRecord record = createRecord("partition1", "value1", 123, 456L,
TIME, null);
+ ComplexAvroKeyGenerator keyGenerator = new
ComplexAvroKeyGenerator(getKeyGenProperties("partition_field", 3));
Review Comment:
Why are we testing `ComplexKeyGenerator` here?
We need to test 2 concerns:
- `AutoRecordKeyGenerator` itself
- All existing key-generators in auto-gen mode (just one parameterized test
for `getRecordKey` should be fine)
##########
hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java:
##########
@@ -32,11 +32,16 @@
*/
public abstract class BaseKeyGenerator extends KeyGenerator {
+ protected static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+ public static final String SPLIT_REGEX = ":";
+
protected List<String> recordKeyFields;
protected List<String> partitionPathFields;
protected final boolean encodePartitionPath;
protected final boolean hiveStylePartitioning;
protected final boolean consistentLogicalTimestampEnabled;
+ protected final boolean autoGenerateRecordKeys;
Review Comment:
We don't need a separate boolean, let's hold auto-gen as an option
--
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]