voonhous commented on code in PR #17833:
URL: https://github.com/apache/hudi/pull/17833#discussion_r2945092862
##########
hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala:
##########
@@ -196,4 +198,80 @@ abstract class BaseSpark4Adapter extends SparkAdapter with
Logging {
storageConf.getBoolean(SQLConf.CASE_SENSITIVE.key,
sqlConf.caseSensitiveAnalysis),
getRebaseSpec("CORRECTED"))
}
+
+ override def getVariantDataType: Option[DataType] = {
+ Some(VariantType)
+ }
+
+ override def isDataTypeEqualForParquet(requiredType: DataType, fileType:
DataType): Option[Boolean] = {
+ /**
+ * Checks if a StructType is the physical representation of VariantType in
Parquet.
+ * VariantType is stored in Parquet as a struct with two binary fields:
"value" and "metadata".
+ */
+ def isVariantPhysicalSchema(structType: StructType): Boolean = {
+ if (structType.fields.length != 2) {
+ false
+ } else {
+ val fieldMap = structType.fields.map(f => (f.name, f.dataType)).toMap
+ fieldMap.contains("value") && fieldMap.contains("metadata") &&
+ fieldMap("value") == BinaryType && fieldMap("metadata") == BinaryType
+ }
+ }
+
+ // Handle VariantType comparisons
+ (requiredType, fileType) match {
+ case (_: VariantType, s: StructType) if isVariantPhysicalSchema(s) =>
Some(true)
+ case (s: StructType, _: VariantType) if isVariantPhysicalSchema(s) =>
Some(true)
+ case _ => None // Not a VariantType comparison, use default logic
+ }
+ }
+
+ override def isVariantType(dataType: DataType): Boolean = {
+ dataType.isInstanceOf[VariantType]
+ }
+
+ override def createVariantValueWriter(
+ dataType: DataType,
+ writeValue: Consumer[Array[Byte]],
+ writeMetadata: Consumer[Array[Byte]]
+ ): BiConsumer[SpecializedGetters, Integer] = {
+ if (!isVariantType(dataType)) {
+ throw new IllegalArgumentException(s"Expected VariantType but got
$dataType")
+ }
+
+ (row: SpecializedGetters, ordinal: Integer) => {
+ val variant = row.getVariant(ordinal)
+ writeValue.accept(variant.getValue)
+ writeMetadata.accept(variant.getMetadata)
+ }
+ }
+
+ override def convertVariantFieldToParquetType(
+ dataType: DataType,
+ fieldName: String,
+ fieldSchema: HoodieSchema,
+ repetition: Repetition
+ ): Type = {
+ if (!isVariantType(dataType)) {
+ throw new IllegalArgumentException(s"Expected VariantType but got
$dataType")
+ }
+
+ // Determine if this is a shredded variant
+ val isShredded = fieldSchema match {
+ case variant: HoodieSchema.Variant => variant.isShredded
+ case _ => false
+ }
+
+ // For shredded variants, the value field is OPTIONAL (nullable)
+ // For unshredded variants, the value field is REQUIRED
+ val valueRepetition = if (isShredded) Repetition.OPTIONAL else
Repetition.REQUIRED
+
+ // VariantType is always stored in Parquet as a struct with separate value
and metadata binary fields.
+ // This matches how the HoodieRowParquetWriteSupport writes variant data.
+ // Note: We intentionally omit 'typed_value' for shredded variants as this
writer only accesses raw binary blobs.
+ Types.buildGroup(repetition)
+ .addField(Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY,
valueRepetition).named("value"))
+ .addField(Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY,
Repetition.REQUIRED).named("metadata"))
+ .named(fieldName)
Review Comment:
Spark4.0 uses parquet 1.15.2, if our version here mismatches with Spark's,
we run the risk of class conflicts.
The alternative is shading, but my shading opens another can of worms where
after we support a spark version > 4.0, we need to remove the shading again.
The TODO of removing the shading might get lost somewhere in along the
development lifecycle.
--
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]