rahil-c commented on code in PR #17573:
URL: https://github.com/apache/hudi/pull/17573#discussion_r2617417565


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -795,29 +793,30 @@ object HoodieBaseRelation extends SparkAdapterSupport {
    * @param tableSchema schema to project (either of [[InternalSchema]] or 
Avro's [[Schema]])
    * @param requiredColumns required top-level columns to be projected
    */
-  def projectSchema(tableSchema: Either[Schema, InternalSchema], 
requiredColumns: Array[String]): (Schema, StructType, InternalSchema) = {
+  def projectSchema(tableSchema: Either[HoodieSchema, InternalSchema], 
requiredColumns: Array[String]): (HoodieSchema, StructType, InternalSchema) = {
     tableSchema match {
       case Right(internalSchema) =>
         checkState(!internalSchema.isEmptySchema)
         val prunedInternalSchema = 
InternalSchemaUtils.pruneInternalSchema(internalSchema, 
requiredColumns.toList.asJava)
-        val requiredAvroSchema = 
InternalSchemaConverter.convert(prunedInternalSchema, "schema").toAvroSchema
-        val requiredStructSchema = 
AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
+        val requiredSchema = 
InternalSchemaConverter.convert(prunedInternalSchema, "schema")
+        val requiredStructSchema = 
HoodieSchemaConversionUtils.convertHoodieSchemaToStructType(requiredSchema)
 
-        (requiredAvroSchema, requiredStructSchema, prunedInternalSchema)
+        (requiredSchema, requiredStructSchema, prunedInternalSchema)
 
-      case Left(avroSchema) =>
-        val fieldMap = avroSchema.getFields.asScala.map(f => f.name() -> 
f).toMap
+      case Left(hoodieSchema) =>
+        val fieldMap = hoodieSchema.getFields.asScala.map(f => f.name() -> 
f).toMap
         val requiredFields = requiredColumns.map { col =>
-          val f = fieldMap(col)
-          // We have to create a new [[Schema.Field]] since Avro schemas can't 
share field
-          // instances (and will throw "org.apache.avro.AvroRuntimeException: 
Field already used")
-          createNewSchemaField(f.name(), f.schema(), f.doc(), f.defaultVal(), 
f.order())

Review Comment:
   yea i think i faced an issue in some of the tests so added this 
https://github.com/apache/hudi/commit/9fe07c055ee605e6d981918e686bb03f6de4070c
   ```
   // We have to create a new HoodieSchemaField since Avro schemas can't share 
field
             // instances (and will throw 
"org.apache.avro.AvroRuntimeException: Field already used")
             HoodieSchemaField.of(f.name(), f.schema(), f.doc().orElse(null), 
f.defaultVal().orElse(null))
           }.toList
   
   ```



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