This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new d149d787e9d [HUDI-7017] Prevent full schema evolution from wrongly 
falling back to OOB schema evolution (#9966)
d149d787e9d is described below

commit d149d787e9d3dbf425c0dd5ca0265bed5fe2795f
Author: voonhous <[email protected]>
AuthorDate: Thu Nov 9 10:44:01 2023 +0800

    [HUDI-7017] Prevent full schema evolution from wrongly falling back to OOB 
schema evolution (#9966)
---
 .../scala/org/apache/hudi/HoodieBaseRelation.scala    | 19 ++++++++++++++++---
 1 file changed, 16 insertions(+), 3 deletions(-)

diff --git 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala
 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala
index 78c5cc4ca47..eaeff8bc7e9 100644
--- 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala
+++ 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala
@@ -715,16 +715,29 @@ abstract class HoodieBaseRelation(val sqlContext: 
SQLContext,
     if (extractPartitionValuesFromPartitionPath) {
       val partitionSchema = 
filterInPartitionColumns(tableSchema.structTypeSchema)
       val prunedDataStructSchema = 
prunePartitionColumns(tableSchema.structTypeSchema)
-      val prunedRequiredSchema = 
prunePartitionColumns(requiredSchema.structTypeSchema)
+      val prunedDataInternalSchema = pruneInternalSchema(tableSchema, 
prunedDataStructSchema)
+      val prunedRequiredStructSchema = 
prunePartitionColumns(requiredSchema.structTypeSchema)
+      val prunedRequiredInternalSchema = pruneInternalSchema(requiredSchema, 
prunedRequiredStructSchema)
 
       (partitionSchema,
-        HoodieTableSchema(prunedDataStructSchema, 
convertToAvroSchema(prunedDataStructSchema, tableName).toString),
-        HoodieTableSchema(prunedRequiredSchema, 
convertToAvroSchema(prunedRequiredSchema, tableName).toString))
+        HoodieTableSchema(prunedDataStructSchema,
+          convertToAvroSchema(prunedDataStructSchema, tableName).toString, 
prunedDataInternalSchema),
+        HoodieTableSchema(prunedRequiredStructSchema,
+          convertToAvroSchema(prunedRequiredStructSchema, tableName).toString, 
prunedRequiredInternalSchema))
     } else {
       (StructType(Nil), tableSchema, requiredSchema)
     }
   }
 
+  private def pruneInternalSchema(hoodieTableSchema: HoodieTableSchema, 
prunedStructSchema: StructType): Option[InternalSchema] = {
+    if (hoodieTableSchema.internalSchema.isEmpty || 
hoodieTableSchema.internalSchema.get.isEmptySchema) {
+      Option.empty[InternalSchema]
+    } else {
+      
Some(InternalSchemaUtils.pruneInternalSchema(hoodieTableSchema.internalSchema.get,
+        prunedStructSchema.fields.map(_.name).toList.asJava))
+    }
+  }
+
   private def filterInPartitionColumns(structType: StructType): StructType =
     StructType(structType.filter(f => partitionColumns.exists(col => 
resolver(f.name, col))))
 

Reply via email to