xiarixiaoyao commented on code in PR #8026:
URL: https://github.com/apache/hudi/pull/8026#discussion_r1117842078


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -613,8 +617,11 @@ object HoodieBaseRelation extends SparkAdapterSupport {
     def apply(file: PartitionedFile): Iterator[InternalRow] = read.apply(file)
   }
 
-  def convertToAvroSchema(structSchema: StructType): Schema =
-    sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = 
false, "Record")
+  def convertToAvroSchema(structSchema: StructType, tableName: String ): 
Schema = {
+    val (recordName, namespace) = 
AvroConversionUtils.getAvroRecordNameAndNamespace(tableName)
+    val avroSchema = 
sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, 
recordName, namespace)
+    getAvroSchemaWithDefaults(avroSchema, structSchema)
+  }

Review Comment:
   yes,
   schemaConverters.toAvroType  will lost default value. see  
https://github.com/apache/hudi/pull/2765
   In the schema evolution scenario, the default value is very important, 
avroSchema cares about this。
   
   eg:  If we add a new column  newCol: string  to the table,  the default 
value of newCol will be null
   after schemaConverters.toAvroType , the default vaule of newCol will be lost
   now if we use this schema to read old avro log(not contains column newCol), 
avro will complain that there is no default value, and throw exception.
   https://github.com/apache/hudi/pull/7915 The root cause of this pr is that 
we lost the default value in the conversion process
   



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to