Skip to content

Vector Schema incompatibility during write #18537

@voonhous

Description

@voonhous

Bug Description

What happened:

What you expected:

Steps to reproduce:

test("VECTOR column persists as VECTOR in Hudi tableCreateSchema") {
    withTempDir { tmp =>
      val tableName = generateTableName
      spark.sql(
        s"""
           |CREATE TABLE $tableName (
           |  id BIGINT,
           |  embedding VECTOR(3)
           |) USING hudi
           |LOCATION '${tmp.getCanonicalPath}'
           |TBLPROPERTIES (
           |  primaryKey = 'id'
           |)
           """.stripMargin)

      val metaClient = createMetaClient(spark, tmp.getCanonicalPath)
      val persistedOpt = metaClient.getTableConfig.getTableCreateSchema
      assertTrue(persistedOpt.isPresent, "tableCreateSchema should be present")
      val persisted = persistedOpt.get()
      val embedding = persisted.getField("embedding").get().schema().getNonNullType()
      assertEquals(HoodieSchemaType.VECTOR, embedding.getType)
      assertEquals(3, embedding.asInstanceOf[HoodieSchema.Vector].getDimension)

      spark.sql(s"""
        INSERT INTO $tableName VALUES
          (1, array(cast(0.1 as float), cast(0.2 as float), cast(0.3 as float))),
          (2, array(cast(0.4 as float), cast(0.5 as float), cast(0.6 as float)))
      """)
    }
  }
""")

spark.sql("""
  INSERT INTO vector_test VALUES
    (1, 'doc1', array(cast(0.1 as float), cast(0.2 as float), cast(0.3 as float)), '2024-01-01'),
    (2, 'doc2', array(cast(0.4 as float), cast(0.5 as float), cast(0.6 as float)), '2024-01-01')
""")

// Verify table is readable via Spark
spark.sql("select id, name, embedding, dt from vector_test").show(10, false)
println("VECTOR_SQL_TEST_SUCCESS")

Environment

Hudi version:
Query engine: (Spark/Flink/Trino etc)
Spark3.5.3 Scala2.12
Relevant configs:

Logs and Stack Trace

Schema validation backwards compatibility check failed with the following issues: {MISSING_UNION_BRANCH: reader union lacking writer type: VECTOR for field: 'htestcreatetable_1_record.embedding'}
writerSchema: {"type":"record","name":"htestcreatetable_1_record","namespace":"hoodie.htestcreatetable_1","fields":[{"name":"id","type":["null","long"],"default":null},{"name":"embedding","type":["null",{"type":"fixed","name":"vector_float_3","namespace":"","size":12,"logicalType":"vector","dimension":3,"elementType":"FLOAT","storageBacking":"FIXED_BYTES"}],"default":null}]}
tableSchema: {"type":"record","name":"htestcreatetable_1_record","namespace":"hoodie.htestcreatetable_1","fields":[{"name":"id","type":["null","long"],"default":null},{"name":"embedding","type":["null",{"type":"array","items":"float"}],"default":null}]}
org.apache.hudi.exception.SchemaBackwardsCompatibilityException: Schema validation backwards compatibility check failed with the following issues: {MISSING_UNION_BRANCH: reader union lacking writer type: VECTOR for field: 'htestcreatetable_1_record.embedding'}
writerSchema: {"type":"record","name":"htestcreatetable_1_record","namespace":"hoodie.htestcreatetable_1","fields":[{"name":"id","type":["null","long"],"default":null},{"name":"embedding","type":["null",{"type":"fixed","name":"vector_float_3","namespace":"","size":12,"logicalType":"vector","dimension":3,"elementType":"FLOAT","storageBacking":"FIXED_BYTES"}],"default":null}]}
tableSchema: {"type":"record","name":"htestcreatetable_1_record","namespace":"hoodie.htestcreatetable_1","fields":[{"name":"id","type":["null","long"],"default":null},{"name":"embedding","type":["null",{"type":"array","items":"float"}],"default":null}]}
	at org.apache.hudi.common.schema.HoodieSchemaCompatibility.checkValidEvolution(HoodieSchemaCompatibility.java:156)
	at org.apache.hudi.HoodieSchemaUtils$.deduceWriterSchemaWithoutReconcile(HoodieSchemaUtils.scala:180)
	at org.apache.hudi.HoodieSchemaUtils$.deduceWriterSchema(HoodieSchemaUtils.scala:147)
	at org.apache.hudi.HoodieSparkSqlWriterInternal.writeInternal(HoodieSparkSqlWriter.scala:469)
	at org.apache.hudi.HoodieSparkSqlWriterInternal.$anonfun$write$1(HoodieSparkSqlWriter.scala:187)
	at org.apache.hudi.HoodieSparkSqlWriterInternal.write(HoodieSparkSqlWriter.scala:205)
	at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:127)
	at org.apache.spark.sql.hudi.command.InsertIntoHoodieTableCommand$.run(InsertIntoHoodieTableCommand.scala:118)
	at org.apache.spark.sql.hudi.command.InsertIntoHoodieTableCommand.run(InsertIntoHoodieTableCommand.scala:73)
	at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:113)
	at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:111)
	at org.apache.spark.sql.execution.command.DataWritingCommandExec.executeCollect(commands.scala:125)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:107)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:107)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
	at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
	at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437)
	at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:98)
	at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:85)
	at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:83)
	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:220)
	at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
	at org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:691)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:682)
	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:713)
	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:744)
	at org.apache.spark.sql.hudi.ddl.TestCreateTable.$anonfun$new$130(TestCreateTable.scala:2315)
	at org.apache.spark.sql.hudi.ddl.TestCreateTable.$anonfun$new$130$adapted(TestCreateTable.scala:2292)
	at org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase.withTempDir(HoodieSparkSqlTestBase.scala:102)
	at org.apache.spark.sql.hudi.ddl.TestCreateTable.$anonfun$new$129(TestCreateTable.scala:2292)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase.$anonfun$test$1(HoodieSparkSqlTestBase.scala:114)
	at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:189)
	at org.scalatest.TestSuite.withFixture(TestSuite.scala:196)
	at org.scalatest.TestSuite.withFixture$(TestSuite.scala:195)
	at org.scalatest.funsuite.AnyFunSuite.withFixture(AnyFunSuite.scala:1562)
	at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:187)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:199)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:199)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:181)
	at org.scalatest.funsuite.AnyFunSuite.runTest(AnyFunSuite.scala:1562)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:232)
	at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
	at scala.collection.immutable.List.foreach(List.scala:431)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:232)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:231)
	at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1562)
	at org.scalatest.Suite.run(Suite.scala:1112)
	at org.scalatest.Suite.run$(Suite.scala:1094)
	at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1562)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:236)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
	at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:236)
	at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:235)
	at org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase.org$scalatest$BeforeAndAfterAll$$super$run(HoodieSparkSqlTestBase.scala:58)
	at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
	at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
	at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
	at org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase.run(HoodieSparkSqlTestBase.scala:58)
	at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1314)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1308)
	at scala.collection.immutable.List.foreach(List.scala:431)
	at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1308)
	at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:993)
	at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:971)
	at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1474)
	at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:971)
	at org.scalatest.tools.Runner$.run(Runner.scala:798)
	at org.scalatest.tools.Runner.run(Runner.scala)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2or3(ScalaTestRunner.java:43)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:26)

Metadata

Metadata

Assignees

Labels

type:bugBug reports and fixes

Type

No type
No fields configured for issues without a type.

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions