[HUDI-4039] Make sure all builtin KeyGenerators properly implement Spark specific APIs (#5523)
This set of changes makes sure that all builtin KeyGenerators properly implement Spark-specific APIs in a performant way (minimizing key-generators overhead)
This commit is contained in:
@@ -44,7 +44,7 @@ public class ComplexAvroKeyGenerator extends BaseKeyGenerator {
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFields(), isConsistentLogicalTimestampEnabled());
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -103,13 +103,13 @@ public class CustomAvroKeyGenerator extends BaseKeyGenerator {
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
validateRecordKeyFields();
|
||||
return getRecordKeyFields().size() == 1
|
||||
return getRecordKeyFieldNames().size() == 1
|
||||
? new SimpleAvroKeyGenerator(config).getRecordKey(record)
|
||||
: new ComplexAvroKeyGenerator(config).getRecordKey(record);
|
||||
}
|
||||
|
||||
private void validateRecordKeyFields() {
|
||||
if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) {
|
||||
if (getRecordKeyFieldNames() == null || getRecordKeyFieldNames().isEmpty()) {
|
||||
throw new HoodieKeyException("Unable to find field names for record key in cfg");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -40,7 +40,7 @@ public class GlobalAvroDeleteKeyGenerator extends BaseKeyGenerator {
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFields(), isConsistentLogicalTimestampEnabled());
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -42,6 +42,7 @@ public class KeyGenUtils {
|
||||
|
||||
protected static final String HUDI_DEFAULT_PARTITION_PATH = PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
|
||||
public static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
public static final String DEFAULT_RECORD_KEY_PARTS_SEPARATOR = ",";
|
||||
|
||||
/**
|
||||
* Fetches record key from the GenericRecord.
|
||||
|
||||
@@ -56,10 +56,10 @@ public class NonpartitionedAvroKeyGenerator extends BaseKeyGenerator {
|
||||
// for backward compatibility, we need to use the right format according to the number of record key fields
|
||||
// 1. if there is only one record key field, the format of record key is just "<value>"
|
||||
// 2. if there are multiple record key fields, the format is "<field1>:<value1>,<field2>:<value2>,..."
|
||||
if (getRecordKeyFields().size() == 1) {
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFields().get(0), isConsistentLogicalTimestampEnabled());
|
||||
if (getRecordKeyFieldNames().size() == 1) {
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames().get(0), isConsistentLogicalTimestampEnabled());
|
||||
}
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFields(), isConsistentLogicalTimestampEnabled());
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled());
|
||||
}
|
||||
|
||||
public String getEmptyPartition() {
|
||||
|
||||
@@ -47,7 +47,7 @@ public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFields().get(0), isConsistentLogicalTimestampEnabled());
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames().get(0), isConsistentLogicalTimestampEnabled());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -111,7 +111,7 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
|
||||
try {
|
||||
return getPartitionPath(partitionVal);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieKeyGeneratorException("Unable to parse input partition field :" + partitionVal, e);
|
||||
throw new HoodieKeyGeneratorException("Unable to parse input partition field: " + partitionVal, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -181,7 +181,7 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
|
||||
timeMs = convertLongTimeToMillis(((LocalDate) partitionVal).toEpochDay());
|
||||
} else if (partitionVal instanceof CharSequence) {
|
||||
if (!inputFormatter.isPresent()) {
|
||||
throw new HoodieException("Missing inputformatter. Ensure " + KeyGeneratorOptions.Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP + " config is set when timestampType is DATE_STRING or MIXED!");
|
||||
throw new HoodieException("Missing input formatter. Ensure " + KeyGeneratorOptions.Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP + " config is set when timestampType is DATE_STRING or MIXED!");
|
||||
}
|
||||
DateTime parsedDateTime = inputFormatter.get().parseDateTime(partitionVal.toString());
|
||||
if (this.outputDateTimeZone == null) {
|
||||
|
||||
Reference in New Issue
Block a user