1
0

Support nested types for recordKey, partitionPath and combineKey

This commit is contained in:
Balaji Varadarajan
2019-05-14 19:59:04 -07:00
committed by vinoth chandar
parent e43efa042f
commit a7e6cf5197
3 changed files with 12 additions and 4 deletions

View File

@@ -53,6 +53,14 @@ public class DataSourceUtils {
* 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 == 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) {
String[] parts = fieldName.split("\\.");
GenericRecord valueNode = record;
int i = 0;
@@ -65,7 +73,7 @@ public class DataSourceUtils {
// return, if last part of name
if (i == parts.length - 1) {
return val.toString();
return val;
} else {
// VC: Need a test here
if (!(val instanceof GenericRecord)) {

View File

@@ -243,7 +243,7 @@ public class HoodieDeltaStreamer implements Serializable {
JavaRDD<GenericRecord> avroRDD = avroRDDOptional.get();
JavaRDD<HoodieRecord> records = avroRDD.map(gr -> {
HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr,
(Comparable) gr.get(cfg.sourceOrderingField));
(Comparable) DataSourceUtils.getNestedFieldVal(gr, cfg.sourceOrderingField));
return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
});

View File

@@ -83,7 +83,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
@Override
public HoodieKey getKey(GenericRecord record) {
Object partitionVal = record.get(partitionPathField);
Object partitionVal = DataSourceUtils.getNestedFieldVal(record, partitionPathField);
SimpleDateFormat partitionPathFormat = new SimpleDateFormat(outputDateFormat);
partitionPathFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
@@ -102,7 +102,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
"Unexpected type for partition field: " + partitionVal.getClass().getName());
}
return new HoodieKey(record.get(recordKeyField).toString(),
return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField),
partitionPathFormat.format(new Date(unixTime * 1000)));
} catch (ParseException pe) {
throw new HoodieDeltaStreamerException(