From 6efa16317c0f0f13798d739d9615dda24bf91bcf Mon Sep 17 00:00:00 2001 From: n3nash Date: Wed, 17 Jul 2019 03:25:54 -0700 Subject: [PATCH] Fixing default value for avro 1.7 which assumes NULL value instead of a jsonnode that is null (#792) --- .../uber/hoodie/common/util/HoodieAvroUtils.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java index 9b34fab63..3a9443e37 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java @@ -102,15 +102,15 @@ public class HoodieAvroUtils { List parentFields = new ArrayList<>(); Schema.Field commitTimeField = new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, - METADATA_FIELD_SCHEMA, "", null); + METADATA_FIELD_SCHEMA, "", NullNode.getInstance()); Schema.Field commitSeqnoField = new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, - METADATA_FIELD_SCHEMA, "", null); + METADATA_FIELD_SCHEMA, "", NullNode.getInstance()); Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, - METADATA_FIELD_SCHEMA, "", null); + METADATA_FIELD_SCHEMA, "", NullNode.getInstance()); Schema.Field partitionPathField = new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, - METADATA_FIELD_SCHEMA, "", null); + METADATA_FIELD_SCHEMA, "", NullNode.getInstance()); Schema.Field fileNameField = new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD, - METADATA_FIELD_SCHEMA, "", null); + METADATA_FIELD_SCHEMA, "", NullNode.getInstance()); parentFields.add(commitTimeField); parentFields.add(commitSeqnoField); @@ -119,7 +119,7 @@ public class HoodieAvroUtils { parentFields.add(fileNameField); for (Schema.Field field : schema.getFields()) { if (!isMetadataField(field.name())) { - Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), null); + Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()); for (Map.Entry prop : field.getJsonProps().entrySet()) { newField.addProp(prop.getKey(), prop.getValue()); } @@ -135,7 +135,7 @@ public class HoodieAvroUtils { private static Schema initRecordKeySchema() { Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, - METADATA_FIELD_SCHEMA, "", null); + METADATA_FIELD_SCHEMA, "", NullNode.getInstance()); Schema recordKeySchema = Schema.createRecord("HoodieRecordKey", "", "", false); recordKeySchema.setFields(Arrays.asList(recordKeyField)); return recordKeySchema;