[HUDI-406]: added default partition path in TimestampBasedKeyGenerator
This commit is contained in:
committed by
Balaji Varadarajan
parent
2d5b79d96f
commit
8f935e779a
@@ -61,7 +61,7 @@ public class ComplexKeyGenerator extends KeyGenerator {
|
||||
boolean keyIsNullEmpty = true;
|
||||
StringBuilder recordKey = new StringBuilder();
|
||||
for (String recordKeyField : recordKeyFields) {
|
||||
String recordKeyValue = DataSourceUtils.getNullableNestedFieldValAsString(record, recordKeyField);
|
||||
String recordKeyValue = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true);
|
||||
if (recordKeyValue == null) {
|
||||
recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ",");
|
||||
} else if (recordKeyValue.isEmpty()) {
|
||||
@@ -79,7 +79,7 @@ public class ComplexKeyGenerator extends KeyGenerator {
|
||||
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
for (String partitionPathField : partitionPathFields) {
|
||||
String fieldVal = DataSourceUtils.getNullableNestedFieldValAsString(record, partitionPathField);
|
||||
String fieldVal = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField, true);
|
||||
if (fieldVal == null || fieldVal.isEmpty()) {
|
||||
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + DEFAULT_PARTITION_PATH
|
||||
: DEFAULT_PARTITION_PATH);
|
||||
|
||||
@@ -52,29 +52,18 @@ import java.util.stream.Collectors;
|
||||
*/
|
||||
public class DataSourceUtils {
|
||||
|
||||
/**
|
||||
* Obtain value of the provided nullable field as string, denoted by dot notation. e.g: a.b.c
|
||||
*/
|
||||
public static String getNullableNestedFieldValAsString(GenericRecord record, String fieldName) {
|
||||
try {
|
||||
return getNestedFieldValAsString(record, fieldName);
|
||||
} catch (HoodieException e) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c
|
||||
*/
|
||||
public static String getNestedFieldValAsString(GenericRecord record, String fieldName) {
|
||||
Object obj = getNestedFieldVal(record, fieldName);
|
||||
return obj.toString();
|
||||
public static String getNestedFieldValAsString(GenericRecord record, String fieldName, boolean returnNullIfNotFound) {
|
||||
Object obj = getNestedFieldVal(record, fieldName, returnNullIfNotFound);
|
||||
return (obj == null) ? null : obj.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain value of the provided field, denoted by dot notation. e.g: a.b.c
|
||||
*/
|
||||
public static Object getNestedFieldVal(GenericRecord record, String fieldName) {
|
||||
public static Object getNestedFieldVal(GenericRecord record, String fieldName, boolean returnNullIfNotFound) {
|
||||
String[] parts = fieldName.split("\\.");
|
||||
GenericRecord valueNode = record;
|
||||
int i = 0;
|
||||
@@ -96,9 +85,14 @@ public class DataSourceUtils {
|
||||
valueNode = (GenericRecord) val;
|
||||
}
|
||||
}
|
||||
throw new HoodieException(
|
||||
|
||||
if (returnNullIfNotFound) {
|
||||
return null;
|
||||
} else {
|
||||
throw new HoodieException(
|
||||
fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :"
|
||||
+ valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList()));
|
||||
+ valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -37,7 +37,7 @@ public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
String recordKey = DataSourceUtils.getNullableNestedFieldValAsString(record, recordKeyField);
|
||||
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true);
|
||||
if (recordKey == null || recordKey.isEmpty()) {
|
||||
throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty.");
|
||||
}
|
||||
|
||||
@@ -51,12 +51,12 @@ public class SimpleKeyGenerator extends KeyGenerator {
|
||||
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
|
||||
String recordKey = DataSourceUtils.getNullableNestedFieldValAsString(record, recordKeyField);
|
||||
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true);
|
||||
if (recordKey == null || recordKey.isEmpty()) {
|
||||
throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty.");
|
||||
}
|
||||
|
||||
String partitionPath = DataSourceUtils.getNullableNestedFieldValAsString(record, partitionPathField);
|
||||
String partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField, true);
|
||||
if (partitionPath == null || partitionPath.isEmpty()) {
|
||||
partitionPath = DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
|
||||
@@ -98,7 +98,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace)
|
||||
val hoodieAllIncomingRecords = genericRecords.map(gr => {
|
||||
val orderingVal = DataSourceUtils.getNestedFieldValAsString(
|
||||
gr, parameters(PRECOMBINE_FIELD_OPT_KEY)).asInstanceOf[Comparable[_]]
|
||||
gr, parameters(PRECOMBINE_FIELD_OPT_KEY), false).asInstanceOf[Comparable[_]]
|
||||
DataSourceUtils.createHoodieRecord(gr,
|
||||
orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY))
|
||||
}).toJavaRDD()
|
||||
|
||||
Reference in New Issue
Block a user