[HUDI-327] Add null/empty checks to key generators (#1040)
* Adds null and empty checks to all key generators. * Also improves error messaging for key generator issues.
This commit is contained in:
@@ -23,7 +23,7 @@ import java.util.List;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
/**
|
||||
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
|
||||
@@ -31,8 +31,9 @@ import org.apache.hudi.exception.HoodieException;
|
||||
public class ComplexKeyGenerator extends KeyGenerator {
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH = "default";
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
private static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
|
||||
private static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
|
||||
|
||||
protected final List<String> recordKeyFields;
|
||||
|
||||
@@ -48,24 +49,40 @@ public class ComplexKeyGenerator extends KeyGenerator {
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyFields == null || partitionPathFields == null) {
|
||||
throw new HoodieException("Unable to find field names for record key or partition path in cfg");
|
||||
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
|
||||
boolean keyIsNullEmpty = true;
|
||||
StringBuilder recordKey = new StringBuilder();
|
||||
for (String recordKeyField : recordKeyFields) {
|
||||
recordKey.append(recordKeyField + ":" + DataSourceUtils.getNestedFieldValAsString(record, recordKeyField) + ",");
|
||||
String recordKeyValue = DataSourceUtils.getNullableNestedFieldValAsString(record, recordKeyField);
|
||||
if (recordKeyValue == null) {
|
||||
recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ",");
|
||||
} else if (recordKeyValue.isEmpty()) {
|
||||
recordKey.append(recordKeyField + ":" + EMPTY_RECORDKEY_PLACEHOLDER + ",");
|
||||
} else {
|
||||
recordKey.append(recordKeyField + ":" + recordKeyValue + ",");
|
||||
keyIsNullEmpty = false;
|
||||
}
|
||||
}
|
||||
recordKey.deleteCharAt(recordKey.length() - 1);
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
try {
|
||||
for (String partitionPathField : partitionPathFields) {
|
||||
partitionPath.append(DataSourceUtils.getNestedFieldValAsString(record, partitionPathField));
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
|
||||
}
|
||||
partitionPath.deleteCharAt(partitionPath.length() - 1);
|
||||
} catch (HoodieException e) {
|
||||
partitionPath = partitionPath.append(DEFAULT_PARTITION_PATH);
|
||||
if (keyIsNullEmpty) {
|
||||
throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
|
||||
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
|
||||
}
|
||||
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
for (String partitionPathField : partitionPathFields) {
|
||||
String fieldVal = DataSourceUtils.getNullableNestedFieldValAsString(record, partitionPathField);
|
||||
if (fieldVal == null || fieldVal.isEmpty()) {
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH);
|
||||
} else {
|
||||
partitionPath.append(fieldVal);
|
||||
}
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
|
||||
}
|
||||
partitionPath.deleteCharAt(partitionPath.length() - 1);
|
||||
|
||||
return new HoodieKey(recordKey.toString(), partitionPath.toString());
|
||||
}
|
||||
|
||||
|
||||
@@ -51,6 +51,17 @@ import org.apache.spark.api.java.JavaSparkContext;
|
||||
*/
|
||||
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
|
||||
*/
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
/**
|
||||
* Simple Key generator for unpartitioned Hive Tables
|
||||
@@ -35,7 +36,10 @@ public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
|
||||
String recordKey = DataSourceUtils.getNullableNestedFieldValAsString(record, recordKeyField);
|
||||
if (recordKey == null || recordKey.isEmpty()) {
|
||||
throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty.");
|
||||
}
|
||||
return new HoodieKey(recordKey, EMPTY_PARTITION);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,7 +21,7 @@ package org.apache.hudi;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
/**
|
||||
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
|
||||
@@ -43,15 +43,16 @@ public class SimpleKeyGenerator extends KeyGenerator {
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyField == null || partitionPathField == null) {
|
||||
throw new HoodieException("Unable to find field names for record key or partition path in cfg");
|
||||
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
|
||||
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
|
||||
String partitionPath;
|
||||
try {
|
||||
partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField);
|
||||
} catch (HoodieException e) {
|
||||
// if field is not found, lump it into default partition
|
||||
String recordKey = DataSourceUtils.getNullableNestedFieldValAsString(record, recordKeyField);
|
||||
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);
|
||||
if (partitionPath == null || partitionPath.isEmpty()) {
|
||||
partitionPath = DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user