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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -241,39 +240,49 @@ object HoodieSparkSqlWriter {
             sparkContext.getConf.registerKryoClasses(
               Array(classOf[org.apache.avro.generic.GenericData],
                 classOf[org.apache.avro.Schema]))
-            var schema = 
AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, 
nameSpace)
-            val lastestSchema = getLatestTableSchema(fs, basePath, 
sparkContext, schema)
+
+            // TODO(HUDI-4472) revisit and simplify schema handling
+            val sourceSchema = 
AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, 
nameSpace)
+            val latestTableSchema = getLatestTableSchema(fs, basePath, 
sparkContext).getOrElse(sourceSchema)
+
+            val enabledSchemaEvolution = 
parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), 
"false").toBoolean
             var internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, 
sparkContext)
-            if (reconcileSchema && 
parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), 
"false").toBoolean
-              && internalSchemaOpt.isEmpty) {
-              // force apply full schema evolution.
-              internalSchemaOpt = 
Some(AvroInternalSchemaConverter.convert(schema))
-            }
-            if (reconcileSchema) {
-              schema = lastestSchema
-            }
-            if (internalSchemaOpt.isDefined) {
-              // Apply schema evolution.
-              val mergedSparkSchema = if (!reconcileSchema) {
-                
AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema,
 lastestSchema))
+
+            val writerSchema: Schema =
+              if (reconcileSchema) {
+                // In case we need to reconcile the schema and schema 
evolution is enabled,
+                // we will force-apply schema evolution to the writer's schema
+                if (enabledSchemaEvolution && internalSchemaOpt.isEmpty) {
+                  internalSchemaOpt = 
Some(AvroInternalSchemaConverter.convert(sourceSchema))
+                }
+
+                if (internalSchemaOpt.isDefined) {
+                  // Apply schema evolution, by auto-merging write schema and 
read schema
+                  val mergedInternalSchema = 
AvroSchemaEvolutionUtils.reconcileSchema(sourceSchema, internalSchemaOpt.get)
+                  AvroInternalSchemaConverter.convert(mergedInternalSchema, 
latestTableSchema.getName)
+                } else if 
(TableSchemaResolver.isSchemaCompatible(sourceSchema, latestTableSchema)) {
+                  // In case schema reconciliation is enabled and source and 
latest table schemas
+                  // are compatible (as defined by 
[[TableSchemaResolver#isSchemaCompatible]], then we will
+                  // pick latest table's schema as the writer's schema
+                  latestTableSchema
+                } else {
+                  // Otherwise fallback to original source's schema
+                  sourceSchema
+                }
               } else {
-                // Auto merge write schema and read schema.
-                val mergedInternalSchema = 
AvroSchemaEvolutionUtils.reconcileSchema(schema, internalSchemaOpt.get)
-                
AvroConversionUtils.convertAvroSchemaToStructType(AvroInternalSchemaConverter.convert(mergedInternalSchema,
 lastestSchema.getName))
+                // In case reconciliation is disabled, we still have to do 
nullability attributes
+                // (minor) reconciliation, making sure schema of the incoming 
batch is in-line with
+                // the data already committed in the table
+                
AvroSchemaEvolutionUtils.canonicalizeColumnNullability(sourceSchema, 
latestTableSchema)

Review Comment:
   @alexeykudinkin  
   I have encounter this problem before,So in the original code logic, I 
finally added avroconversionutils Convertstructtypetoavroschema to maintain the 
consistency of the namespace. let me check it again with new branch code.
   another things:
   line 281, we register a writerSchema to kryo, let named this writerSchema as 
schema1;
   line 175 in HoodieSparkUtils we use another schema to convert internal row 
to avro record,  let named this schema as schema2
   schema1 and schema2 has different namespace,  I'm not sure if this will 
affect performance
   
   



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