the-other-tim-brown commented on code in PR #17475:
URL: https://github.com/apache/hudi/pull/17475#discussion_r2596421100


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieSchemaInternalConverters.scala:
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.avro
+
+import org.apache.hudi.HoodieSchemaConverters
+import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaField, 
HoodieSchemaType}
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.sql.types._
+
+import scala.collection.JavaConverters._
+
+/**
+ * This object contains methods that are used to convert HoodieSchema to Spark 
SQL schemas and vice versa.
+ *
+ * NOTE: This provides direct conversion between HoodieSchema and Spark 
DataType
+ * without going through Avro Schema intermediary.
+ */
+
+@DeveloperApi
+private[sql] object HoodieSchemaInternalConverters extends 
HoodieSchemaConverters {
+
+  /**
+   * Internal wrapper for SQL data type and nullability.
+   */
+  case class SchemaType(dataType: DataType, nullable: Boolean)
+
+  override def toSqlType(hoodieSchema: HoodieSchema): (DataType, Boolean) = {
+    val result = toSqlTypeHelper(hoodieSchema, Set.empty)
+    (result.dataType, result.nullable)
+  }
+
+  override def toHoodieType(catalystType: DataType, nullable: Boolean, 
recordName: String, nameSpace: String): HoodieSchema = {
+    val schema = catalystType match {
+      // Primitive types
+      case BooleanType => HoodieSchema.create(HoodieSchemaType.BOOLEAN)
+      case ByteType | ShortType | IntegerType => 
HoodieSchema.create(HoodieSchemaType.INT)
+      case LongType => HoodieSchema.create(HoodieSchemaType.LONG)
+      case DateType => HoodieSchema.createDate()
+      case TimestampType => HoodieSchema.createTimestampMicros()
+      case TimestampNTZType => HoodieSchema.createLocalTimestampMicros()
+      case FloatType => HoodieSchema.create(HoodieSchemaType.FLOAT)
+      case DoubleType => HoodieSchema.create(HoodieSchemaType.DOUBLE)
+      case StringType | _: CharType | _: VarcharType => 
HoodieSchema.create(HoodieSchemaType.STRING)
+      case NullType => HoodieSchema.create(HoodieSchemaType.NULL)
+      case BinaryType => HoodieSchema.create(HoodieSchemaType.BYTES)
+
+      case d: DecimalType =>
+        HoodieSchema.createDecimal(d.precision, d.scale)
+
+      // Complex types
+      case ArrayType(elementType, containsNull) =>
+        val elementSchema = toHoodieType(elementType, containsNull, 
recordName, nameSpace)
+        HoodieSchema.createArray(elementSchema)
+
+      case MapType(StringType, valueType, valueContainsNull) =>
+        val valueSchema = toHoodieType(valueType, valueContainsNull, 
recordName, nameSpace)
+        HoodieSchema.createMap(valueSchema)
+
+      case st: StructType =>
+        val childNameSpace = if (nameSpace != "") s"$nameSpace.$recordName" 
else recordName
+
+        // Check if this might be a union (using heuristic like Avro converter)
+        if (canBeUnion(st)) {
+          val nonNullUnionFieldTypes = st.map { f =>
+            toHoodieType(f.dataType, nullable = false, f.name, childNameSpace)
+          }
+          val unionFieldTypes = if (nullable) {
+            val types = new java.util.ArrayList[HoodieSchema]()
+            types.add(HoodieSchema.create(HoodieSchemaType.NULL))
+            nonNullUnionFieldTypes.foreach(types.add)
+            types
+          } else {
+            val types = new java.util.ArrayList[HoodieSchema]()
+            nonNullUnionFieldTypes.foreach(types.add)
+            types
+          }
+          HoodieSchema.createUnion(unionFieldTypes)
+        } else {
+          // Create record
+          val fields = st.map { f =>
+            val fieldSchema = toHoodieType(f.dataType, f.nullable, f.name, 
childNameSpace)
+            val doc = if (f.metadata.contains("comment")) 
f.metadata.getString("comment") else null
+            HoodieSchemaField.of(f.name, fieldSchema, doc)
+          }
+
+          val fieldsJava = new java.util.ArrayList[HoodieSchemaField]()
+          fields.foreach(fieldsJava.add)
+
+          HoodieSchema.createRecord(recordName, nameSpace, null, fieldsJava)
+        }
+
+      case other => throw new IncompatibleSchemaException(s"Unexpected Spark 
DataType: $other")
+    }
+
+    // Wrap with null union if nullable (and not already a union)
+    if (nullable && catalystType != NullType && schema.getType != 
HoodieSchemaType.UNION) {
+      HoodieSchema.createNullable(schema)
+    } else {
+      schema
+    }
+  }
+
+  private def toSqlTypeHelper(hoodieSchema: HoodieSchema, existingRecordNames: 
Set[String]): SchemaType = {
+    hoodieSchema.getType match {
+      // Primitive types
+      case HoodieSchemaType.INT => SchemaType(IntegerType, nullable = false)
+      case HoodieSchemaType.STRING => SchemaType(StringType, nullable = false)
+      case HoodieSchemaType.BOOLEAN => SchemaType(BooleanType, nullable = 
false)
+      case HoodieSchemaType.BYTES => SchemaType(BinaryType, nullable = false)
+      case HoodieSchemaType.DOUBLE => SchemaType(DoubleType, nullable = false)
+      case HoodieSchemaType.FLOAT => SchemaType(FloatType, nullable = false)
+      case HoodieSchemaType.LONG => SchemaType(LongType, nullable = false)
+      case HoodieSchemaType.NULL => SchemaType(NullType, nullable = true)
+      case HoodieSchemaType.ENUM => SchemaType(StringType, nullable = false)
+
+      // Logical types
+      case HoodieSchemaType.DATE =>
+        SchemaType(DateType, nullable = false)
+
+      case HoodieSchemaType.TIMESTAMP =>
+        hoodieSchema match {
+          case ts: HoodieSchema.Timestamp =>
+            if (ts.isUtcAdjusted) {
+              SchemaType(TimestampType, nullable = false)
+            } else {
+              SchemaType(TimestampNTZType, nullable = false)
+            }
+          case _ =>
+            // Fallback for non-specialized timestamp schema
+            SchemaType(TimestampType, nullable = false)
+        }
+
+      case HoodieSchemaType.DECIMAL =>
+        hoodieSchema match {
+          case dec: HoodieSchema.Decimal =>
+            SchemaType(DecimalType(dec.getPrecision, dec.getScale), nullable = 
false)
+          case _ =>
+            throw new IncompatibleSchemaException(
+              s"DECIMAL type must be HoodieSchema.Decimal instance, got: 
${hoodieSchema.getClass}")
+        }
+
+      case HoodieSchemaType.FIXED =>
+        // FIXED can be either binary or decimal with logical type
+        hoodieSchema match {
+          case dec: HoodieSchema.Decimal =>
+            SchemaType(DecimalType(dec.getPrecision, dec.getScale), nullable = 
false)
+          case _ =>
+            SchemaType(BinaryType, nullable = false)
+        }
+
+      // Complex types
+      case HoodieSchemaType.RECORD =>
+        val fullName = hoodieSchema.getFullName
+        if (existingRecordNames.contains(fullName)) {
+          throw new IncompatibleSchemaException(
+            s"""
+               |Found recursive reference in HoodieSchema, which cannot be 
processed by Spark:
+               |$fullName
+             """.stripMargin)
+        }
+        val newRecordNames = existingRecordNames + fullName
+        val fields = hoodieSchema.getFields.asScala.map { f =>
+          val schemaType = toSqlTypeHelper(f.schema(), newRecordNames)
+          StructField(f.name(), schemaType.dataType, schemaType.nullable)

Review Comment:
   
https://github.com/apache/hudi/blob/master/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala#L105



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