-
Notifications
You must be signed in to change notification settings - Fork 2.5k
feat: Add Unshredded Variant read & write support #17833
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
This file was deleted.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,28 +18,31 @@ | |
|
|
||
| package org.apache.spark.sql.avro | ||
|
|
||
| import org.apache.hudi.common.schema.HoodieSchema.TimePrecision | ||
| import org.apache.hudi.SparkAdapterSupport | ||
| import org.apache.hudi.common.schema.{HoodieJsonProperties, HoodieSchema, HoodieSchemaField, HoodieSchemaType} | ||
| import org.apache.hudi.common.schema.HoodieSchema.TimePrecision | ||
|
|
||
| import org.apache.spark.annotation.DeveloperApi | ||
| import org.apache.spark.sql.types.Decimal.minBytesForPrecision | ||
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.sql.types.Decimal.minBytesForPrecision | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| /** | ||
| * This object contains methods that are used to convert HoodieSchema to Spark SQL schemas and vice versa. | ||
| * Object containing methods to convert HoodieSchema to Spark SQL schemas and vice versa. | ||
| * | ||
| * This provides direct conversion between HoodieSchema and Spark DataType | ||
| * without going through Avro Schema intermediary. | ||
| * | ||
| * Version-specific types (like VariantType in Spark >4.x) are handled via SparkAdapterSupport. | ||
| * | ||
| * NOTE: the package of this class is intentionally kept as "org.apache.spark.sql.avro" which is similar to the existing | ||
| * Spark Avro connector's SchemaConverters.scala | ||
| * (https://github.com/apache/spark/blob/master/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala). | ||
| * The reason for this is so that Spark 3.3 is able to access private spark sql type classes like TimestampNTZType. | ||
| */ | ||
|
|
||
| @DeveloperApi | ||
| object HoodieSparkSchemaConverters { | ||
| object HoodieSparkSchemaConverters extends SparkAdapterSupport { | ||
|
|
||
| /** | ||
| * Internal wrapper for SQL data type and nullability. | ||
|
|
@@ -118,7 +121,12 @@ object HoodieSparkSchemaConverters { | |
| HoodieSchema.createRecord(recordName, nameSpace, null, fields.asJava) | ||
| } | ||
|
|
||
| case other => throw new IncompatibleSchemaException(s"Unexpected Spark DataType: $other") | ||
| // VARIANT type (Spark >4.x only), which will be handled via SparkAdapter | ||
| case other if sparkAdapter.isVariantType(other) => | ||
| HoodieSchema.createVariant(recordName, nameSpace, null) | ||
|
|
||
| case other => | ||
| throw new IncompatibleSchemaException(s"Unexpected Spark DataType: $other") | ||
| } | ||
|
|
||
| // Wrap with null union if nullable (and not already a union) | ||
|
|
@@ -129,6 +137,9 @@ object HoodieSparkSchemaConverters { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Helper method to convert HoodieSchema to Catalyst DataType. | ||
| */ | ||
| private def toSqlTypeHelper(hoodieSchema: HoodieSchema, existingRecordNames: Set[String]): SchemaType = { | ||
| hoodieSchema.getType match { | ||
| // Primitive types | ||
|
|
@@ -241,7 +252,16 @@ object HoodieSparkSchemaConverters { | |
| } | ||
| } | ||
|
|
||
| case other => throw new IncompatibleSchemaException(s"Unsupported HoodieSchemaType: $other") | ||
| // VARIANT type (Spark >4.x only), which will be handled via SparkAdapter | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For lower spark versions, do we want to just return the underlying struct?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In the current implementation in Spark3.5, this is dead code as the variant column will not have a logicalType of variant. It's a record instead. The reason why it's dead code is because is:
This might change if the table has an internalSchema though, which i think we need to investigate. I'll create an issue for this! Will inline the issue too. |
||
| // 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+") | ||
| } | ||
|
|
||
| case other => | ||
| throw new IncompatibleSchemaException(s"Unsupported HoodieSchemaType: $other") | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,11 +19,13 @@ | |
|
|
||
| package org.apache.spark.sql.execution.datasources.parquet | ||
|
|
||
| import org.apache.hudi.HoodieSparkUtils | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.parquet.hadoop.metadata.FileMetaData | ||
| import org.apache.spark.sql.HoodieSchemaUtils | ||
| import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection | ||
| import org.apache.spark.sql.catalyst.expressions.{ArrayTransform, Attribute, Cast, CreateNamedStruct, CreateStruct, Expression, GetStructField, LambdaFunction, Literal, MapEntries, MapFromEntries, NamedLambdaVariable, UnsafeProjection} | ||
| import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection | ||
| import org.apache.spark.sql.types.{ArrayType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampNTZType} | ||
|
|
||
| object HoodieParquetFileFormatHelper { | ||
|
|
@@ -46,7 +48,7 @@ object HoodieParquetFileFormatHelper { | |
| val requiredType = f.dataType | ||
| if (fileStructMap.contains(f.name) && !isDataTypeEqual(requiredType, fileStructMap(f.name))) { | ||
| val readerType = addMissingFields(requiredType, fileStructMap(f.name)) | ||
| implicitTypeChangeInfo.put(new Integer(requiredSchema.fieldIndex(f.name)), org.apache.hudi.common.util.collection.Pair.of(requiredType, readerType)) | ||
| implicitTypeChangeInfo.put(Integer.valueOf(requiredSchema.fieldIndex(f.name)), org.apache.hudi.common.util.collection.Pair.of(requiredType, readerType)) | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| StructField(f.name, readerType, f.nullable) | ||
| } else { | ||
| f | ||
|
|
@@ -55,35 +57,43 @@ object HoodieParquetFileFormatHelper { | |
| (implicitTypeChangeInfo, StructType(sparkRequestStructFields)) | ||
| } | ||
|
|
||
| def isDataTypeEqual(requiredType: DataType, fileType: DataType): Boolean = (requiredType, fileType) match { | ||
| case (requiredType, fileType) if requiredType == fileType => true | ||
| def isDataTypeEqual(requiredType: DataType, fileType: DataType): Boolean = { | ||
| // Check if adapter can handle this comparison (e.g., VariantType in Spark 4.0+) | ||
| val adapterResult = HoodieSparkUtils.sparkAdapter.isDataTypeEqualForParquet(requiredType, fileType) | ||
| if (adapterResult.isDefined) { | ||
| adapterResult.get | ||
| } else { | ||
| (requiredType, fileType) match { | ||
| case (requiredType, fileType) if requiredType == fileType => true | ||
|
|
||
| // prevent illegal cast | ||
| case (TimestampNTZType, LongType) => true | ||
| // prevent illegal cast | ||
| case (TimestampNTZType, LongType) => true | ||
|
|
||
| case (ArrayType(rt, _), ArrayType(ft, _)) => | ||
| // Do not care about nullability as schema evolution require fields to be nullable | ||
| isDataTypeEqual(rt, ft) | ||
| case (ArrayType(rt, _), ArrayType(ft, _)) => | ||
| // Do not care about nullability as schema evolution require fields to be nullable | ||
| isDataTypeEqual(rt, ft) | ||
|
|
||
| case (MapType(requiredKey, requiredValue, _), MapType(fileKey, fileValue, _)) => | ||
| // Likewise, do not care about nullability as schema evolution require fields to be nullable | ||
| isDataTypeEqual(requiredKey, fileKey) && isDataTypeEqual(requiredValue, fileValue) | ||
| case (MapType(requiredKey, requiredValue, _), MapType(fileKey, fileValue, _)) => | ||
| // Likewise, do not care about nullability as schema evolution require fields to be nullable | ||
| isDataTypeEqual(requiredKey, fileKey) && isDataTypeEqual(requiredValue, fileValue) | ||
|
|
||
| case (StructType(requiredFields), StructType(fileFields)) => | ||
| // Find fields that are in requiredFields and fileFields as they might not be the same during add column + change column operations | ||
| val commonFieldNames = requiredFields.map(_.name) intersect fileFields.map(_.name) | ||
| case (StructType(requiredFields), StructType(fileFields)) => | ||
| // Find fields that are in requiredFields and fileFields as they might not be the same during add column + change column operations | ||
| val commonFieldNames = requiredFields.map(_.name) intersect fileFields.map(_.name) | ||
|
|
||
| // Need to match by name instead of StructField as name will stay the same whilst type may change | ||
| val fileFilteredFields = fileFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) | ||
| val requiredFilteredFields = requiredFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) | ||
| // Need to match by name instead of StructField as name will stay the same whilst type may change | ||
| val fileFilteredFields = fileFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) | ||
| val requiredFilteredFields = requiredFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) | ||
|
|
||
| // Sorting ensures that the same field names are being compared for type differences | ||
| requiredFilteredFields.zip(fileFilteredFields).forall { | ||
| case (requiredField, fileFilteredField) => | ||
| isDataTypeEqual(requiredField.dataType, fileFilteredField.dataType) | ||
| } | ||
| // Sorting ensures that the same field names are being compared for type differences | ||
| requiredFilteredFields.zip(fileFilteredFields).forall { | ||
| case (requiredField, fileFilteredField) => | ||
| isDataTypeEqual(requiredField.dataType, fileFilteredField.dataType) | ||
| } | ||
|
|
||
| case _ => false | ||
| case _ => false | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def addMissingFields(requiredType: DataType, fileType: DataType): DataType = (requiredType, fileType) match { | ||
|
|
@@ -195,3 +205,4 @@ object HoodieParquetFileFormatHelper { | |
| } | ||
| } | ||
| } | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need something similar in
HoodieAvroWriteSupport?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Don't think so, i have test that uses
HoodieRecordType.{AVRO, SPARK}. They should trigger both write support and it seems there are no test failures.In Avro, Variant is already an Avro record from
HoodieSchema.createVariant. WhereFields: value (bytes), metadata (bytes).IIUC, Parquet's AvroWriteSupport handles this automatically as it will know how to convert:
HoodieAvroWriteSupportjust wrapsAvroWriteSupportto add bloom filter support and does not override write logic.In the Spark Row path, custom handling is needed because Spark's
VariantTyperequires special APIs (createVariantValueWriter) to extract the raw bytes as there are no automatic Spark VariantType -> Parquet conversion from what i can see in our code.