afilipchik commented on a change in pull request #1513: [HUDI-793] Adding 
proper default to hudi metadata fields and proper handling to rewrite routine
URL: https://github.com/apache/incubator-hudi/pull/1513#discussion_r408477566
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
 ##########
 @@ -104,15 +105,15 @@ public static Schema addMetadataFields(Schema schema) {
     List<Schema.Field> parentFields = new ArrayList<>();
 
     Schema.Field commitTimeField =
-        new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", (Object) null);
+        new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
     Schema.Field commitSeqnoField =
-        new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", (Object) null);
+        new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
     Schema.Field recordKeyField =
-        new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", (Object) null);
+        new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
     Schema.Field partitionPathField =
-        new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", (Object) null);
+        new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
     Schema.Field fileNameField =
-        new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", (Object) null);
+        new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD, 
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
 
 Review comment:
   Sure, there might be another way. I think there are changes with nulls in 
1.9.0, but upgrading is not easy. Just to clarify, it is not for the write 
support, but for the read one. Consider situation:
   We have avro with 4 metada fields written, and defaults are not set. Not an 
issue as it is written.
   We add another metadata field (sourceOffset, or something), which also 
doesn't have default field and redeploy. When we try to read old record with 
new schema it will fail because old record doesn't have new field and there is 
no default for it. I hit this issue with spark-> avro transformation, have 
another PR to address

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to