Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,10 @@ import org.apache.avro.{AvroRuntimeException, Schema}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.avro.HoodieSparkSchemaConverters
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType}
import org.apache.spark.sql.types.{ArrayType, DataType, MapType, MetadataBuilder, StructField, StructType}
Comment thread
voonhous marked this conversation as resolved.
import org.apache.spark.sql.{Dataset, Row, SparkSession}

import java.util.Locale
import java.util.concurrent.ConcurrentHashMap

import scala.collection.JavaConverters._
Expand Down Expand Up @@ -145,6 +146,82 @@ object HoodieSchemaConversionUtils {
}
}

/**
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@voonhous you think we can make this java doc comment more concise if possibe?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's try to use multi-line selects next time.

image

This is not showing everything...

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

* Re-attach catalog metadata/nullability that Spark's write-path rewrites strip:
*
* 1. VECTOR/BLOB logical-type metadata - dropped by TableOutputResolver's Cast and
* UPDATE's castIfNeeded; without it, conversion falls back to the physical type
* (ARRAY/STRUCT) and schema compatibility fails.
* 2. Nullability (only when `alignNullability = true`) - VALUES literals and Cast
* outputs are nullable by default, which conflicts with non-null catalog columns
* (e.g. primaryKey) once this schema becomes the writer's canonical schema.
*
* `alignNullability = true` is only safe when upstream null-assertion expressions
* guarantee no nulls flow into non-null columns (INSERT/UPDATE via TableOutputResolver/
* castIfNeeded). MERGE must pass `false`: assignments run later inside ExpressionPayload,
* so raw source rows may legitimately carry nulls.
*
* BLOB is projected nullable-everywhere by [[HoodieSparkSchemaConverters.toSqlType]], so
* the nullability branch is a no-op inside BLOB subtrees; RFC-100 non-null invariants are
* enforced at the physical-schema boundary in HoodieSchema.Blob#createBlob.
*
* Recurses into nested structs, struct-element arrays, and struct-value maps. Source-only
* fields (e.g. MERGE join keys) are returned unchanged.
*/
def alignSchemaWithCatalog(sourceSchema: StructType,
Comment thread
voonhous marked this conversation as resolved.
targetSchema: StructType,
caseSensitive: Boolean,
alignNullability: Boolean): StructType = {
val lookupKey: String => String =
if (caseSensitive) identity else (_: String).toLowerCase(Locale.ROOT)
val targetByName: Map[String, StructField] =
targetSchema.fields.map(f => lookupKey(f.name) -> f).toMap

StructType(sourceSchema.fields.map { field =>
targetByName.get(lookupKey(field.name)) match {
case Some(target) => alignField(field, target, caseSensitive, alignNullability)
case None => field
}
})
}

private def alignField(source: StructField,
Comment thread
voonhous marked this conversation as resolved.
target: StructField,
caseSensitive: Boolean,
alignNullability: Boolean): StructField = {
val alignedNullable = if (alignNullability) target.nullable else source.nullable
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤖 For VECTOR (ArrayType[FloatType]), neither element is a StructType, so this branch is skipped and only top-level nullability is adjusted; source's containsNull is preserved. In the MERGE path (alignNullability=false) with literals like array(cast(0.9 as float), ...) whose containsNull may be true, the metadata is re-attached on top of an array-of-nullable-floats. Have you verified that convertStructTypeToHoodieSchema emits the correct non-null VECTOR element schema in that case (rather than e.g. failing schema-compat with a nullable element union)?

- AI-generated; verify before applying. React 👍/👎 to flag quality.

val alignedField = (source.dataType, target.dataType) match {
case (s: StructType, t: StructType) =>
source.copy(
dataType = alignSchemaWithCatalog(s, t, caseSensitive, alignNullability),
nullable = alignedNullable)
case (ArrayType(sElem: StructType, sContainsNull), ArrayType(tElem: StructType, tContainsNull)) =>
val alignedContainsNull = if (alignNullability) tContainsNull else sContainsNull
source.copy(
dataType = ArrayType(alignSchemaWithCatalog(sElem, tElem, caseSensitive, alignNullability), alignedContainsNull),
nullable = alignedNullable)
case (MapType(sKey, sVal: StructType, sValueContainsNull), MapType(_, tVal: StructType, tValueContainsNull)) =>
val alignedValueContainsNull = if (alignNullability) tValueContainsNull else sValueContainsNull
source.copy(
dataType = MapType(sKey, alignSchemaWithCatalog(sVal, tVal, caseSensitive, alignNullability), alignedValueContainsNull),
nullable = alignedNullable)
case _ =>
source.copy(nullable = alignedNullable)
}

if (target.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) {
val enrichedMetadata = new MetadataBuilder()
.withMetadata(alignedField.metadata)
.putString(
HoodieSchema.TYPE_METADATA_FIELD,
target.metadata.getString(HoodieSchema.TYPE_METADATA_FIELD))
.build()
alignedField.copy(metadata = enrichedMetadata)
} else {
alignedField
}
}

