balaji-varadarajan-ai commented on code in PR #17833:
URL: https://github.com/apache/hudi/pull/17833#discussion_r2913023720


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala:
##########
@@ -293,7 +304,16 @@ object HoodieSparkSchemaConverters {
           }
         }
 
-      case other => throw new IncompatibleSchemaException(s"Unsupported 
HoodieSchemaType: $other")
+      // VARIANT type (Spark >4.x only), which will be handled via SparkAdapter
+      // TODO: Check if internalSchema will throw any errors here: #18021
+      case HoodieSchemaType.VARIANT =>
+        sparkAdapter.getVariantDataType match {
+          case Some(variantType) => SchemaType(variantType, nullable = false)
+          case None => throw new IncompatibleSchemaException("VARIANT type is 
only supported in Spark 4.0+")

Review Comment:
   just checking, should we be returning as binary type here as opposed to this 
? 



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/HoodieSchemaConverter.java:
##########
@@ -550,6 +552,26 @@ private static DataType convertUnion(HoodieSchema schema) {
     return nullable ? rawDataType.nullable() : rawDataType;
   }
 
+  /**
+   * Converts a Variant schema to Flink's ROW type.
+   * Variant is represented as ROW<`value` BYTES, `metadata` BYTES> in Flink.
+   * // TODO: We are only supporting unshredded for now, support shredded in 
the future
+   *
+   * @param schema HoodieSchema to convert (must be a VARIANT type)
+   * @return DataType representing the Variant as a ROW with binary fields
+   */
+  private static DataType convertVariant(HoodieSchema schema) {
+    if (schema.getType() != HoodieSchemaType.VARIANT) {
+      throw new IllegalStateException("Expected HoodieSchema.Variant but got: 
" + schema.getClass());
+    }
+
+    // Variant is stored as a struct with two binary fields: value and metadata
+    return DataTypes.ROW(

Review Comment:
   same issue here about the ordering .  seems inconsistent with Parquet. 



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkSchemaTransformUtils.scala:
##########
@@ -19,10 +19,11 @@
 
 package org.apache.spark.sql.execution.datasources
 
+import org.apache.hudi.HoodieSparkUtils
 import org.apache.spark.sql.HoodieSchemaUtils
 import 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
 import org.apache.spark.sql.catalyst.expressions.{ArrayTransform, Attribute, 
AttributeReference, Cast, CreateNamedStruct, CreateStruct, Expression, 
GetStructField, LambdaFunction, Literal, MapEntries, MapFromEntries, 
NamedLambdaVariable, UnsafeProjection}
-import org.apache.spark.sql.types.{ArrayType, DataType, DateType, DecimalType, 
DoubleType, FloatType, IntegerType, LongType, MapType, StringType, StructField, 
StructType, TimestampNTZType}
+import org.apache.spark.sql.types._

Review Comment:
   +1



##########
hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala:
##########
@@ -259,6 +260,32 @@ private[sql] class AvroSerializer(rootCatalystType: 
DataType,
           java.util.Arrays.asList(result: _*)
         }
 
+      case (VariantType, RECORD) if avroType.getProp("logicalType") == 
HoodieSchema.VARIANT_TYPE_NAME =>

Review Comment:
   @voonhous : Have you run an end-to-end round-trip test specifically through 
the MOR Avro path?  it's worth confirming that the test asserts on actual 
byte-level fidelity and not just that no exception is thrown.



##########
hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java:
##########
@@ -2383,6 +2389,20 @@ public String getName() {
       return VARIANT_DEFAULT_NAME;
     }
 
+    /**
+     * Returns the type of this schema.
+     * Note: This override is not strictly necessary as the base class 
constructor
+     * already sets the type correctly via HoodieSchemaType.fromAvro() which 
detects
+     * the VariantLogicalType. This explicit override is provided for 
consistency
+     * with other logical type subclasses (e.g., Blob) and for clarity.
+     *
+     * @return HoodieSchemaType.VARIANT
+     */
+    @Override
+    public HoodieSchemaType getType() {

Review Comment:
   The introduction of HoodieSchemaType.VARIANT is fine, but it's worth asking: 
are there any switch statements or match blocks elsewhere in the codebase that 
enumerate HoodieSchemaType values without a default/catch-all case? If so, 
adding a new enum value silently breaks those paths. Ideally a grep for 
exhaustive enum usages would be part of this review.  Just want to check if you 
did an exhaustive grep ? 



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -2054,9 +2054,13 @@ private static boolean 
isColumnTypeSupportedV1(HoodieSchema schema, Option<Hoodi
   private static boolean isColumnTypeSupportedV2(HoodieSchema schema) {
     HoodieSchemaType type = schema.getType();
     // Check for precision and scale if the schema has a logical decimal type.
+    // VARIANT (unshredded) type is excluded because it stores semi-structured 
data as opaque binary blobs,
+    // making min/max statistics meaningless
+    // TODO: For shredded, we are able to store colstats, explore that: #17988
     return type != HoodieSchemaType.RECORD && type != HoodieSchemaType.MAP

Review Comment:
   High level question : The RFC mentions column statistics and indexing as 
explicitly unsupported for Variant. It would be good to see explicit guard 
clauses or early exits in HoodieTableMetadataUtil when processing Variant 
columns, rather than silently skipping them or — worse — attempting to compute 
statistics on raw binary blobs and getting garbage output. Have you audited 
what happens today if a Variant column makes it into the column stats 
computation path?



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