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_r408480553
 
 

 ##########
 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:
   On schema generation, we had to do some stuff... Our service emits protobufs 
to kafka, which we transform to avro using ProtoToAvro converter. Because those 
are service to service messages, they are not perfect and we had to do some 
bending to make sure schema can evolve. Some things: 
   1) We made everything optional
   2) Avro and Protobuf allows records to have no fields. But parquet doesn't, 
so, we inject marker field called exists pretty much everywhere. Originally 
added it only to records with no fields, but found out that when fields are 
added and exists is autoremoved, parquet-avro reader breaks as it expects every 
field in parquet schema to be present in avro. I have a fix to parquet-avro to 
skip those fields, will send a PR to show so we can discuss whether it is a 
good idea or not. 
   3) Then we sometimes run sql transformation (same kafka stream produces 
multiple outputs). In this case, we infer schema from Spark (using 
NullTargetSchemaProvider). This schema must be correct as well which is not the 
case as spark omits defaults, so compaction breaks. Have a PR to adress.

----------------------------------------------------------------
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