You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/07/28 02:00:35 UTC

[GitHub] [hudi] xiarixiaoyao commented on a diff in pull request #6213: [HUDI-4081][HUDI-4472] Addressing Spark SQL vs Spark DS performance gap

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


##########
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:
   Notice:
   The avro Schema created by AvroSchemaEvolutionUtils.canonicalizecolumnnullability  has a different namespce with the schema created by AvroConversionUtils.convertStructTypeToAvroSchema.
   Avro 1.8.2 is Namespace sensitive.
   In the original logic, we will eventually call AvroConversionUtils.convertStructTypeToAvroSchema to maintain namespace consistency
   



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org