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_r408447606
 
 

 ##########
 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:
   Incorrect avro schema is a disaster waiting to happen. Current one says: it 
takes "null" or String, but if you try to read the message with reader schema 
that doesn't have those fields it will break as default will not be provided. 
Given that it is serialized in both log and parquet files it has to be correct, 
as if we add new field it will not be readable. Here is example exception when 
message was written without a field and  attempted to be read with field that 
doesn't have default:
   `Caused by: org.apache.avro.AvroTypeException: Found Event, expecting Event, 
missing required field exist
        at 
org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:292)
        at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
        at 
org.apache.avro.io.ResolvingDecoder.readFieldOrder(ResolvingDecoder.java:130)
        at 
org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:215)
        at 
org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
        at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
        at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145)`
   

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to