/**
* Recursively aligns the nullable property of Spark schema fields with HoodieSchema.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,12 @@ import org.apache.hudi.common.schema.HoodieSchema.TimePrecision
import org.apache.hudi.internal.schema.HoodieSchemaException

import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.types.Decimal.minBytesForPrecision

import java.util.Locale

import scala.collection.JavaConverters._

/**
Expand Down Expand Up @@ -257,6 +260,7 @@ object HoodieSparkSchemaConverters extends SparkAdapterSupport {
SchemaType(ArrayType(sparkElementType, containsNull = false), nullable = false, Some(metadata))

case HoodieSchemaType.BLOB | HoodieSchemaType.RECORD =>
val isBlob = hoodieSchema.getType == HoodieSchemaType.BLOB
val fullName = hoodieSchema.getFullName
if (existingRecordNames.contains(fullName)) {
throw new IncompatibleSchemaException(
Expand All @@ -278,10 +282,22 @@ object HoodieSparkSchemaConverters extends SparkAdapterSupport {
metadataBuilder.putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchema.Blob.TYPE_DESCRIPTOR)
}
val metadata = metadataBuilder.build()
StructField(f.name(), schemaType.dataType, schemaType.nullable, metadata)
// For BLOB: force nullable-everywhere at the Spark type layer. The RFC-100
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@yihua @voonhous I really feel like we should revisit the BLOB schema definition around what fields should be non-nullable, as this seems to cause complexities in this and others PRs as well as UX of having to provide values for sub fields that may not be needed during write time.

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@yihua Should we file some github issue for this to maybe relax this constraint?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Uhm, KIV this please, i have encountered quite a bit of errors around this.This is not a 1-day ordeal. Let's take more time to look into this after 1.2 as i've found that Spark3.3,3.4,3.5 and 4.0 have different schema validation/check flows.

And i have encountered quite abit of errors around this error which i had to fix and patch.

Agree that we should file an issue here and find the best way forward.

// canonical schema declares `type`, `reference.external_path`, and
// `reference.managed` as strictly non-null, but that contract is conditional
// ("required when parent is present") and Spark's type system can't model it.
// Projecting BLOB as nullable-everywhere for Spark avoids downstream pain
// (Cast / TableOutputResolver / Cast.canCast rewrites); the on-disk physical
// schema stays RFC-100 compliant because the write path goes through
// HoodieSchema.Blob.createBlob(), which uses the canonical fields verbatim.
if (isBlob) {
StructField(f.name(), withAllFieldsNullable(schemaType.dataType), nullable = true, metadata)
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@voonhous @yihua Does the change for now mean that then users can provide something like this, where user can pass null type?

  ▎ CREATE TABLE blob_t (id BIGINT, payload BLOB) USING hudi                          
  ▎   TBLPROPERTIES (primaryKey = 'id');
  ▎                                                                                   
  ▎ -- type=null: Spark side BLOB.type is now nullable, so analysis passes.           
  ▎ -- Avro side BLOB.type is the non-null enum {INLINE, OUT_OF_LINE} → fails at      
  ▎ encode.                                                                           
  ▎ INSERT INTO blob_t VALUES (
  ▎   1,                                                                              
  ▎   named_struct(
  ▎     'type', cast(null as string),
  ▎     'data', cast(X'010203' as binary),
  ▎     'reference', cast(null as                                                     
  ▎ struct<external_path:string,offset:bigint,length:bigint,managed:boolean>)         
  ▎   )  

Copy link
Copy Markdown
Member Author

@voonhous voonhous Apr 26, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Your example should fail during the write stage IIUC as the write side create_blob by Hoodie schema will see a TYPE as null as invalid. i.e. writer side will fail.

Followup task with more details on why this is written this way: #18601

} else {
StructField(f.name(), schemaType.dataType, schemaType.nullable, metadata)
}
}
// For BLOB types, propagate type metadata via SchemaType
val schemaTypeMetadata = if (hoodieSchema.getType == HoodieSchemaType.BLOB) {
val schemaTypeMetadata = if (isBlob) {
Some(new MetadataBuilder()
.putString(HoodieSchema.TYPE_METADATA_FIELD, hoodieSchema.asInstanceOf[HoodieSchema.Blob].toTypeDescriptor)
.build())
Expand Down Expand Up @@ -350,11 +366,17 @@ object HoodieSparkSchemaConverters extends SparkAdapterSupport {
/**
* Validates that a StructType matches the expected blob schema structure defined in {@link HoodieSchema.Blob}.
*
* Purely structural: compares field names and data types recursively, ignoring nullability.
* At the Spark type layer, BLOB is projected as nullable-everywhere by [[toSqlType]] (see the
* comment on the BLOB case there); nullability is therefore not part of the structural
* contract. The RFC-100 non-null invariants are enforced at the physical-schema write
* boundary by {@link HoodieSchema.Blob#createBlob}.
*
* @param structType the StructType to validate
* @throws IllegalArgumentException if the structure does not match the expected blob schema
*/
private def validateBlobStructure(structType: StructType): Unit = {
if (!structType.equals(expectedBlobStructType)) {
if (!matchesStructure(structType, expectedBlobStructType, SQLConf.get.caseSensitiveAnalysis)) {
throw new IllegalArgumentException(
s"""Invalid blob schema structure. Expected schema:
|${expectedBlobStructType.toDDL}
Expand All @@ -363,6 +385,31 @@ object HoodieSparkSchemaConverters extends SparkAdapterSupport {
}
}

private def matchesStructure(source: DataType, expected: DataType, caseSensitive: Boolean): Boolean =
(source, expected) match {
case (s: StructType, e: StructType) =>
s.length == e.length && s.fields.zip(e.fields).forall { case (sf, ef) =>
nameEquals(sf.name, ef.name, caseSensitive) &&
matchesStructure(sf.dataType, ef.dataType, caseSensitive)
}
case _ => source == expected
}

private def nameEquals(a: String, b: String, caseSensitive: Boolean): Boolean =
if (caseSensitive) a == b else a.equalsIgnoreCase(b)

private def withAllFieldsNullable(dataType: DataType): DataType = dataType match {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤖 nit: withAllFieldsNullable implies it only touches struct fields, but the method also sets containsNull = true on arrays and valueContainsNull = true on maps — a future reader scanning the name alone might not expect those side effects. Something like makeFullyNullable or forceNullable would describe the full scope more accurately.

- Generated by an AI agent and may contain mistakes. Please verify any suggestions before applying.

case s: StructType =>
StructType(s.fields.map(f => f.copy(
dataType = withAllFieldsNullable(f.dataType),
nullable = true)))
case ArrayType(elementType, _) =>
ArrayType(withAllFieldsNullable(elementType), containsNull = true)
case MapType(keyType, valueType, _) =>
MapType(keyType, withAllFieldsNullable(valueType), valueContainsNull = true)
case other => other
}

private lazy val expectedVariantStructType: StructType = {
val metadataField = StructField(HoodieSchema.Variant.VARIANT_METADATA_FIELD, BinaryType, nullable = false)
val valueField = StructField(HoodieSchema.Variant.VARIANT_VALUE_FIELD, BinaryType, nullable = false)
Expand All @@ -373,14 +420,25 @@ object HoodieSparkSchemaConverters extends SparkAdapterSupport {
* Validates that a StructType matches the expected unshredded variant schema
* (two non-null {@code BinaryType} fields: {@code metadata} and {@code value}).
*
* Note on nullability: unlike BLOB, VARIANT is not projected nullable-everywhere at the
* Spark type layer because the user-facing path is gated. Spark 3.x rejects VARIANT at
* schema resolution, and Spark 4.0+ exposes it as the native {@code VariantType} populated
* via {@code parse_json(...)}, never a user-supplied {@code named_struct}. The internal
* physical layout ({@code struct<metadata, value>} with non-null fields) only appears
* through {@link HoodieSparkSchemaConverters#toSqlType}, which produces the canonical
* non-null shape this validator expects.
*
* @param structType the StructType to validate
* @throws IllegalArgumentException if the structure does not match the expected variant schema
*/
private def validateVariantStructure(structType: StructType): Unit = {
val fieldsByName = structType.fields.map(f => f.name -> f).toMap
val caseSensitive = SQLConf.get.caseSensitiveAnalysis
val key: String => String =
if (caseSensitive) identity else (_: String).toLowerCase(Locale.ROOT)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤖 nit: key reads as a value (like a map key) rather than a normalizer function, and it's inconsistent with lookupKey used for the same pattern a few dozen lines up in alignSchemaWithCatalog. Could you rename it to lookupKey (or normalizeKey) here for consistency?

- Generated by an AI agent and may contain mistakes. Please verify any suggestions before applying.

val fieldsByName = structType.fields.map(f => key(f.name) -> f).toMap
val ok = structType.length == 2 &&
fieldsByName.get(HoodieSchema.Variant.VARIANT_METADATA_FIELD).exists(f => f.dataType == BinaryType && !f.nullable) &&
fieldsByName.get(HoodieSchema.Variant.VARIANT_VALUE_FIELD).exists(f => f.dataType == BinaryType && !f.nullable)
fieldsByName.get(key(HoodieSchema.Variant.VARIANT_METADATA_FIELD)).exists(f => f.dataType == BinaryType && !f.nullable) &&
fieldsByName.get(key(HoodieSchema.Variant.VARIANT_VALUE_FIELD)).exists(f => f.dataType == BinaryType && !f.nullable)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟠 Major

Reject reordered VARIANT physical structs.

This validator only checks for two named binary fields via a map, so struct<value: binary, metadata: binary> currently passes. The documented canonical layout is struct<metadata: binary, value: binary>, and Spark rows are positional, so accepting reordered fields can corrupt the metadata/value swap on write.

🔧 Suggested fix
   private def validateVariantStructure(structType: StructType): Unit = {
     val caseSensitive = SQLConf.get.caseSensitiveAnalysis
-    val key: String => String =
-      if (caseSensitive) identity else (_: String).toLowerCase(Locale.ROOT)
-    val fieldsByName = structType.fields.map(f => key(f.name) -> f).toMap
-    val ok = structType.length == 2 &&
-      fieldsByName.get(key(HoodieSchema.Variant.VARIANT_METADATA_FIELD)).exists(f => f.dataType == BinaryType && !f.nullable) &&
-      fieldsByName.get(key(HoodieSchema.Variant.VARIANT_VALUE_FIELD)).exists(f => f.dataType == BinaryType && !f.nullable)
+    val expectedFields = expectedVariantStructType.fields
+    val ok = structType.length == expectedFields.length &&
+      structType.fields.zip(expectedFields).forall { case (actual, expected) =>
+        nameEquals(actual.name, expected.name, caseSensitive) &&
+          actual.dataType == expected.dataType &&
+          actual.nullable == expected.nullable
+      }
     if (!ok) {
       throw new IllegalArgumentException(
         s"""Invalid variant schema structure. Expected schema:
🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In
`@hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala`
around lines 434 - 441, The validator validateVariantStructure currently accepts
any two-field binary struct by name lookup, which allows reordered physical
structs like struct<value: binary, metadata: binary>; update
validateVariantStructure to enforce the canonical positional layout by checking
structType.fields length == 2 and then validating that the first field (apply
case-sensitive keying using SQLConf.get.caseSensitiveAnalysis and the key
function) has name equal to HoodieSchema.Variant.VARIANT_METADATA_FIELD and is
BinaryType non-nullable, and the second field has name equal to
HoodieSchema.Variant.VARIANT_VALUE_FIELD and is BinaryType non-nullable (do not
rely on a fieldsByName map for this check).

CodeRabbit (original) (source:comment#3143071733)

if (!ok) {
throw new IllegalArgumentException(
s"""Invalid variant schema structure. Expected schema:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -839,6 +839,143 @@ class TestHoodieSchemaConversionUtils extends FunSuite with Matchers {
assert(parsedVector.getVectorElementType == HoodieSchema.Vector.VectorElementType.FLOAT)
}

test("alignSchemaWithCatalog narrows nullability when alignNullability = true") {
val sourceSchema = new StructType()
.add("id", LongType, nullable = true)
.add("payload", new StructType()
.add("inner", StringType, nullable = true), nullable = true)
.add("items", ArrayType(new StructType()
.add("x", IntegerType, nullable = true), containsNull = true), nullable = true)
.add("lookup", MapType(StringType, new StructType()
.add("y", IntegerType, nullable = true), valueContainsNull = true), nullable = true)

val targetSchema = new StructType()
.add("id", LongType, nullable = false)
.add("payload", new StructType()
.add("inner", StringType, nullable = false), nullable = false)
.add("items", ArrayType(new StructType()
.add("x", IntegerType, nullable = false), containsNull = false), nullable = false)
.add("lookup", MapType(StringType, new StructType()
.add("y", IntegerType, nullable = false), valueContainsNull = false), nullable = false)

val aligned = HoodieSchemaConversionUtils.alignSchemaWithCatalog(
sourceSchema, targetSchema, caseSensitive = false, alignNullability = true)

assert(!aligned("id").nullable)
assert(!aligned("payload").nullable)
assert(!aligned("payload").dataType.asInstanceOf[StructType]("inner").nullable)
val itemsArray = aligned("items").dataType.asInstanceOf[ArrayType]
assert(!aligned("items").nullable && !itemsArray.containsNull)
assert(!itemsArray.elementType.asInstanceOf[StructType]("x").nullable)
val lookupMap = aligned("lookup").dataType.asInstanceOf[MapType]
assert(!aligned("lookup").nullable && !lookupMap.valueContainsNull)
assert(!lookupMap.valueType.asInstanceOf[StructType]("y").nullable)
}

test("alignSchemaWithCatalog preserves source nullability when alignNullability = false") {
val sourceSchema = new StructType()
.add("id", LongType, nullable = true)
.add("payload", new StructType()
.add("inner", StringType, nullable = true), nullable = true)
.add("items", ArrayType(new StructType()
.add("x", IntegerType, nullable = true), containsNull = true), nullable = true)
.add("lookup", MapType(StringType, new StructType()
.add("y", IntegerType, nullable = true), valueContainsNull = true), nullable = true)

val targetSchema = new StructType()
.add("id", LongType, nullable = false)
.add("payload", new StructType()
.add("inner", StringType, nullable = false), nullable = false)
.add("items", ArrayType(new StructType()
.add("x", IntegerType, nullable = false), containsNull = false), nullable = false)
.add("lookup", MapType(StringType, new StructType()
.add("y", IntegerType, nullable = false), valueContainsNull = false), nullable = false)

val aligned = HoodieSchemaConversionUtils.alignSchemaWithCatalog(
sourceSchema, targetSchema, caseSensitive = false, alignNullability = false)

assert(aligned("id").nullable)
assert(aligned("payload").nullable)
assert(aligned("payload").dataType.asInstanceOf[StructType]("inner").nullable)
val itemsArray = aligned("items").dataType.asInstanceOf[ArrayType]
assert(aligned("items").nullable && itemsArray.containsNull)
assert(itemsArray.elementType.asInstanceOf[StructType]("x").nullable)
val lookupMap = aligned("lookup").dataType.asInstanceOf[MapType]
assert(aligned("lookup").nullable && lookupMap.valueContainsNull)
assert(lookupMap.valueType.asInstanceOf[StructType]("y").nullable)
}

test("alignSchemaWithCatalog reattaches custom-type metadata regardless of alignNullability") {
val vectorMetadata = new MetadataBuilder()
.putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(4, FLOAT)")
.build()
val sourceSchema = new StructType()
.add("embedding", ArrayType(FloatType, containsNull = false), nullable = true)
val targetSchema = new StructType()
.add("embedding", ArrayType(FloatType, containsNull = false), nullable = false, vectorMetadata)

Seq(true, false).foreach { flag =>
val aligned = HoodieSchemaConversionUtils.alignSchemaWithCatalog(
sourceSchema, targetSchema, caseSensitive = false, alignNullability = flag)
assert(aligned("embedding").metadata.contains(HoodieSchema.TYPE_METADATA_FIELD),
s"metadata missing when alignNullability=$flag")
assertEquals("VECTOR(4, FLOAT)",
aligned("embedding").metadata.getString(HoodieSchema.TYPE_METADATA_FIELD))
}
}

test("BLOB structure validator accepts mixed-case field names under case-insensitive analysis") {
// Matches RFC-100 BLOB layout (type, data, reference) but with mixed-case field names.
// Under Spark's default case-insensitive analysis, this should validate successfully.
val referenceStruct = new StructType()
.add("External_Path", StringType, nullable = true)
.add("OFFSET", LongType, nullable = true)
.add("Length", LongType, nullable = true)
.add("managed", BooleanType, nullable = true)
val mixedCaseBlob = new StructType()
.add("TYPE", StringType, nullable = true)
.add("Data", BinaryType, nullable = true)
.add("REFERENCE", referenceStruct, nullable = true)
val blobMetadata = new MetadataBuilder()
.putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchema.Blob.TYPE_DESCRIPTOR)
.build()
val outerStruct = new StructType()
.add("id", LongType, nullable = false)
.add("payload", mixedCaseBlob, nullable = true, blobMetadata)

val hoodieSchema = HoodieSchemaConversionUtils.convertStructTypeToHoodieSchema(
outerStruct, "MixedCaseBlobTest", "test")
val payload = hoodieSchema.getField("payload").get().schema().getNonNullType
assertEquals(HoodieSchemaType.BLOB, payload.getType)
}

test("BLOB structure validator rejects wrong field ordering even under case-insensitive analysis") {
// Correct field names but reversed ordering - matchesStructure is positional and must reject.
val referenceStruct = new StructType()
.add("external_path", StringType, nullable = true)
.add("offset", LongType, nullable = true)
.add("length", LongType, nullable = true)
.add("managed", BooleanType, nullable = true)
val wrongOrderBlob = new StructType()
.add("reference", referenceStruct, nullable = true)
.add("data", BinaryType, nullable = true)
.add("type", StringType, nullable = true)
val blobMetadata = new MetadataBuilder()
.putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchema.Blob.TYPE_DESCRIPTOR)
.build()
val outerStruct = new StructType()
.add("id", LongType, nullable = false)
.add("payload", wrongOrderBlob, nullable = true, blobMetadata)

val ex = intercept[Exception] {
HoodieSchemaConversionUtils.convertStructTypeToHoodieSchema(
outerStruct, "WrongOrderBlobTest", "test")
}
assert(ex.getMessage.contains("Invalid blob schema structure") ||
(ex.getCause != null && ex.getCause.getMessage.contains("Invalid blob schema structure")),
s"unexpected exception: ${ex.getMessage}")
}

test("test VECTOR element type mismatch throws error") {
// Metadata says DOUBLE, but Spark array element type is Float
val mismatchMetadata = new MetadataBuilder()
Expand Down
Loading
Loading