From 2ceee889f4554fdcce9458346fbaf3f47d73c3b9 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Mon, 13 Apr 2026 09:03:01 -0700 Subject: [PATCH 01/15] feat(lance): write Hudi VECTOR columns as native Lance fixed-size lists MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Translate the Hudi VECTOR logical-type metadata (`hudi_type = "VECTOR(dim[,elem])"`) into the lance-spark metadata key `arrow.fixed-size-list.size` before calling `LanceArrowUtils.toArrowSchema`, so the Lance writer emits a native Arrow FixedSizeList (Lance's vector column encoding) instead of a plain variable-length list. No change needed at `LanceFileWriter.open(...)`; the encoding is driven by the Arrow schema itself. - New private helper `enrichSparkSchemaForLanceVectors` in `HoodieSparkLanceWriter` reuses `VectorConversionUtils.detectVectorColumnsFromMetadata` to find VECTOR fields and attaches the Lance metadata key; non-vector fields pass through unchanged. - Fails fast with `HoodieNotSupportedException` for non-ArrayType or non- Float/Double element types (matches lance-spark's `shouldBeFixedSizeList`). - Tests in `TestLanceDataSource` (COW + MOR): - `testFloatVectorRoundTrip` - `testDoubleVectorRoundTrip` - `testMultipleVectorColumns` Each opens the written `.lance` file via `LanceFileReader` and asserts the field is `ArrowType.FixedSizeList` with the expected `listSize` — the direct regression guard that fails pre-fix and passes post-fix. Co-Authored-By: Claude Opus 4.6 --- .../io/storage/HoodieSparkLanceWriter.java | 65 ++++++- .../hudi/functional/TestLanceDataSource.scala | 176 +++++++++++++++++- 2 files changed, 238 insertions(+), 3 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java index 02f9c7e3b16b8..a9fad32a61fd8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java @@ -23,7 +23,9 @@ import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.io.lance.HoodieBaseLanceWriter; import org.apache.hudi.io.storage.row.HoodieBloomFilterRowWriteSupport; import org.apache.hudi.io.storage.row.HoodieInternalRowFileWriter; @@ -36,11 +38,19 @@ import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.types.pojo.Schema; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DoubleType$; +import org.apache.spark.sql.types.FloatType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.MetadataBuilder; +import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.LanceArrowUtils; import org.apache.spark.unsafe.types.UTF8String; import java.io.IOException; +import java.util.Map; import java.util.function.Function; import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD; @@ -120,8 +130,11 @@ private HoodieSparkLanceWriter(StoragePath file, Option bloomFilterOpt, long maxFileSize) { super(file, DEFAULT_BATCH_SIZE, bloomFilterOpt.map(HoodieBloomFilterRowWriteSupport::new)); - this.sparkSchema = sparkSchema; - this.arrowSchema = LanceArrowUtils.toArrowSchema(sparkSchema, DEFAULT_TIMEZONE, true, false); + // Enrich fields carrying the Hudi VECTOR logical type with the Spark metadata key + // that lance-spark's toArrowSchema / LanceArrowWriter use to emit a native Arrow + // FixedSizeList (i.e. Lance's vector column encoding) instead of a plain List. + this.sparkSchema = enrichSparkSchemaForLanceVectors(sparkSchema); + this.arrowSchema = LanceArrowUtils.toArrowSchema(this.sparkSchema, DEFAULT_TIMEZONE, true, false); this.fileName = UTF8String.fromString(file.getName()); this.instantTime = UTF8String.fromString(instantTime); this.populateMetaFields = populateMetaFields; @@ -132,6 +145,54 @@ private HoodieSparkLanceWriter(StoragePath file, }; } + /** + * For every field carrying a Hudi VECTOR logical type annotation + * (Spark metadata key {@link HoodieSchema#TYPE_METADATA_FIELD} starting with {@code "VECTOR"}), + * attach the lance-spark metadata key {@link LanceArrowUtils#ARROW_FIXED_SIZE_LIST_SIZE_KEY()} + * with the vector's dimension so that {@link LanceArrowUtils#toArrowSchema} emits a native + * Arrow {@code FixedSizeList} (Lance's vector column encoding) and + * {@link LanceArrowWriter} selects its fixed-size-list field writer when serializing values. + * + *

Currently only FLOAT and DOUBLE element vectors are supported on Lance, matching + * lance-spark's {@code VectorUtils.shouldBeFixedSizeList}. Other element types would + * silently fall through to a plain list write, so we fail fast instead. + */ + private static StructType enrichSparkSchemaForLanceVectors(StructType sparkSchema) { + Map vectorColumns = + VectorConversionUtils.detectVectorColumnsFromMetadata(sparkSchema); + if (vectorColumns.isEmpty()) { + return sparkSchema; + } + StructField[] fields = sparkSchema.fields(); + StructField[] newFields = new StructField[fields.length]; + for (int i = 0; i < fields.length; i++) { + StructField field = fields[i]; + HoodieSchema.Vector vec = vectorColumns.get(i); + if (vec == null) { + newFields[i] = field; + continue; + } + DataType dt = field.dataType(); + if (!(dt instanceof ArrayType)) { + throw new HoodieNotSupportedException( + "Hudi VECTOR column '" + field.name() + "' must be Spark ArrayType to be written to Lance; got " + dt); + } + DataType elemType = ((ArrayType) dt).elementType(); + boolean elemSupported = elemType instanceof FloatType$ || elemType instanceof DoubleType$; + if (!elemSupported) { + throw new HoodieNotSupportedException( + "Lance base-file format currently supports FLOAT/DOUBLE VECTOR columns only; " + + "got element type " + elemType.simpleString() + " for field '" + field.name() + "'"); + } + Metadata enriched = new MetadataBuilder() + .withMetadata(field.metadata()) + .putLong(LanceArrowUtils.ARROW_FIXED_SIZE_LIST_SIZE_KEY(), vec.getDimension()) + .build(); + newFields[i] = new StructField(field.name(), field.dataType(), field.nullable(), enriched); + } + return new StructType(newFields); + } + @Override public void writeRowWithMetadata(HoodieKey key, InternalRow row) throws IOException { UTF8String recordKey = UTF8String.fromString(key.getRecordKey()); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index 662a6885e3e7b..6b23cceb57dd7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -22,6 +22,7 @@ import org.apache.hudi.DefaultSparkRecordMerger import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMetadataConfig} import org.apache.hudi.common.engine.HoodieLocalEngineContext import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.schema.HoodieSchema import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.view.{FileSystemViewManager, FileSystemViewStorageConfig} import org.apache.hudi.common.testutils.HoodieTestUtils @@ -30,12 +31,16 @@ import org.apache.hudi.io.storage.HoodieSparkLanceReader import org.apache.hudi.storage.StoragePath import org.apache.hudi.testutils.HoodieSparkClientTestBase -import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} +import org.apache.arrow.memory.RootAllocator +import org.apache.arrow.vector.types.pojo.ArrowType +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession} +import org.apache.spark.sql.types._ import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotNull, assertTrue} import org.junit.jupiter.api.condition.DisabledIfSystemProperty import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.EnumSource +import org.lance.file.LanceFileReader import java.util.stream.Collectors @@ -784,6 +789,175 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { fsView.close() } + @ParameterizedTest + @EnumSource(value = classOf[HoodieTableType]) + def testFloatVectorRoundTrip(tableType: HoodieTableType): Unit = { + val tableName = s"test_lance_vec_f32_${tableType.name().toLowerCase}" + val tablePath = s"$basePath/$tableName" + + val dim = 4 + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("embedding", + ArrayType(FloatType, containsNull = false), + nullable = false, + vectorMetadata(s"VECTOR($dim)")) + )) + val data = Seq( + Row(1, Array(1.0f, 2.0f, 4.0f, 8.0f)), + Row(2, Array(0.5f, 0.25f, 0.125f, 0.0625f)), + Row(3, Array(-1.0f, -2.0f, -4.0f, -8.0f)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) + + writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite) + + val readDf = spark.read.format("hudi").load(tablePath).select("id", "embedding") + assertEquals( + ArrayType(FloatType, containsNull = false), + readDf.schema("embedding").dataType) + + val actualByKey = readDf.collect().map(r => + r.getInt(0) -> r.getAs[Seq[Float]](1).toSeq).toMap + val expectedByKey = data.map(r => + r.getInt(0) -> r.getAs[Array[Float]](1).toSeq).toMap + assertEquals(expectedByKey, actualByKey) + + assertLanceFieldIsFixedSizeList(tablePath, "embedding", dim) + } + + @ParameterizedTest + @EnumSource(value = classOf[HoodieTableType]) + def testDoubleVectorRoundTrip(tableType: HoodieTableType): Unit = { + val tableName = s"test_lance_vec_f64_${tableType.name().toLowerCase}" + val tablePath = s"$basePath/$tableName" + + val dim = 4 + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("embedding", + ArrayType(DoubleType, containsNull = false), + nullable = false, + vectorMetadata(s"VECTOR($dim, DOUBLE)")) + )) + val data = Seq( + Row(1, Array(1.0d, 2.0d, 4.0d, 8.0d)), + Row(2, Array(0.5d, 0.25d, 0.125d, 0.0625d)), + Row(3, Array(-1.0d, -2.0d, -4.0d, -8.0d)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) + + writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite) + + val readDf = spark.read.format("hudi").load(tablePath).select("id", "embedding") + assertEquals( + ArrayType(DoubleType, containsNull = false), + readDf.schema("embedding").dataType) + + val actualByKey = readDf.collect().map(r => + r.getInt(0) -> r.getAs[Seq[Double]](1).toSeq).toMap + val expectedByKey = data.map(r => + r.getInt(0) -> r.getAs[Array[Double]](1).toSeq).toMap + assertEquals(expectedByKey, actualByKey) + + assertLanceFieldIsFixedSizeList(tablePath, "embedding", dim) + } + + @ParameterizedTest + @EnumSource(value = classOf[HoodieTableType]) + def testMultipleVectorColumns(tableType: HoodieTableType): Unit = { + val tableName = s"test_lance_vec_multi_${tableType.name().toLowerCase}" + val tablePath = s"$basePath/$tableName" + + val embeddingDim = 3 + val featuresDim = 2 + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("embedding", + ArrayType(FloatType, containsNull = false), + nullable = false, + vectorMetadata(s"VECTOR($embeddingDim)")), + StructField("features", + ArrayType(DoubleType, containsNull = false), + nullable = false, + vectorMetadata(s"VECTOR($featuresDim, DOUBLE)")) + )) + val data = Seq( + Row(1, Array(1.0f, 2.0f, 3.0f), Array(10.0d, 20.0d)), + Row(2, Array(4.0f, 5.0f, 6.0f), Array(30.0d, 40.0d)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) + + writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite) + + val readDf = spark.read.format("hudi").load(tablePath).select("id", "embedding", "features") + val rows = readDf.collect().map { r => + (r.getInt(0), r.getAs[Seq[Float]](1).toSeq, r.getAs[Seq[Double]](2).toSeq) + }.toSet + val expected = data.map { r => + (r.getInt(0), + r.getAs[Array[Float]](1).toSeq, + r.getAs[Array[Double]](2).toSeq) + }.toSet + assertEquals(expected, rows) + + assertLanceFieldIsFixedSizeList(tablePath, "embedding", embeddingDim) + assertLanceFieldIsFixedSizeList(tablePath, "features", featuresDim) + } + + private def vectorMetadata(descriptor: String): Metadata = + new MetadataBuilder().putString(HoodieSchema.TYPE_METADATA_FIELD, descriptor).build() + + /** + * Opens the raw Lance base file(s) for the given Hudi table and asserts that the + * named field is encoded as an Arrow FixedSizeList with the expected listSize. + * This proves that the write path used Lance's native vector column encoding + * rather than a plain variable-length list. + */ + private def assertLanceFieldIsFixedSizeList(tablePath: String, fieldName: String, expectedDim: Int): Unit = { + val metaClient = HoodieTableMetaClient.builder() + .setConf(HoodieTestUtils.getDefaultStorageConf) + .setBasePath(tablePath) + .build() + val engineContext = new HoodieLocalEngineContext(metaClient.getStorageConf) + val metadataConfig = HoodieMetadataConfig.newBuilder.build + val viewManager = FileSystemViewManager.createViewManager( + engineContext, metadataConfig, FileSystemViewStorageConfig.newBuilder.build, + HoodieCommonConfig.newBuilder.build, + (mc: HoodieTableMetaClient) => metaClient.getTableFormat + .getMetadataFactory.create(engineContext, mc.getStorage, metadataConfig, tablePath)) + val fsView = viewManager.getFileSystemView(metaClient) + try { + val baseFiles = fsView.getLatestBaseFiles("") + .collect(Collectors.toList[org.apache.hudi.common.model.HoodieBaseFile]) + assertTrue(baseFiles.size() > 0, "Expected at least one Lance base file") + val allocator = new RootAllocator() + try { + baseFiles.asScala.foreach { bf => + val reader = LanceFileReader.open(bf.getPath, allocator) + try { + val field = reader.schema().findField(fieldName) + assertNotNull(field, s"Field $fieldName not found in Lance schema for ${bf.getPath}") + field.getType match { + case fsl: ArrowType.FixedSizeList => + assertEquals(expectedDim, fsl.getListSize, + s"Lance field $fieldName in ${bf.getPath} should be FixedSizeList of size $expectedDim") + case other => + throw new AssertionError( + s"Lance field $fieldName in ${bf.getPath} should be FixedSizeList but was $other") + } + } finally { + reader.close() + } + } + } finally { + allocator.close() + } + } finally { + fsView.close() + } + } + private def createDataFrame(records: Seq[(Int, String, Int, Double)]) = { spark.createDataFrame(records).toDF("id", "name", "age", "score").coalesce(1) } From ee93e5a00b6c2ee4da9cb55697cb2aab496589c4 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Mon, 13 Apr 2026 09:35:19 -0700 Subject: [PATCH 02/15] feat(lance): restore VECTOR metadata on Lance read path MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Companion to the Lance writer's native FixedSizeList encoding: on read, rehydrate the Hudi `hudi_type = VECTOR(...)` Spark metadata that `LanceArrowUtils.fromArrowSchema` drops, so the read schema matches the Parquet path. Gate the Parquet-only ArrayType→BinaryType vector rewrite in HoodieFileGroupReaderBasedFileFormat on format == PARQUET; Lance returns vectors natively as ArrayType so the rewrite would trigger a spurious cast and break the read. - VectorConversionUtils.restoreVectorMetadataFromArrowSchema walks the Arrow schema and re-attaches VECTOR(dim[,DOUBLE]) for FixedSizeList fields. - HoodieSparkLanceReader.getSchema and SparkLanceReaderBase.read now call it so downstream VECTOR-aware code sees the same schema as on Parquet. - TestLanceDataSource: assert hudi_type metadata is restored on read for float, double, and multi-vector round-trips. Co-Authored-By: Claude Opus 4.6 --- .../io/storage/HoodieSparkLanceReader.java | 3 +- .../io/storage/VectorConversionUtils.java | 86 +++++++++++++++++++ .../lance/SparkLanceReaderBase.scala | 10 ++- ...HoodieFileGroupReaderBasedFileFormat.scala | 16 +++- .../hudi/functional/TestLanceDataSource.scala | 11 +++ 5 files changed, 120 insertions(+), 6 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java index 3bf6625a4fd57..2c0822233452f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java @@ -202,7 +202,8 @@ private ClosableIterator getUnsafeRowIterator(HoodieSchema requestedS @Override public HoodieSchema getSchema() { try { - StructType structType = LanceArrowUtils.fromArrowSchema(arrowSchema); + StructType structType = VectorConversionUtils.restoreVectorMetadataFromArrowSchema( + arrowSchema, LanceArrowUtils.fromArrowSchema(arrowSchema)); return HoodieSchemaConversionUtils.convertStructTypeToHoodieSchema(structType, "record", "", false); } catch (Exception e) { throw new HoodieException("Failed to read schema from Lance file: " + path, e); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index 2bf8e86b5d0fd..906e73f8d1fc8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -22,10 +22,14 @@ import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.BinaryType$; +import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; @@ -235,4 +239,86 @@ public static void convertRowVectorColumns(InternalRow row, GenericInternalRow r } } } + + /** + * Restores Hudi's VECTOR logical-type metadata on a Spark schema derived from a Lance + * (Arrow) schema. The upstream conversion {@code LanceArrowUtils.fromArrowSchema} maps + * Arrow {@code FixedSizeList} to Spark {@code ArrayType(FloatType|DoubleType)} + * but drops all field-level metadata, so a column that was written as a Hudi VECTOR + * comes back indistinguishable from a plain array. This method walks the original Arrow + * schema (which still carries the fixed-size-list intent and dimension) and re-attaches + * {@link HoodieSchema#TYPE_METADATA_FIELD} = {@code "VECTOR(dim[, DOUBLE])"} to matching + * fields so downstream VECTOR-aware code sees the same schema it would on the Parquet path. + * + *

Only FLOAT and DOUBLE element types are recognized, matching what the Lance write + * path can produce today. Any other Arrow type (including non-FixedSizeList or + * FixedSizeList of non-floating-point children) is passed through unchanged. + * + * @param arrowSchema the original Arrow schema read from the Lance file + * @param convertedSpark the StructType produced by {@code LanceArrowUtils.fromArrowSchema} + * (must have the same field count and order as {@code arrowSchema}) + * @return a new StructType with VECTOR metadata restored on eligible fields + */ + public static StructType restoreVectorMetadataFromArrowSchema( + org.apache.arrow.vector.types.pojo.Schema arrowSchema, StructType convertedSpark) { + if (arrowSchema == null || convertedSpark == null) { + return convertedSpark; + } + List arrowFields = arrowSchema.getFields(); + StructField[] sparkFields = convertedSpark.fields(); + if (arrowFields.size() != sparkFields.length) { + // Schemas desynchronized — bail out rather than risk corrupting metadata. + return convertedSpark; + } + StructField[] newFields = new StructField[sparkFields.length]; + boolean anyChanged = false; + for (int i = 0; i < sparkFields.length; i++) { + StructField sf = sparkFields[i]; + String descriptor = deriveVectorDescriptor(arrowFields.get(i)); + if (descriptor == null) { + newFields[i] = sf; + } else { + newFields[i] = new StructField( + sf.name(), + sf.dataType(), + sf.nullable(), + new MetadataBuilder() + .withMetadata(sf.metadata()) + .putString(HoodieSchema.TYPE_METADATA_FIELD, descriptor) + .build()); + anyChanged = true; + } + } + return anyChanged ? new StructType(newFields) : convertedSpark; + } + + /** + * Derives Hudi's VECTOR type descriptor for an Arrow field if it represents a + * fixed-size list of single- or double-precision floats, otherwise returns null. + */ + private static String deriveVectorDescriptor(Field arrowField) { + ArrowType type = arrowField.getType(); + if (!(type instanceof ArrowType.FixedSizeList)) { + return null; + } + int dim = ((ArrowType.FixedSizeList) type).getListSize(); + List children = arrowField.getChildren(); + if (children.size() != 1) { + return null; + } + ArrowType childType = children.get(0).getType(); + if (!(childType instanceof ArrowType.FloatingPoint)) { + return null; + } + FloatingPointPrecision precision = ((ArrowType.FloatingPoint) childType).getPrecision(); + HoodieSchema.Vector.VectorElementType elementType; + if (precision == FloatingPointPrecision.SINGLE) { + elementType = HoodieSchema.Vector.VectorElementType.FLOAT; + } else if (precision == FloatingPointPrecision.DOUBLE) { + elementType = HoodieSchema.Vector.VectorElementType.DOUBLE; + } else { + return null; + } + return ((HoodieSchema.Vector) HoodieSchema.createVector(dim, elementType)).toTypeDescriptor(); + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala index bc24051a633f3..27d2b8d89b5d4 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala @@ -23,7 +23,7 @@ import org.apache.hudi.SparkAdapterSupport.sparkAdapter import org.apache.hudi.common.util import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.io.memory.HoodieArrowAllocator -import org.apache.hudi.io.storage.{HoodieSparkLanceReader, LanceRecordIterator} +import org.apache.hudi.io.storage.{HoodieSparkLanceReader, LanceRecordIterator, VectorConversionUtils} import org.apache.hudi.storage.StorageConfiguration import org.apache.hadoop.conf.Configuration @@ -90,9 +90,13 @@ class SparkLanceReaderBase(enableVectorizedReader: Boolean) extends SparkColumna // Open Lance file reader val lanceReader = LanceFileReader.open(filePath, allocator) - // Get schema from Lance file + // Get schema from Lance file. lance-spark drops all Arrow field metadata during + // StructType conversion, so re-attach Hudi's VECTOR logical-type descriptor onto + // fields that are encoded as Arrow FixedSizeList. This mirrors + // the Parquet read path, where VECTOR metadata is restored from footer entries. val arrowSchema = lanceReader.schema() - val fileSchema = LanceArrowUtils.fromArrowSchema(arrowSchema) + val fileSchema = VectorConversionUtils.restoreVectorMetadataFromArrowSchema( + arrowSchema, LanceArrowUtils.fromArrowSchema(arrowSchema)) // Build type change info for schema evolution val (implicitTypeChangeInfo, sparkRequestSchema) = diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala index 9ba25424a9ca8..afcee5820fc4c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala @@ -431,11 +431,23 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, /** * Detects vector columns and replaces them with BinaryType in one step. + * + *

The BinaryType rewrite is Parquet-specific: Hudi stores VECTOR columns as + * FIXED_LEN_BYTE_ARRAY in Parquet, so the reader must see BinaryType and the raw + * bytes are post-converted back to ArrayType. Other formats (e.g. Lance) encode + * vectors natively as Arrow FixedSizeList and return ArrayType directly, so the + * rewrite would introduce a spurious ArrayType→BinaryType cast during schema + * evolution and break the read. Skip the rewrite for those formats. + * * @return (modified schema with BinaryType for vectors, vector column ordinal map) */ private def withVectorRewrite(schema: StructType): (StructType, Map[Int, HoodieSchema.Vector]) = { - val vecs = detectVectorColumns(schema) - if (vecs.nonEmpty) (replaceVectorFieldsWithBinary(schema, vecs), vecs) else (schema, vecs) + if (hoodieFileFormat != HoodieFileFormat.PARQUET) { + (schema, Map.empty[Int, HoodieSchema.Vector]) + } else { + val vecs = detectVectorColumns(schema) + if (vecs.nonEmpty) (replaceVectorFieldsWithBinary(schema, vecs), vecs) else (schema, vecs) + } } /** diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index 6b23cceb57dd7..a6953c6c553cb 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -816,6 +816,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertEquals( ArrayType(FloatType, containsNull = false), readDf.schema("embedding").dataType) + assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim)") val actualByKey = readDf.collect().map(r => r.getInt(0) -> r.getAs[Seq[Float]](1).toSeq).toMap @@ -853,6 +854,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertEquals( ArrayType(DoubleType, containsNull = false), readDf.schema("embedding").dataType) + assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim, DOUBLE)") val actualByKey = readDf.collect().map(r => r.getInt(0) -> r.getAs[Seq[Double]](1).toSeq).toMap @@ -900,11 +902,20 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { r.getAs[Array[Double]](2).toSeq) }.toSet assertEquals(expected, rows) + assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($embeddingDim)") + assertHudiTypeMetadata(readDf.schema("features"), s"VECTOR($featuresDim, DOUBLE)") assertLanceFieldIsFixedSizeList(tablePath, "embedding", embeddingDim) assertLanceFieldIsFixedSizeList(tablePath, "features", featuresDim) } + private def assertHudiTypeMetadata(field: StructField, expectedDescriptor: String): Unit = { + assertTrue(field.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD), + s"Expected field ${field.name} to carry ${HoodieSchema.TYPE_METADATA_FIELD} metadata after read") + assertEquals(expectedDescriptor, field.metadata.getString(HoodieSchema.TYPE_METADATA_FIELD), + s"Expected ${HoodieSchema.TYPE_METADATA_FIELD}=$expectedDescriptor on field ${field.name}") + } + private def vectorMetadata(descriptor: String): Metadata = new MetadataBuilder().putString(HoodieSchema.TYPE_METADATA_FIELD, descriptor).build() From 099aadfc10130041b9a322d8cd139a761369b43e Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Mon, 13 Apr 2026 10:36:23 -0700 Subject: [PATCH 03/15] feat(lance): write hoodie.vector.columns footer as forward-compat guard Mirrors the Parquet writer: emit the comma-separated `colName:VECTOR(dim[,elemType])` descriptor list under the existing `hoodie.vector.columns` key in the Lance file-footer key-value metadata. Reader still derives VECTOR identity from the Arrow FixedSizeList type today; this footer entry is insurance for future descriptor fields the Arrow type cannot express (quantization tags, distance metrics, etc.) and keeps Lance files symmetric with Parquet files. - HoodieBaseLanceWriter: new protected `additionalSchemaMetadata()` hook invoked during close(), so subclasses can contribute footer KV entries alongside bloom-filter metadata. - HoodieSparkLanceWriter: override `additionalSchemaMetadata()` to emit `hoodie.vector.columns` when the Spark schema has any VECTOR column. - VectorConversionUtils: add `buildVectorColumnsMetadataValue(StructType)` matching the Parquet-path helper's output format. - TestLanceDataSource: assert footer carries the expected descriptor list for float, double, and multi-vector round-trips. Co-Authored-By: Claude Opus 4.6 --- .../io/storage/HoodieSparkLanceWriter.java | 22 ++++++++ .../io/storage/VectorConversionUtils.java | 33 +++++++++++ .../hudi/io/lance/HoodieBaseLanceWriter.java | 23 ++++++++ .../hudi/functional/TestLanceDataSource.scala | 55 +++++++++++++++++++ 4 files changed, 133 insertions(+) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java index a9fad32a61fd8..5ee9898fc503e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java @@ -259,6 +259,28 @@ protected Schema getArrowSchema() { return arrowSchema; } + /** + * Emit Hudi's {@code hoodie.vector.columns} footer entry alongside any + * bloom-filter metadata. Mirrors the Parquet writer (see + * {@code HoodieRowParquetWriteSupport#init}) so Lance files carry the same + * self-describing VECTOR descriptor list that Parquet files do. + * + *

The read side today derives VECTOR identity from the Arrow + * {@code FixedSizeList} type — this footer entry is a + * forward-compat guard: it lets future readers recover the exact descriptor + * (including fields the Arrow type cannot express, e.g. quantization tags) + * without a writer bump. + */ + @Override + protected java.util.Map additionalSchemaMetadata() { + String value = VectorConversionUtils.buildVectorColumnsMetadataValue(sparkSchema); + if (value.isEmpty()) { + return java.util.Collections.emptyMap(); + } + return java.util.Collections.singletonMap( + org.apache.hudi.common.schema.HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY, value); + } + /** * Update Hudi metadata fields in the InternalRow. * diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index 906e73f8d1fc8..85e2a3f96d4d0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -79,6 +79,39 @@ public static Map detectVectorColumns(HoodieSchema return vectorColumnInfo; } + /** + * Builds the value string for {@link HoodieSchema#PARQUET_VECTOR_COLUMNS_METADATA_KEY} + * from a Spark {@link StructType}, matching the comma-separated + * {@code colName:VECTOR(dim[,elemType])} format produced on the Parquet path by + * {@link HoodieSchema#buildVectorColumnsMetadataValue}. + * + *

Intended to be written as file-footer metadata (e.g. via + * {@code LanceFileWriter.addSchemaMetadata}) so any reader can identify VECTOR + * columns from the file alone, mirroring the Parquet footer convention. + * + * @param schema Spark StructType (may be null) + * @return comma-separated descriptor list, or empty string if no VECTOR columns + */ + public static String buildVectorColumnsMetadataValue(StructType schema) { + if (schema == null) { + return ""; + } + StringBuilder sb = new StringBuilder(); + StructField[] fields = schema.fields(); + Map detected = detectVectorColumnsFromMetadata(schema); + for (int i = 0; i < fields.length; i++) { + HoodieSchema.Vector v = detected.get(i); + if (v == null) { + continue; + } + if (sb.length() > 0) { + sb.append(','); + } + sb.append(fields[i].name()).append(':').append(v.toTypeDescriptor()); + } + return sb.toString(); + } + /** * Detects VECTOR columns from Spark StructType metadata annotations. * Fields with metadata key {@link HoodieSchema#TYPE_METADATA_FIELD} starting with "VECTOR" diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java index 8f76c61fa7249..7835880c7406f 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java @@ -105,6 +105,20 @@ protected HoodieBaseLanceWriter(StoragePath path, int batchSize, */ protected abstract Schema getArrowSchema(); + /** + * Subclass hook for emitting additional Lance file-footer key-value metadata + * alongside any bloom-filter entries. Called once during {@link #close()}. + * + *

Default implementation returns an empty map. Overriders should return a + * fresh map; the caller does not retain a reference. Colliding keys are + * overwritten per {@code LanceFileWriter.addSchemaMetadata} semantics. + * + * @return map of footer metadata key-value pairs, or empty map for none + */ + protected Map additionalSchemaMetadata() { + return java.util.Collections.emptyMap(); + } + /** * Write a single record. Records are buffered and flushed in batches. * @@ -170,6 +184,15 @@ public void close() throws IOException { writer.addSchemaMetadata(metadata); } } + + // Allow subclasses to contribute additional footer key-value metadata + // (e.g. Spark writer emits `hoodie.vector.columns` for forward-compat read). + if (writer != null) { + Map extra = additionalSchemaMetadata(); + if (extra != null && !extra.isEmpty()) { + writer.addSchemaMetadata(extra); + } + } } catch (Exception e) { primaryException = e; } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index a6953c6c553cb..2a983ccc10a54 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -825,6 +825,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertEquals(expectedByKey, actualByKey) assertLanceFieldIsFixedSizeList(tablePath, "embedding", dim) + assertLanceFooterHasVectorColumns(tablePath, s"embedding:VECTOR($dim)") } @ParameterizedTest @@ -863,6 +864,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertEquals(expectedByKey, actualByKey) assertLanceFieldIsFixedSizeList(tablePath, "embedding", dim) + assertLanceFooterHasVectorColumns(tablePath, s"embedding:VECTOR($dim, DOUBLE)") } @ParameterizedTest @@ -907,6 +909,8 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertLanceFieldIsFixedSizeList(tablePath, "embedding", embeddingDim) assertLanceFieldIsFixedSizeList(tablePath, "features", featuresDim) + assertLanceFooterHasVectorColumns(tablePath, + s"embedding:VECTOR($embeddingDim),features:VECTOR($featuresDim, DOUBLE)") } private def assertHudiTypeMetadata(field: StructField, expectedDescriptor: String): Unit = { @@ -925,6 +929,57 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { * This proves that the write path used Lance's native vector column encoding * rather than a plain variable-length list. */ + /** + * Opens the raw Lance base file(s) for the given Hudi table and asserts that the + * Arrow schema's custom metadata carries `hoodie.vector.columns` with the given + * expected descriptor list. This is the forward-compat guard — future readers + * can recover the exact Hudi VECTOR descriptor from the footer without + * re-deriving from the Arrow type. + * + *

Descriptor list order is writer-determined (matches Spark schema field + * order), so callers must pass the entries in that order. + */ + private def assertLanceFooterHasVectorColumns(tablePath: String, expected: String): Unit = { + val metaClient = HoodieTableMetaClient.builder() + .setConf(HoodieTestUtils.getDefaultStorageConf) + .setBasePath(tablePath) + .build() + val engineContext = new HoodieLocalEngineContext(metaClient.getStorageConf) + val metadataConfig = HoodieMetadataConfig.newBuilder.build + val viewManager = FileSystemViewManager.createViewManager( + engineContext, metadataConfig, FileSystemViewStorageConfig.newBuilder.build, + HoodieCommonConfig.newBuilder.build, + (mc: HoodieTableMetaClient) => metaClient.getTableFormat + .getMetadataFactory.create(engineContext, mc.getStorage, metadataConfig, tablePath)) + val fsView = viewManager.getFileSystemView(metaClient) + try { + val baseFiles = fsView.getLatestBaseFiles("") + .collect(Collectors.toList[org.apache.hudi.common.model.HoodieBaseFile]) + assertTrue(baseFiles.size() > 0, "Expected at least one Lance base file") + val allocator = new RootAllocator() + try { + baseFiles.asScala.foreach { bf => + val reader = LanceFileReader.open(bf.getPath, allocator) + try { + val meta = reader.schema().getCustomMetadata + assertNotNull(meta, s"Lance footer metadata null for ${bf.getPath}") + val key = HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY + assertTrue(meta.containsKey(key), + s"Lance file ${bf.getPath} should have footer key $key, got keys ${meta.keySet()}") + assertEquals(expected, meta.get(key), + s"Lance file ${bf.getPath} footer $key mismatch") + } finally { + reader.close() + } + } + } finally { + allocator.close() + } + } finally { + fsView.close() + } + } + private def assertLanceFieldIsFixedSizeList(tablePath: String, fieldName: String, expectedDim: Int): Unit = { val metaClient = HoodieTableMetaClient.builder() .setConf(HoodieTestUtils.getDefaultStorageConf) From ff5beb354fba06411cbf8da170186b76bc000a20 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Tue, 14 Apr 2026 15:45:54 -0700 Subject: [PATCH 04/15] address comments --- .../io/storage/HoodieSparkLanceWriter.java | 7 +- .../io/storage/VectorConversionUtils.java | 8 +- .../hudi/io/lance/HoodieBaseLanceWriter.java | 19 +- .../hudi/functional/TestLanceDataSource.scala | 165 ++++++++++++++++++ 4 files changed, 185 insertions(+), 14 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java index 5ee9898fc503e..6bb9070d61af6 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java @@ -172,12 +172,11 @@ private static StructType enrichSparkSchemaForLanceVectors(StructType sparkSchem newFields[i] = field; continue; } - DataType dt = field.dataType(); - if (!(dt instanceof ArrayType)) { + if (!(field.dataType() instanceof ArrayType)) { throw new HoodieNotSupportedException( - "Hudi VECTOR column '" + field.name() + "' must be Spark ArrayType to be written to Lance; got " + dt); + "Hudi VECTOR column '" + field.name() + "' must be Spark ArrayType to be written to Lance; got " + field.dataType()); } - DataType elemType = ((ArrayType) dt).elementType(); + DataType elemType = ((ArrayType) field.dataType()).elementType(); boolean elemSupported = elemType instanceof FloatType$ || elemType instanceof DoubleType$; if (!elemSupported) { throw new HoodieNotSupportedException( diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index 85e2a3f96d4d0..854761f0665bc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -83,7 +83,11 @@ public static Map detectVectorColumns(HoodieSchema * Builds the value string for {@link HoodieSchema#PARQUET_VECTOR_COLUMNS_METADATA_KEY} * from a Spark {@link StructType}, matching the comma-separated * {@code colName:VECTOR(dim[,elemType])} format produced on the Parquet path by - * {@link HoodieSchema#buildVectorColumnsMetadataValue}. + * {@link HoodieSchema#buildVectorColumnsMetadataValue(org.apache.hudi.common.schema.HoodieSchema)}. + * + *

This Spark-specific variant operates on {@link StructType} field metadata + * rather than {@link org.apache.hudi.common.schema.HoodieSchema}, since the Lance + * writer works with Spark schemas directly and does not convert to HoodieSchema. * *

Intended to be written as file-footer metadata (e.g. via * {@code LanceFileWriter.addSchemaMetadata}) so any reader can identify VECTOR @@ -283,6 +287,8 @@ public static void convertRowVectorColumns(InternalRow row, GenericInternalRow r * {@link HoodieSchema#TYPE_METADATA_FIELD} = {@code "VECTOR(dim[, DOUBLE])"} to matching * fields so downstream VECTOR-aware code sees the same schema it would on the Parquet path. * + *

Only top-level fields are inspected; nested struct children are not recursed into. + * *

Only FLOAT and DOUBLE element types are recognized, matching what the Lance write * path can produce today. Any other Arrow type (including non-FixedSizeList or * FixedSizeList of non-floating-point children) is passed through unchanged. diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java index 7835880c7406f..903a6d38c586c 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java @@ -177,17 +177,18 @@ public void close() throws IOException { writer.write(root); } - // Finalize and write bloom filter metadata - if (writer != null && bloomFilterWriteSupportOpt.isPresent()) { - Map metadata = bloomFilterWriteSupportOpt.get().finalizeMetadata(); - if (!metadata.isEmpty()) { - writer.addSchemaMetadata(metadata); + if (writer != null) { + // Finalize and write bloom filter metadata + if (bloomFilterWriteSupportOpt.isPresent()) { + Map metadata = bloomFilterWriteSupportOpt.get().finalizeMetadata(); + if (!metadata.isEmpty()) { + writer.addSchemaMetadata(metadata); + } } - } - // Allow subclasses to contribute additional footer key-value metadata - // (e.g. Spark writer emits `hoodie.vector.columns` for forward-compat read). - if (writer != null) { + // Allow subclasses to contribute additional footer key-value metadata + // (e.g. Spark writer emits `hoodie.vector.columns` for forward-compat read). + // Called unconditionally; returns an empty map when no VECTOR columns are present. Map extra = additionalSchemaMetadata(); if (extra != null && !extra.isEmpty()) { writer.addSchemaMetadata(extra); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index 2a983ccc10a54..52e42d4f8c1b8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -913,6 +913,171 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { s"embedding:VECTOR($embeddingDim),features:VECTOR($featuresDim, DOUBLE)") } + @ParameterizedTest + @EnumSource(value = classOf[HoodieTableType]) + def testNullableVectorRoundTrip(tableType: HoodieTableType): Unit = { + val tableName = s"test_lance_vec_nullable_${tableType.name().toLowerCase}" + val tablePath = s"$basePath/$tableName" + + val dim = 3 + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("embedding", + ArrayType(FloatType, containsNull = false), + nullable = true, + vectorMetadata(s"VECTOR($dim)")) + )) + val data = Seq( + Row(1, Array(1.0f, 2.0f, 3.0f)), + Row(2, null), + Row(3, Array(7.0f, 8.0f, 9.0f)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) + + writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite) + + val readDf = spark.read.format("hudi").load(tablePath).select("id", "embedding") + assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim)") + + val rows = readDf.collect().sortBy(_.getInt(0)) + assertEquals(3, rows.length) + assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getAs[Seq[Float]](1).toSeq) + assertTrue(rows(1).isNullAt(1), "Row with id=2 should have null embedding") + assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getAs[Seq[Float]](1).toSeq) + } + + @ParameterizedTest + @EnumSource(value = classOf[HoodieTableType]) + def testVectorMorUpdatePath(tableType: HoodieTableType): Unit = { + val tableName = s"test_lance_vec_update_${tableType.name().toLowerCase}" + val tablePath = s"$basePath/$tableName" + + val dim = 3 + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("name", StringType, nullable = false), + StructField("age", IntegerType, nullable = false), + StructField("embedding", + ArrayType(FloatType, containsNull = false), + nullable = false, + vectorMetadata(s"VECTOR($dim)")) + )) + + // Initial insert + val data1 = Seq( + Row(1, "Alice", 30, Array(1.0f, 2.0f, 3.0f)), + Row(2, "Bob", 25, Array(4.0f, 5.0f, 6.0f)), + Row(3, "Charlie", 35, Array(7.0f, 8.0f, 9.0f)) + ) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(data1), schema).coalesce(1) + writeDataframe(tableType, tableName, tablePath, df1, saveMode = SaveMode.Overwrite, operation = Some("insert")) + + // Upsert — update Bob's embedding + val data2 = Seq( + Row(2, "Bob", 40, Array(10.0f, 20.0f, 30.0f)) + ) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(data2), schema).coalesce(1) + writeDataframe(tableType, tableName, tablePath, df2, operation = Some("upsert")) + + // Read merged result + val readDf = spark.read.format("hudi").load(tablePath).select("id", "name", "age", "embedding") + assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim)") + + val rows = readDf.collect().sortBy(_.getInt(0)) + assertEquals(3, rows.length) + assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getAs[Seq[Float]](3).toSeq) + assertEquals(Seq(10.0f, 20.0f, 30.0f), rows(1).getAs[Seq[Float]](3).toSeq) + assertEquals(40, rows(1).getInt(2), "Bob's age should be updated to 40") + assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getAs[Seq[Float]](3).toSeq) + } + + @ParameterizedTest + @EnumSource(value = classOf[HoodieTableType]) + def testVectorProjection(tableType: HoodieTableType): Unit = { + val tableName = s"test_lance_vec_proj_${tableType.name().toLowerCase}" + val tablePath = s"$basePath/$tableName" + + val dim = 4 + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("name", StringType, nullable = false), + StructField("embedding", + ArrayType(FloatType, containsNull = false), + nullable = false, + vectorMetadata(s"VECTOR($dim)")) + )) + val data = Seq( + Row(1, "Alice", Array(1.0f, 2.0f, 3.0f, 4.0f)), + Row(2, "Bob", Array(5.0f, 6.0f, 7.0f, 8.0f)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) + writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite) + + // Project only the vector column + val vecOnly = spark.read.format("hudi").load(tablePath).select("embedding") + assertEquals(1, vecOnly.schema.fields.length) + assertHudiTypeMetadata(vecOnly.schema("embedding"), s"VECTOR($dim)") + val vecRows = vecOnly.collect().map(_.getAs[Seq[Float]](0).toSeq).toSet + assertEquals(Set(Seq(1.0f, 2.0f, 3.0f, 4.0f), Seq(5.0f, 6.0f, 7.0f, 8.0f)), vecRows) + + // Project vector alongside Hudi metadata columns + val withMeta = spark.read.format("hudi").load(tablePath) + .select("_hoodie_record_key", "embedding") + assertEquals(2, withMeta.schema.fields.length) + assertHudiTypeMetadata(withMeta.schema("embedding"), s"VECTOR($dim)") + val metaRows = withMeta.collect().map(r => + r.getString(0) -> r.getAs[Seq[Float]](1).toSeq).toMap + assertEquals(Seq(1.0f, 2.0f, 3.0f, 4.0f), metaRows("1")) + assertEquals(Seq(5.0f, 6.0f, 7.0f, 8.0f), metaRows("2")) + } + + @ParameterizedTest + @EnumSource(value = classOf[HoodieTableType]) + def testVectorPartitionedTable(tableType: HoodieTableType): Unit = { + val tableName = s"test_lance_vec_part_${tableType.name().toLowerCase}" + val tablePath = s"$basePath/$tableName" + + val dim = 3 + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("department", StringType, nullable = false), + StructField("age", IntegerType, nullable = false), + StructField("embedding", + ArrayType(FloatType, containsNull = false), + nullable = false, + vectorMetadata(s"VECTOR($dim)")) + )) + val data = Seq( + Row(1, "engineering", 30, Array(1.0f, 2.0f, 3.0f)), + Row(2, "sales", 25, Array(4.0f, 5.0f, 6.0f)), + Row(3, "engineering", 35, Array(7.0f, 8.0f, 9.0f)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) + writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite, + extraOptions = Map(PARTITIONPATH_FIELD.key() -> "department")) + + // Read back all partitions + val readDf = spark.read.format("hudi").load(tablePath).select("id", "department", "embedding") + assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim)") + + val rows = readDf.collect().sortBy(_.getInt(0)) + assertEquals(3, rows.length) + assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getAs[Seq[Float]](2).toSeq) + assertEquals("engineering", rows(0).getString(1)) + assertEquals(Seq(4.0f, 5.0f, 6.0f), rows(1).getAs[Seq[Float]](2).toSeq) + assertEquals("sales", rows(1).getString(1)) + assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getAs[Seq[Float]](2).toSeq) + assertEquals("engineering", rows(2).getString(1)) + + // Read a single partition + val engDf = spark.read.format("hudi").load(tablePath) + .filter("department = 'engineering'").select("id", "embedding") + val engRows = engDf.collect().sortBy(_.getInt(0)) + assertEquals(2, engRows.length) + assertEquals(Seq(1.0f, 2.0f, 3.0f), engRows(0).getAs[Seq[Float]](1).toSeq) + assertEquals(Seq(7.0f, 8.0f, 9.0f), engRows(1).getAs[Seq[Float]](1).toSeq) + } + private def assertHudiTypeMetadata(field: StructField, expectedDescriptor: String): Unit = { assertTrue(field.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD), s"Expected field ${field.name} to carry ${HoodieSchema.TYPE_METADATA_FIELD} metadata after read") From 66276af9989fbca4d9007e6cd0ba038906dee384 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Tue, 14 Apr 2026 20:20:56 -0700 Subject: [PATCH 05/15] intial self review --- .../io/storage/HoodieSparkLanceWriter.java | 27 ++---- .../io/storage/VectorConversionUtils.java | 64 +++++-------- .../row/HoodieRowParquetWriteSupport.java | 2 +- .../hudi/common/schema/HoodieSchema.java | 48 +++++++--- .../hudi/avro/HoodieAvroWriteSupport.java | 2 +- .../hudi/io/lance/HoodieBaseLanceWriter.java | 3 +- ...HoodieFileGroupReaderBasedFileFormat.scala | 7 +- .../hudi/functional/TestLanceDataSource.scala | 89 ++++++------------- .../functional/TestVectorDataSource.scala | 6 +- 9 files changed, 101 insertions(+), 147 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java index 6bb9070d61af6..b93c2633837f1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java @@ -38,10 +38,6 @@ import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.types.pojo.Schema; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.ArrayType; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DoubleType$; -import org.apache.spark.sql.types.FloatType$; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.StructField; @@ -50,6 +46,7 @@ import org.apache.spark.unsafe.types.UTF8String; import java.io.IOException; +import java.util.Collections; import java.util.Map; import java.util.function.Function; @@ -130,9 +127,6 @@ private HoodieSparkLanceWriter(StoragePath file, Option bloomFilterOpt, long maxFileSize) { super(file, DEFAULT_BATCH_SIZE, bloomFilterOpt.map(HoodieBloomFilterRowWriteSupport::new)); - // Enrich fields carrying the Hudi VECTOR logical type with the Spark metadata key - // that lance-spark's toArrowSchema / LanceArrowWriter use to emit a native Arrow - // FixedSizeList (i.e. Lance's vector column encoding) instead of a plain List. this.sparkSchema = enrichSparkSchemaForLanceVectors(sparkSchema); this.arrowSchema = LanceArrowUtils.toArrowSchema(this.sparkSchema, DEFAULT_TIMEZONE, true, false); this.fileName = UTF8String.fromString(file.getName()); @@ -172,16 +166,12 @@ private static StructType enrichSparkSchemaForLanceVectors(StructType sparkSchem newFields[i] = field; continue; } - if (!(field.dataType() instanceof ArrayType)) { - throw new HoodieNotSupportedException( - "Hudi VECTOR column '" + field.name() + "' must be Spark ArrayType to be written to Lance; got " + field.dataType()); - } - DataType elemType = ((ArrayType) field.dataType()).elementType(); - boolean elemSupported = elemType instanceof FloatType$ || elemType instanceof DoubleType$; - if (!elemSupported) { + HoodieSchema.Vector.VectorElementType elemType = vec.getVectorElementType(); + if (elemType != HoodieSchema.Vector.VectorElementType.FLOAT + && elemType != HoodieSchema.Vector.VectorElementType.DOUBLE) { throw new HoodieNotSupportedException( "Lance base-file format currently supports FLOAT/DOUBLE VECTOR columns only; " - + "got element type " + elemType.simpleString() + " for field '" + field.name() + "'"); + + "got element type " + elemType + " for field '" + field.name() + "'"); } Metadata enriched = new MetadataBuilder() .withMetadata(field.metadata()) @@ -272,12 +262,11 @@ protected Schema getArrowSchema() { */ @Override protected java.util.Map additionalSchemaMetadata() { - String value = VectorConversionUtils.buildVectorColumnsMetadataValue(sparkSchema); + String value = VectorConversionUtils.buildVectorColumnsFooterValue(sparkSchema); if (value.isEmpty()) { - return java.util.Collections.emptyMap(); + return Collections.emptyMap(); } - return java.util.Collections.singletonMap( - org.apache.hudi.common.schema.HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY, value); + return Collections.singletonMap(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY, value); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index 854761f0665bc..13a84390d4d67 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi.io.storage; +import org.apache.arrow.vector.types.pojo.Schema; import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.schema.HoodieSchemaType; @@ -80,40 +81,25 @@ public static Map detectVectorColumns(HoodieSchema } /** - * Builds the value string for {@link HoodieSchema#PARQUET_VECTOR_COLUMNS_METADATA_KEY} - * from a Spark {@link StructType}, matching the comma-separated - * {@code colName:VECTOR(dim[,elemType])} format produced on the Parquet path by - * {@link HoodieSchema#buildVectorColumnsMetadataValue(org.apache.hudi.common.schema.HoodieSchema)}. - * - *

This Spark-specific variant operates on {@link StructType} field metadata - * rather than {@link org.apache.hudi.common.schema.HoodieSchema}, since the Lance - * writer works with Spark schemas directly and does not convert to HoodieSchema. - * - *

Intended to be written as file-footer metadata (e.g. via - * {@code LanceFileWriter.addSchemaMetadata}) so any reader can identify VECTOR - * columns from the file alone, mirroring the Parquet footer convention. + * Builds the {@link HoodieSchema#VECTOR_COLUMNS_METADATA_KEY} footer value + * from a Spark {@link StructType} by detecting VECTOR metadata annotations and + * delegating to {@link HoodieSchema#serializeVectorColumnsMetadata}. * * @param schema Spark StructType (may be null) * @return comma-separated descriptor list, or empty string if no VECTOR columns + * @see HoodieSchema#serializeVectorColumnsMetadata(java.util.Map) */ - public static String buildVectorColumnsMetadataValue(StructType schema) { + public static String buildVectorColumnsFooterValue(StructType schema) { if (schema == null) { return ""; } - StringBuilder sb = new StringBuilder(); StructField[] fields = schema.fields(); Map detected = detectVectorColumnsFromMetadata(schema); - for (int i = 0; i < fields.length; i++) { - HoodieSchema.Vector v = detected.get(i); - if (v == null) { - continue; - } - if (sb.length() > 0) { - sb.append(','); - } - sb.append(fields[i].name()).append(':').append(v.toTypeDescriptor()); + java.util.LinkedHashMap named = new java.util.LinkedHashMap<>(); + for (Map.Entry entry : detected.entrySet()) { + named.put(fields[entry.getKey()].name(), entry.getValue()); } - return sb.toString(); + return HoodieSchema.serializeVectorColumnsMetadata(named); } /** @@ -278,28 +264,20 @@ public static void convertRowVectorColumns(InternalRow row, GenericInternalRow r } /** - * Restores Hudi's VECTOR logical-type metadata on a Spark schema derived from a Lance - * (Arrow) schema. The upstream conversion {@code LanceArrowUtils.fromArrowSchema} maps - * Arrow {@code FixedSizeList} to Spark {@code ArrayType(FloatType|DoubleType)} - * but drops all field-level metadata, so a column that was written as a Hudi VECTOR - * comes back indistinguishable from a plain array. This method walks the original Arrow - * schema (which still carries the fixed-size-list intent and dimension) and re-attaches - * {@link HoodieSchema#TYPE_METADATA_FIELD} = {@code "VECTOR(dim[, DOUBLE])"} to matching - * fields so downstream VECTOR-aware code sees the same schema it would on the Parquet path. - * - *

Only top-level fields are inspected; nested struct children are not recursed into. + * Re-attaches {@link HoodieSchema#TYPE_METADATA_FIELD} to Spark fields that were + * Arrow {@code FixedSizeList} in the Lance file. + * {@code LanceArrowUtils.fromArrowSchema} drops all field metadata, so without this + * step VECTOR columns are indistinguishable from plain arrays. * - *

Only FLOAT and DOUBLE element types are recognized, matching what the Lance write - * path can produce today. Any other Arrow type (including non-FixedSizeList or - * FixedSizeList of non-floating-point children) is passed through unchanged. + *

Only top-level FLOAT/DOUBLE fields are inspected; nested structs are not recursed. * - * @param arrowSchema the original Arrow schema read from the Lance file - * @param convertedSpark the StructType produced by {@code LanceArrowUtils.fromArrowSchema} - * (must have the same field count and order as {@code arrowSchema}) - * @return a new StructType with VECTOR metadata restored on eligible fields + * @param arrowSchema original Arrow schema from the Lance file + * @param convertedSpark Spark schema from {@code LanceArrowUtils.fromArrowSchema} + * (same field count and order as {@code arrowSchema}) + * @return StructType with VECTOR metadata restored on eligible fields */ public static StructType restoreVectorMetadataFromArrowSchema( - org.apache.arrow.vector.types.pojo.Schema arrowSchema, StructType convertedSpark) { + Schema arrowSchema, StructType convertedSpark) { if (arrowSchema == null || convertedSpark == null) { return convertedSpark; } @@ -358,6 +336,6 @@ private static String deriveVectorDescriptor(Field arrowField) { } else { return null; } - return ((HoodieSchema.Vector) HoodieSchema.createVector(dim, elementType)).toTypeDescriptor(); + return HoodieSchema.createVector(dim, elementType).toTypeDescriptor(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index 7e03716853849..80496dcc0839b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -167,7 +167,7 @@ public WriteContext init(Configuration configuration) { } String vectorMeta = HoodieSchema.buildVectorColumnsMetadataValue(schema); if (!vectorMeta.isEmpty()) { - metadata.put(HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY, vectorMeta); + metadata.put(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY, vectorMeta); } Configuration configurationCopy = new Configuration(configuration); configurationCopy.set(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, Boolean.toString(writeLegacyListFormat)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java index e20aeea249114..9a068fb224bea 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java @@ -209,39 +209,59 @@ private static Pair> tokenizeTypeDescriptor(Strin public static final String PARQUET_ARRAY_AVRO = "." + ARRAY_LIST_ELEMENT; /** - * Parquet file-footer metadata key under which VECTOR column names and type descriptors + * Base-file footer metadata key under which VECTOR column names and type descriptors * are recorded. The value is a comma-separated list of {@code colName:VECTOR(dim[,elemType])} * entries, e.g. {@code "embedding:VECTOR(128),tags:VECTOR(64,INT8)"}. * - *

Stored as file-level key-value metadata (Parquet footer) so that any reader can - * identify vector columns without needing the Hudi schema store. + *

Stored as file-level key-value metadata (Parquet footer, Lance schema metadata) + * so that any reader can identify vector columns without needing the Hudi schema store. */ - public static final String PARQUET_VECTOR_COLUMNS_METADATA_KEY = "hoodie.vector.columns"; + public static final String VECTOR_COLUMNS_METADATA_KEY = "hoodie.vector.columns"; /** - * Builds the value string for {@link #PARQUET_VECTOR_COLUMNS_METADATA_KEY}. + * Serializes a name-to-Vector map into the comma-separated + * {@code colName:VECTOR(dim[,elemType])} format used for {@link #VECTOR_COLUMNS_METADATA_KEY}. + * + *

This is the single canonical serializer — all format-specific code (Parquet, Lance) + * should build the map from their respective schema representation and delegate here. + * + * @param vectorColumns ordered map of field name to Vector descriptor (iteration order is preserved) + * @return comma-separated descriptor list, or empty string if the map is null or empty + */ + public static String serializeVectorColumnsMetadata(java.util.Map vectorColumns) { + if (vectorColumns == null || vectorColumns.isEmpty()) { + return ""; + } + StringBuilder sb = new StringBuilder(); + for (java.util.Map.Entry entry : vectorColumns.entrySet()) { + if (sb.length() > 0) { + sb.append(','); + } + sb.append(entry.getKey()).append(':').append(entry.getValue().toTypeDescriptor()); + } + return sb.toString(); + } + + /** + * Builds the value string for {@link #VECTOR_COLUMNS_METADATA_KEY} from a {@link HoodieSchema}. * * @param schema a HoodieSchema of type RECORD (or null) * @return comma-separated {@code colName:VECTOR(dim[,elemType])} entries, or empty string * if the schema is null or has no VECTOR columns + * @see #serializeVectorColumnsMetadata(java.util.Map) */ public static String buildVectorColumnsMetadataValue(HoodieSchema schema) { if (schema == null || schema.isSchemaNull()) { return ""; } - List fields = schema.getFields(); - StringBuilder sb = new StringBuilder(); - for (HoodieSchemaField field : fields) { + java.util.LinkedHashMap vectorColumns = new java.util.LinkedHashMap<>(); + for (HoodieSchemaField field : schema.getFields()) { HoodieSchema fieldSchema = field.schema().getNonNullType(); if (fieldSchema.getType() == HoodieSchemaType.VECTOR) { - Vector vectorSchema = (Vector) fieldSchema; - if (sb.length() > 0) { - sb.append(','); - } - sb.append(field.name()).append(':').append(vectorSchema.toTypeDescriptor()); + vectorColumns.put(field.name(), (Vector) fieldSchema); } } - return sb.toString(); + return serializeVectorColumnsMetadata(vectorColumns); } private Schema avroSchema; diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java index 547b02787369f..df7930060fe93 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java @@ -50,7 +50,7 @@ public HoodieAvroWriteSupport(MessageType schema, HoodieSchema hoodieSchema, Opt this.properties = properties; String vectorMeta = HoodieSchema.buildVectorColumnsMetadataValue(hoodieSchema); if (!vectorMeta.isEmpty()) { - footerMetadata.put(HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY, vectorMeta); + footerMetadata.put(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY, vectorMeta); } } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java index 903a6d38c586c..f43c56625f592 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/lance/HoodieBaseLanceWriter.java @@ -37,6 +37,7 @@ import java.io.Closeable; import java.io.IOException; +import java.util.Collections; import java.util.Map; /** @@ -116,7 +117,7 @@ protected HoodieBaseLanceWriter(StoragePath path, int batchSize, * @return map of footer metadata key-value pairs, or empty map for none */ protected Map additionalSchemaMetadata() { - return java.util.Collections.emptyMap(); + return Collections.emptyMap(); } /** diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala index afcee5820fc4c..ef14369107de2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala @@ -442,11 +442,16 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, * @return (modified schema with BinaryType for vectors, vector column ordinal map) */ private def withVectorRewrite(schema: StructType): (StructType, Map[Int, HoodieSchema.Vector]) = { + // Only Parquet needs the BinaryType rewrite; other formats (Lance) return ArrayType natively. if (hoodieFileFormat != HoodieFileFormat.PARQUET) { (schema, Map.empty[Int, HoodieSchema.Vector]) } else { val vecs = detectVectorColumns(schema) - if (vecs.nonEmpty) (replaceVectorFieldsWithBinary(schema, vecs), vecs) else (schema, vecs) + if (vecs.isEmpty) { + (schema, vecs) + } else { + (replaceVectorFieldsWithBinary(schema, vecs), vecs) + } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index 52e42d4f8c1b8..73a8f63961264 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -21,7 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.DefaultSparkRecordMerger import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMetadataConfig} import org.apache.hudi.common.engine.HoodieLocalEngineContext -import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.model.{HoodieBaseFile, HoodieTableType} import org.apache.hudi.common.schema.HoodieSchema import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.view.{FileSystemViewManager, FileSystemViewStorageConfig} @@ -1088,23 +1088,8 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { private def vectorMetadata(descriptor: String): Metadata = new MetadataBuilder().putString(HoodieSchema.TYPE_METADATA_FIELD, descriptor).build() - /** - * Opens the raw Lance base file(s) for the given Hudi table and asserts that the - * named field is encoded as an Arrow FixedSizeList with the expected listSize. - * This proves that the write path used Lance's native vector column encoding - * rather than a plain variable-length list. - */ - /** - * Opens the raw Lance base file(s) for the given Hudi table and asserts that the - * Arrow schema's custom metadata carries `hoodie.vector.columns` with the given - * expected descriptor list. This is the forward-compat guard — future readers - * can recover the exact Hudi VECTOR descriptor from the footer without - * re-deriving from the Arrow type. - * - *

Descriptor list order is writer-determined (matches Spark schema field - * order), so callers must pass the entries in that order. - */ - private def assertLanceFooterHasVectorColumns(tablePath: String, expected: String): Unit = { + /** Runs `check` against each Lance base file's Arrow schema. */ + private def forEachLanceSchema(tablePath: String)(check: (org.apache.arrow.vector.types.pojo.Schema, String) => Unit): Unit = { val metaClient = HoodieTableMetaClient.builder() .setConf(HoodieTestUtils.getDefaultStorageConf) .setBasePath(tablePath) @@ -1126,13 +1111,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { baseFiles.asScala.foreach { bf => val reader = LanceFileReader.open(bf.getPath, allocator) try { - val meta = reader.schema().getCustomMetadata - assertNotNull(meta, s"Lance footer metadata null for ${bf.getPath}") - val key = HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY - assertTrue(meta.containsKey(key), - s"Lance file ${bf.getPath} should have footer key $key, got keys ${meta.keySet()}") - assertEquals(expected, meta.get(key), - s"Lance file ${bf.getPath} footer $key mismatch") + check(reader.schema(), bf.getPath) } finally { reader.close() } @@ -1145,47 +1124,29 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { } } + private def assertLanceFooterHasVectorColumns(tablePath: String, expected: String): Unit = { + forEachLanceSchema(tablePath) { (schema, path) => + val meta = schema.getCustomMetadata + assertNotNull(meta, s"Lance footer metadata null for $path") + val key = HoodieSchema.VECTOR_COLUMNS_METADATA_KEY + assertTrue(meta.containsKey(key), + s"Lance file $path should have footer key $key, got keys ${meta.keySet()}") + assertEquals(expected, meta.get(key), s"Lance file $path footer $key mismatch") + } + } + private def assertLanceFieldIsFixedSizeList(tablePath: String, fieldName: String, expectedDim: Int): Unit = { - val metaClient = HoodieTableMetaClient.builder() - .setConf(HoodieTestUtils.getDefaultStorageConf) - .setBasePath(tablePath) - .build() - val engineContext = new HoodieLocalEngineContext(metaClient.getStorageConf) - val metadataConfig = HoodieMetadataConfig.newBuilder.build - val viewManager = FileSystemViewManager.createViewManager( - engineContext, metadataConfig, FileSystemViewStorageConfig.newBuilder.build, - HoodieCommonConfig.newBuilder.build, - (mc: HoodieTableMetaClient) => metaClient.getTableFormat - .getMetadataFactory.create(engineContext, mc.getStorage, metadataConfig, tablePath)) - val fsView = viewManager.getFileSystemView(metaClient) - try { - val baseFiles = fsView.getLatestBaseFiles("") - .collect(Collectors.toList[org.apache.hudi.common.model.HoodieBaseFile]) - assertTrue(baseFiles.size() > 0, "Expected at least one Lance base file") - val allocator = new RootAllocator() - try { - baseFiles.asScala.foreach { bf => - val reader = LanceFileReader.open(bf.getPath, allocator) - try { - val field = reader.schema().findField(fieldName) - assertNotNull(field, s"Field $fieldName not found in Lance schema for ${bf.getPath}") - field.getType match { - case fsl: ArrowType.FixedSizeList => - assertEquals(expectedDim, fsl.getListSize, - s"Lance field $fieldName in ${bf.getPath} should be FixedSizeList of size $expectedDim") - case other => - throw new AssertionError( - s"Lance field $fieldName in ${bf.getPath} should be FixedSizeList but was $other") - } - } finally { - reader.close() - } - } - } finally { - allocator.close() + forEachLanceSchema(tablePath) { (schema, path) => + val field = schema.findField(fieldName) + assertNotNull(field, s"Field $fieldName not found in Lance schema for $path") + field.getType match { + case fsl: ArrowType.FixedSizeList => + assertEquals(expectedDim, fsl.getListSize, + s"Lance field $fieldName in $path should be FixedSizeList of size $expectedDim") + case other => + throw new AssertionError( + s"Lance field $fieldName in $path should be FixedSizeList but was $other") } - } finally { - fsView.close() } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestVectorDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestVectorDataSource.scala index 0522950f0d9f3..675e975cfcef5 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestVectorDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestVectorDataSource.scala @@ -770,10 +770,10 @@ class TestVectorDataSource extends HoodieSparkClientTestBase { val reader = ParquetFileReader.open(HadoopInputFile.fromPath(parquetFiles.head.getPath, conf)) try { val footerMeta = reader.getFileMetaData.getKeyValueMetaData.asScala - assertTrue(footerMeta.contains(HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY), - s"Footer should contain ${HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY}, got keys: ${footerMeta.keys.mkString(", ")}") + assertTrue(footerMeta.contains(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY), + s"Footer should contain ${HoodieSchema.VECTOR_COLUMNS_METADATA_KEY}, got keys: ${footerMeta.keys.mkString(", ")}") - val value = footerMeta(HoodieSchema.PARQUET_VECTOR_COLUMNS_METADATA_KEY) + val value = footerMeta(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY) assertTrue(value.contains("embedding"), s"Footer value should reference 'embedding' column, got: $value") assertTrue(value.contains("VECTOR"), s"Footer value should contain 'VECTOR' descriptor, got: $value") } finally { From 4a0acc8aefdbb7fb5dc36e0be4ddb1af012c020c Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 15 Apr 2026 11:39:31 -0700 Subject: [PATCH 06/15] fix(lance): skip VECTOR->BinaryType rewrite for Lance base files MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit SparkFileFormatInternalRowReaderContext.getFileRecordIterator had a second, unconditional rewrite of VECTOR columns from ArrayType to BinaryType (the earlier withVectorRewrite gate in HoodieFileGroupReaderBasedFileFormat only covered the non-FileGroupReader branch). On the MOR / FileGroupReader path this caused Lance reads to fail with scala.MatchError: ArrayType(FloatType,true) in Cast.castToBinaryCode, because Lance returns vectors natively as ArrayType while the caller-supplied schema had been rewritten to BinaryType — the generated UnsafeProjection then injected an unsupported Cast(ArrayType -> BinaryType). Gate the detection + rewrite on the file format: skip it for .lance base files. Hudi log files are always parquet-encoded so they still take the Parquet path. Fixes 14 TestLanceDataSource vector errors (COW + MOR) observed in spark3.5 / spark3.4 CI, including the spark3.4 part2 6h timeout that was the same failure retrying. Co-Authored-By: Claude Opus 4.6 --- ...SparkFileFormatInternalRowReaderContext.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 8c0980e009f05..840b622fdb1da 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{filterIsSafeForBootstrap, filterIsSafeForPrimaryKey, getAppliedRequiredSchema} import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaUtils} import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.table.read.buffer.PositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME @@ -82,8 +82,18 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) // Detect VECTOR columns and replace with BinaryType for the Parquet reader - // (Parquet stores VECTOR as FIXED_LEN_BYTE_ARRAY which Spark maps to BinaryType) - val vectorColumnInfo = SparkFileFormatInternalRowReaderContext.detectVectorColumns(requiredSchema) + // (Parquet stores VECTOR as FIXED_LEN_BYTE_ARRAY which Spark maps to BinaryType). + // Lance stores vectors natively as Arrow FixedSizeList and returns them as ArrayType, + // so the BinaryType rewrite would cause an invalid Cast(ArrayType -> BinaryType) in + // the reader's projection — skip it for .lance base files. Hudi log files are always + // parquet-encoded, so we only need to special-case Lance base files here. + val isLanceBaseFile = !FSUtils.isLogFile(filePath) && + filePath.getName.endsWith(HoodieFileFormat.LANCE.getFileExtension) + val vectorColumnInfo: Map[Int, HoodieSchema.Vector] = if (isLanceBaseFile) { + Map.empty + } else { + SparkFileFormatInternalRowReaderContext.detectVectorColumns(requiredSchema) + } val parquetReadStructType = if (vectorColumnInfo.nonEmpty) { SparkFileFormatInternalRowReaderContext.replaceVectorColumnsWithBinary(structType, vectorColumnInfo) } else { From e70393566a401f1391ae0acd22d765054d404ec0 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 15 Apr 2026 12:13:19 -0700 Subject: [PATCH 07/15] fix comment to be concise --- .../hudi/SparkFileFormatInternalRowReaderContext.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 840b622fdb1da..e0b6f136c162e 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -81,12 +81,8 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR } val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) - // Detect VECTOR columns and replace with BinaryType for the Parquet reader - // (Parquet stores VECTOR as FIXED_LEN_BYTE_ARRAY which Spark maps to BinaryType). - // Lance stores vectors natively as Arrow FixedSizeList and returns them as ArrayType, - // so the BinaryType rewrite would cause an invalid Cast(ArrayType -> BinaryType) in - // the reader's projection — skip it for .lance base files. Hudi log files are always - // parquet-encoded, so we only need to special-case Lance base files here. + // Parquet stores VECTOR as FIXED_LEN_BYTE_ARRAY, so the reader needs BinaryType + // and we decode back to ArrayType below. Lance returns ArrayType natively, so skip. val isLanceBaseFile = !FSUtils.isLogFile(filePath) && filePath.getName.endsWith(HoodieFileFormat.LANCE.getFileExtension) val vectorColumnInfo: Map[Int, HoodieSchema.Vector] = if (isLanceBaseFile) { From aa47b75b89f3e7ce94d5a59452b3248eda0c9338 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 15 Apr 2026 13:22:02 -0700 Subject: [PATCH 08/15] test(lance): use Row.getSeq[T] for Scala 2.13 compatibility On Scala 2.13, Row.getAs[Seq[Float]] fails at runtime with ClassCastException: scala.collection.mutable.ArraySeq$ofRef cannot be cast to scala.collection.immutable.Seq, because Seq in Scala 2.13 defaults to immutable.Seq while Spark holds array columns as mutable.ArraySeq internally. Row.getSeq[T] is declared as scala.collection.Seq[T] (general), so it works on both 2.12 (where Seq = scala.collection.Seq) and 2.13 (where Seq = scala.collection.immutable.Seq). Same runtime object, no cast. Fixes the 14 TestLanceDataSource errors on java17 CI (scala-2.13, spark3.5 / spark4.0). The earlier VECTOR->BinaryType rewrite fix resolved the scala.MatchError in the read path; this change resolves the subsequent 2.13-only test-side ClassCastException. Co-Authored-By: Claude Opus 4.6 --- .../hudi/functional/TestLanceDataSource.scala | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index 73a8f63961264..8bfb57aededda 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -819,7 +819,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim)") val actualByKey = readDf.collect().map(r => - r.getInt(0) -> r.getAs[Seq[Float]](1).toSeq).toMap + r.getInt(0) -> r.getSeq[Float](1).toSeq).toMap val expectedByKey = data.map(r => r.getInt(0) -> r.getAs[Array[Float]](1).toSeq).toMap assertEquals(expectedByKey, actualByKey) @@ -858,7 +858,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim, DOUBLE)") val actualByKey = readDf.collect().map(r => - r.getInt(0) -> r.getAs[Seq[Double]](1).toSeq).toMap + r.getInt(0) -> r.getSeq[Double](1).toSeq).toMap val expectedByKey = data.map(r => r.getInt(0) -> r.getAs[Array[Double]](1).toSeq).toMap assertEquals(expectedByKey, actualByKey) @@ -896,7 +896,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { val readDf = spark.read.format("hudi").load(tablePath).select("id", "embedding", "features") val rows = readDf.collect().map { r => - (r.getInt(0), r.getAs[Seq[Float]](1).toSeq, r.getAs[Seq[Double]](2).toSeq) + (r.getInt(0), r.getSeq[Float](1).toSeq, r.getSeq[Double](2).toSeq) }.toSet val expected = data.map { r => (r.getInt(0), @@ -941,9 +941,9 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { val rows = readDf.collect().sortBy(_.getInt(0)) assertEquals(3, rows.length) - assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getAs[Seq[Float]](1).toSeq) + assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getSeq[Float](1).toSeq) assertTrue(rows(1).isNullAt(1), "Row with id=2 should have null embedding") - assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getAs[Seq[Float]](1).toSeq) + assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getSeq[Float](1).toSeq) } @ParameterizedTest @@ -985,10 +985,10 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { val rows = readDf.collect().sortBy(_.getInt(0)) assertEquals(3, rows.length) - assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getAs[Seq[Float]](3).toSeq) - assertEquals(Seq(10.0f, 20.0f, 30.0f), rows(1).getAs[Seq[Float]](3).toSeq) + assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getSeq[Float](3).toSeq) + assertEquals(Seq(10.0f, 20.0f, 30.0f), rows(1).getSeq[Float](3).toSeq) assertEquals(40, rows(1).getInt(2), "Bob's age should be updated to 40") - assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getAs[Seq[Float]](3).toSeq) + assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getSeq[Float](3).toSeq) } @ParameterizedTest @@ -1017,7 +1017,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { val vecOnly = spark.read.format("hudi").load(tablePath).select("embedding") assertEquals(1, vecOnly.schema.fields.length) assertHudiTypeMetadata(vecOnly.schema("embedding"), s"VECTOR($dim)") - val vecRows = vecOnly.collect().map(_.getAs[Seq[Float]](0).toSeq).toSet + val vecRows = vecOnly.collect().map(_.getSeq[Float](0).toSeq).toSet assertEquals(Set(Seq(1.0f, 2.0f, 3.0f, 4.0f), Seq(5.0f, 6.0f, 7.0f, 8.0f)), vecRows) // Project vector alongside Hudi metadata columns @@ -1026,7 +1026,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertEquals(2, withMeta.schema.fields.length) assertHudiTypeMetadata(withMeta.schema("embedding"), s"VECTOR($dim)") val metaRows = withMeta.collect().map(r => - r.getString(0) -> r.getAs[Seq[Float]](1).toSeq).toMap + r.getString(0) -> r.getSeq[Float](1).toSeq).toMap assertEquals(Seq(1.0f, 2.0f, 3.0f, 4.0f), metaRows("1")) assertEquals(Seq(5.0f, 6.0f, 7.0f, 8.0f), metaRows("2")) } @@ -1062,11 +1062,11 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { val rows = readDf.collect().sortBy(_.getInt(0)) assertEquals(3, rows.length) - assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getAs[Seq[Float]](2).toSeq) + assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getSeq[Float](2).toSeq) assertEquals("engineering", rows(0).getString(1)) - assertEquals(Seq(4.0f, 5.0f, 6.0f), rows(1).getAs[Seq[Float]](2).toSeq) + assertEquals(Seq(4.0f, 5.0f, 6.0f), rows(1).getSeq[Float](2).toSeq) assertEquals("sales", rows(1).getString(1)) - assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getAs[Seq[Float]](2).toSeq) + assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getSeq[Float](2).toSeq) assertEquals("engineering", rows(2).getString(1)) // Read a single partition @@ -1074,8 +1074,8 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { .filter("department = 'engineering'").select("id", "embedding") val engRows = engDf.collect().sortBy(_.getInt(0)) assertEquals(2, engRows.length) - assertEquals(Seq(1.0f, 2.0f, 3.0f), engRows(0).getAs[Seq[Float]](1).toSeq) - assertEquals(Seq(7.0f, 8.0f, 9.0f), engRows(1).getAs[Seq[Float]](1).toSeq) + assertEquals(Seq(1.0f, 2.0f, 3.0f), engRows(0).getSeq[Float](1).toSeq) + assertEquals(Seq(7.0f, 8.0f, 9.0f), engRows(1).getSeq[Float](1).toSeq) } private def assertHudiTypeMetadata(field: StructField, expectedDescriptor: String): Unit = { From 40bef3c932fbb638dd36bef7ff6c6d4a7326cda4 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 15 Apr 2026 19:02:49 -0700 Subject: [PATCH 09/15] address review: use tableConfig for Lance detection, remove defensive size check - SparkFileFormatInternalRowReaderContext: use tableConfig.getBaseFileFormat instead of filename extension sniff to detect Lance base files - VectorConversionUtils.restoreVectorMetadataFromArrowSchema: remove confusing arrowFields.size != sparkFields.length defensive guard Co-Authored-By: Claude Opus 4.6 --- .../org/apache/hudi/io/storage/VectorConversionUtils.java | 4 ---- .../apache/hudi/SparkFileFormatInternalRowReaderContext.scala | 3 ++- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index 13a84390d4d67..cab3e2a3cdb98 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -283,10 +283,6 @@ public static StructType restoreVectorMetadataFromArrowSchema( } List arrowFields = arrowSchema.getFields(); StructField[] sparkFields = convertedSpark.fields(); - if (arrowFields.size() != sparkFields.length) { - // Schemas desynchronized — bail out rather than risk corrupting metadata. - return convertedSpark; - } StructField[] newFields = new StructField[sparkFields.length]; boolean anyChanged = false; for (int i = 0; i < sparkFields.length; i++) { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index e0b6f136c162e..87e044e6d6790 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -83,8 +83,9 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR // Parquet stores VECTOR as FIXED_LEN_BYTE_ARRAY, so the reader needs BinaryType // and we decode back to ArrayType below. Lance returns ArrayType natively, so skip. + // Log files are always parquet regardless of the table's base-file format. val isLanceBaseFile = !FSUtils.isLogFile(filePath) && - filePath.getName.endsWith(HoodieFileFormat.LANCE.getFileExtension) + tableConfig.getBaseFileFormat == HoodieFileFormat.LANCE val vectorColumnInfo: Map[Int, HoodieSchema.Vector] = if (isLanceBaseFile) { Map.empty } else { From 2ed72a52740c1c67337a3e307ebf3cec1932f311 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 22 Apr 2026 14:01:47 -0700 Subject: [PATCH 10/15] address review: stable vector footer order, fqn cleanup - HoodieSparkLanceWriter.additionalSchemaMetadata: drop redundant java.util.Map FQN (Map is already imported) - VectorConversionUtils.buildVectorColumnsFooterValue: walk fields in ordinal order rather than iterating the detected HashMap, so the hoodie.vector.columns footer value is stable across JDKs - VectorConversionUtils & HoodieSchema: use import for LinkedHashMap instead of fully-qualified class name Co-Authored-By: Claude Opus 4.6 --- .../hudi/io/storage/HoodieSparkLanceWriter.java | 2 +- .../hudi/io/storage/VectorConversionUtils.java | 13 ++++++++++--- .../org/apache/hudi/common/schema/HoodieSchema.java | 3 ++- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java index b93c2633837f1..9f5fe665283c7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java @@ -261,7 +261,7 @@ protected Schema getArrowSchema() { * without a writer bump. */ @Override - protected java.util.Map additionalSchemaMetadata() { + protected Map additionalSchemaMetadata() { String value = VectorConversionUtils.buildVectorColumnsFooterValue(sparkSchema); if (value.isEmpty()) { return Collections.emptyMap(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index cab3e2a3cdb98..c6f9b7b20b967 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -39,6 +39,7 @@ import java.nio.ByteBuffer; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -95,9 +96,15 @@ public static String buildVectorColumnsFooterValue(StructType schema) { } StructField[] fields = schema.fields(); Map detected = detectVectorColumnsFromMetadata(schema); - java.util.LinkedHashMap named = new java.util.LinkedHashMap<>(); - for (Map.Entry entry : detected.entrySet()) { - named.put(fields[entry.getKey()].name(), entry.getValue()); + // Walk fields in ordinal order (not detected.entrySet()) so the footer value + // is emitted in stable, schema-ordered form. detectVectorColumnsFromMetadata + // returns a plain HashMap, whose iteration order is unspecified. + LinkedHashMap named = new LinkedHashMap<>(); + for (int i = 0; i < fields.length; i++) { + HoodieSchema.Vector vec = detected.get(i); + if (vec != null) { + named.put(fields[i].name(), vec); + } } return HoodieSchema.serializeVectorColumnsMetadata(named); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java index 9a068fb224bea..62671b9d76d9f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java @@ -41,6 +41,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.EnumSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -254,7 +255,7 @@ public static String buildVectorColumnsMetadataValue(HoodieSchema schema) { if (schema == null || schema.isSchemaNull()) { return ""; } - java.util.LinkedHashMap vectorColumns = new java.util.LinkedHashMap<>(); + LinkedHashMap vectorColumns = new LinkedHashMap<>(); for (HoodieSchemaField field : schema.getFields()) { HoodieSchema fieldSchema = field.schema().getNonNullType(); if (fieldSchema.getType() == HoodieSchemaType.VECTOR) { From 21c4186f8e7e69f102e6539d388da6ed39532585 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 22 Apr 2026 16:20:02 -0700 Subject: [PATCH 11/15] address review: gate vector metadata restore, auto-attach javadoc, test consolidation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - VectorConversionUtils.restoreVectorMetadataFromArrowSchema: only restore VECTOR descriptor metadata for fields listed in the hoodie.vector.columns footer, so non-VECTOR Arrow FixedSizeList fields cannot be misinterpreted. Drop the anyChanged optimization for simplicity. - HoodieSchema: add parseVectorColumnNames helper to extract field names from the footer value, handling commas inside descriptor parentheses. - HoodieSparkLanceWriter.enrichSparkSchemaForLanceVectors: expand javadoc to explain why we auto-attach arrow.fixed-size-list.size from the VECTOR dimension. - TestLanceDataSource: consolidate testFloatVectorRoundTrip, testDoubleVectorRoundTrip, testVectorMorUpdatePath, testVectorPartitionedTable into testMultipleVectorColumns (two VECTOR columns of different element types + dims, insert then upsert covers MOR merge path). Keep testNullableVectorRoundTrip separate — nullable VECTOR in the upsert/merge path hits a Lance reader bug; tracked as a follow-up. Rename forEachLanceSchema -> validateLanceFileSchema per review suggestion. Co-Authored-By: Claude Opus 4.6 --- .../io/storage/HoodieSparkLanceWriter.java | 9 +- .../io/storage/VectorConversionUtils.java | 21 +- .../hudi/common/schema/HoodieSchema.java | 38 +++ .../hudi/functional/TestLanceDataSource.scala | 245 ++++-------------- 4 files changed, 119 insertions(+), 194 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java index 9f5fe665283c7..cb8c21b433eb8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java @@ -142,11 +142,18 @@ private HoodieSparkLanceWriter(StoragePath file, /** * For every field carrying a Hudi VECTOR logical type annotation * (Spark metadata key {@link HoodieSchema#TYPE_METADATA_FIELD} starting with {@code "VECTOR"}), - * attach the lance-spark metadata key {@link LanceArrowUtils#ARROW_FIXED_SIZE_LIST_SIZE_KEY()} + * auto-attach the lance-spark metadata key {@link LanceArrowUtils#ARROW_FIXED_SIZE_LIST_SIZE_KEY()} * with the vector's dimension so that {@link LanceArrowUtils#toArrowSchema} emits a native * Arrow {@code FixedSizeList} (Lance's vector column encoding) and * {@link LanceArrowWriter} selects its fixed-size-list field writer when serializing values. * + *

Why we auto-attach this metadata: lance-spark requires the + * {@code arrow.fixed-size-list.size} Spark field metadata key to be present as a + * {@code Long} for it to emit a fixed-size list Arrow type. Rather than making + * users manually attach that key alongside the Hudi VECTOR descriptor, the writer + * derives it from the VECTOR dimension so a Hudi table declaring a VECTOR column + * "just works" on the Lance write path. + * *

Currently only FLOAT and DOUBLE element vectors are supported on Lance, matching * lance-spark's {@code VectorUtils.shouldBeFixedSizeList}. Other element types would * silently fall through to a plain list write, so we fail fast instead. diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index c6f9b7b20b967..a9e68557e7131 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -274,9 +274,13 @@ public static void convertRowVectorColumns(InternalRow row, GenericInternalRow r * Re-attaches {@link HoodieSchema#TYPE_METADATA_FIELD} to Spark fields that were * Arrow {@code FixedSizeList} in the Lance file. * {@code LanceArrowUtils.fromArrowSchema} drops all field metadata, so without this - * step VECTOR columns are indistinguishable from plain arrays. + * step VECTOR columns written by Hudi are indistinguishable from plain arrays. * - *

Only top-level FLOAT/DOUBLE fields are inspected; nested structs are not recursed. + *

Only fields whose names appear in the {@link HoodieSchema#VECTOR_COLUMNS_METADATA_KEY} + * footer value are considered. This avoids false-positive annotation of Arrow + * {@code FixedSizeList} columns in externally-written Lance files that + * weren't intended as Hudi VECTORs. Only top-level fields are inspected; nested + * structs are not recursed. * * @param arrowSchema original Arrow schema from the Lance file * @param convertedSpark Spark schema from {@code LanceArrowUtils.fromArrowSchema} @@ -288,12 +292,20 @@ public static StructType restoreVectorMetadataFromArrowSchema( if (arrowSchema == null || convertedSpark == null) { return convertedSpark; } + java.util.Set vectorFieldNames = HoodieSchema.parseVectorColumnNames( + arrowSchema.getCustomMetadata().get(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY)); + if (vectorFieldNames.isEmpty()) { + return convertedSpark; + } List arrowFields = arrowSchema.getFields(); StructField[] sparkFields = convertedSpark.fields(); StructField[] newFields = new StructField[sparkFields.length]; - boolean anyChanged = false; for (int i = 0; i < sparkFields.length; i++) { StructField sf = sparkFields[i]; + if (!vectorFieldNames.contains(sf.name())) { + newFields[i] = sf; + continue; + } String descriptor = deriveVectorDescriptor(arrowFields.get(i)); if (descriptor == null) { newFields[i] = sf; @@ -306,10 +318,9 @@ public static StructType restoreVectorMetadataFromArrowSchema( .withMetadata(sf.metadata()) .putString(HoodieSchema.TYPE_METADATA_FIELD, descriptor) .build()); - anyChanged = true; } } - return anyChanged ? new StructType(newFields) : convertedSpark; + return new StructType(newFields); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java index 62671b9d76d9f..cf45c2c339142 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java @@ -265,6 +265,44 @@ public static String buildVectorColumnsMetadataValue(HoodieSchema schema) { return serializeVectorColumnsMetadata(vectorColumns); } + /** + * Parses the comma-separated {@link #VECTOR_COLUMNS_METADATA_KEY} footer value and + * returns the set of vector column field names. Commas inside parentheses (e.g. inside + * the VECTOR descriptor {@code VECTOR(128, DOUBLE)}) are not treated as separators. + * + * @param footerValue raw value from the file footer, or null / empty + * @return set of field names (preserves insertion order), or empty set if input is null / empty + */ + public static Set parseVectorColumnNames(String footerValue) { + if (footerValue == null || footerValue.isEmpty()) { + return Collections.emptySet(); + } + java.util.LinkedHashSet names = new java.util.LinkedHashSet<>(); + int depth = 0; + int start = 0; + for (int i = 0; i < footerValue.length(); i++) { + char c = footerValue.charAt(i); + if (c == '(') { + depth++; + } else if (c == ')') { + depth--; + } else if (c == ',' && depth == 0) { + addVectorColumnName(footerValue, start, i, names); + start = i + 1; + } + } + addVectorColumnName(footerValue, start, footerValue.length(), names); + return names; + } + + private static void addVectorColumnName(String s, int start, int end, Set names) { + int colon = s.indexOf(':', start); + if (colon > start && colon < end) { + names.add(s.substring(start, colon).trim()); + } + } + + private Schema avroSchema; private HoodieSchemaType type; private transient List fields; diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index 8bfb57aededda..d910dac6fee3a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -789,84 +789,16 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { fsView.close() } - @ParameterizedTest - @EnumSource(value = classOf[HoodieTableType]) - def testFloatVectorRoundTrip(tableType: HoodieTableType): Unit = { - val tableName = s"test_lance_vec_f32_${tableType.name().toLowerCase}" - val tablePath = s"$basePath/$tableName" - - val dim = 4 - val schema = StructType(Seq( - StructField("id", IntegerType, nullable = false), - StructField("embedding", - ArrayType(FloatType, containsNull = false), - nullable = false, - vectorMetadata(s"VECTOR($dim)")) - )) - val data = Seq( - Row(1, Array(1.0f, 2.0f, 4.0f, 8.0f)), - Row(2, Array(0.5f, 0.25f, 0.125f, 0.0625f)), - Row(3, Array(-1.0f, -2.0f, -4.0f, -8.0f)) - ) - val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) - - writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite) - - val readDf = spark.read.format("hudi").load(tablePath).select("id", "embedding") - assertEquals( - ArrayType(FloatType, containsNull = false), - readDf.schema("embedding").dataType) - assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim)") - - val actualByKey = readDf.collect().map(r => - r.getInt(0) -> r.getSeq[Float](1).toSeq).toMap - val expectedByKey = data.map(r => - r.getInt(0) -> r.getAs[Array[Float]](1).toSeq).toMap - assertEquals(expectedByKey, actualByKey) - - assertLanceFieldIsFixedSizeList(tablePath, "embedding", dim) - assertLanceFooterHasVectorColumns(tablePath, s"embedding:VECTOR($dim)") - } - - @ParameterizedTest - @EnumSource(value = classOf[HoodieTableType]) - def testDoubleVectorRoundTrip(tableType: HoodieTableType): Unit = { - val tableName = s"test_lance_vec_f64_${tableType.name().toLowerCase}" - val tablePath = s"$basePath/$tableName" - - val dim = 4 - val schema = StructType(Seq( - StructField("id", IntegerType, nullable = false), - StructField("embedding", - ArrayType(DoubleType, containsNull = false), - nullable = false, - vectorMetadata(s"VECTOR($dim, DOUBLE)")) - )) - val data = Seq( - Row(1, Array(1.0d, 2.0d, 4.0d, 8.0d)), - Row(2, Array(0.5d, 0.25d, 0.125d, 0.0625d)), - Row(3, Array(-1.0d, -2.0d, -4.0d, -8.0d)) - ) - val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) - - writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite) - - val readDf = spark.read.format("hudi").load(tablePath).select("id", "embedding") - assertEquals( - ArrayType(DoubleType, containsNull = false), - readDf.schema("embedding").dataType) - assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim, DOUBLE)") - - val actualByKey = readDf.collect().map(r => - r.getInt(0) -> r.getSeq[Double](1).toSeq).toMap - val expectedByKey = data.map(r => - r.getInt(0) -> r.getAs[Array[Double]](1).toSeq).toMap - assertEquals(expectedByKey, actualByKey) - - assertLanceFieldIsFixedSizeList(tablePath, "embedding", dim) - assertLanceFooterHasVectorColumns(tablePath, s"embedding:VECTOR($dim, DOUBLE)") - } - + /** + * Consolidated vector round-trip test. Covers, parameterized over COW + MOR, + * what used to be separate float-only / double-only / MOR-update-path tests: + * - two non-null VECTOR columns of different element types (FLOAT, DOUBLE) and dims + * - insert then upsert exercises the MOR log-merge path on MOR (COW rewrite on COW) + * + * Nullable-vector coverage lives in {@code testNullableVectorRoundTrip} because + * merging a null-valued vector through the upsert path currently errors out in + * the Lance reader; that is tracked as a separate follow-up. + */ @ParameterizedTest @EnumSource(value = classOf[HoodieTableType]) def testMultipleVectorColumns(tableType: HoodieTableType): Unit = { @@ -877,6 +809,8 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { val featuresDim = 2 val schema = StructType(Seq( StructField("id", IntegerType, nullable = false), + StructField("name", StringType, nullable = false), + StructField("age", IntegerType, nullable = false), StructField("embedding", ArrayType(FloatType, containsNull = false), nullable = false, @@ -886,33 +820,60 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { nullable = false, vectorMetadata(s"VECTOR($featuresDim, DOUBLE)")) )) - val data = Seq( - Row(1, Array(1.0f, 2.0f, 3.0f), Array(10.0d, 20.0d)), - Row(2, Array(4.0f, 5.0f, 6.0f), Array(30.0d, 40.0d)) + + // Initial insert. + val data1 = Seq( + Row(1, "Alice", 30, Array(1.0f, 2.0f, 3.0f), Array(10.0d, 20.0d)), + Row(2, "Bob", 25, Array(4.0f, 5.0f, 6.0f), Array(30.0d, 40.0d)), + Row(3, "Charlie", 35, Array(7.0f, 8.0f, 9.0f), Array(50.0d, 60.0d)) ) - val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(data1), schema).coalesce(1) + writeDataframe(tableType, tableName, tablePath, df1, + saveMode = SaveMode.Overwrite, operation = Some("insert")) - writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite) + // Upsert — update Bob's embedding, age, and features. + val data2 = Seq( + Row(2, "Bob", 40, Array(10.0f, 20.0f, 30.0f), Array(70.0d, 80.0d)) + ) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(data2), schema).coalesce(1) + writeDataframe(tableType, tableName, tablePath, df2, operation = Some("upsert")) - val readDf = spark.read.format("hudi").load(tablePath).select("id", "embedding", "features") - val rows = readDf.collect().map { r => - (r.getInt(0), r.getSeq[Float](1).toSeq, r.getSeq[Double](2).toSeq) - }.toSet - val expected = data.map { r => - (r.getInt(0), - r.getAs[Array[Float]](1).toSeq, - r.getAs[Array[Double]](2).toSeq) - }.toSet - assertEquals(expected, rows) + val readDf = spark.read.format("hudi").load(tablePath) + .select("id", "name", "age", "embedding", "features") + assertEquals( + ArrayType(FloatType, containsNull = false), + readDf.schema("embedding").dataType) + assertEquals( + ArrayType(DoubleType, containsNull = false), + readDf.schema("features").dataType) assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($embeddingDim)") assertHudiTypeMetadata(readDf.schema("features"), s"VECTOR($featuresDim, DOUBLE)") + val rows = readDf.collect().sortBy(_.getInt(0)) + assertEquals(3, rows.length) + // Alice unchanged. + assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getSeq[Float](3).toSeq) + assertEquals(Seq(10.0d, 20.0d), rows(0).getSeq[Double](4).toSeq) + // Bob upserted. + assertEquals(40, rows(1).getInt(2), "Bob's age should be updated to 40") + assertEquals(Seq(10.0f, 20.0f, 30.0f), rows(1).getSeq[Float](3).toSeq) + assertEquals(Seq(70.0d, 80.0d), rows(1).getSeq[Double](4).toSeq) + // Charlie unchanged. + assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getSeq[Float](3).toSeq) + assertEquals(Seq(50.0d, 60.0d), rows(2).getSeq[Double](4).toSeq) + + // Validate Lance file physical schema + footer on the base files produced. assertLanceFieldIsFixedSizeList(tablePath, "embedding", embeddingDim) assertLanceFieldIsFixedSizeList(tablePath, "features", featuresDim) assertLanceFooterHasVectorColumns(tablePath, s"embedding:VECTOR($embeddingDim),features:VECTOR($featuresDim, DOUBLE)") } + /** + * Nullable-vector coverage — kept separate from {@code testMultipleVectorColumns} + * because folding this case into the upsert/merge path errors out in the current + * Lance reader (null vector element read). + */ @ParameterizedTest @EnumSource(value = classOf[HoodieTableType]) def testNullableVectorRoundTrip(tableType: HoodieTableType): Unit = { @@ -946,51 +907,6 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getSeq[Float](1).toSeq) } - @ParameterizedTest - @EnumSource(value = classOf[HoodieTableType]) - def testVectorMorUpdatePath(tableType: HoodieTableType): Unit = { - val tableName = s"test_lance_vec_update_${tableType.name().toLowerCase}" - val tablePath = s"$basePath/$tableName" - - val dim = 3 - val schema = StructType(Seq( - StructField("id", IntegerType, nullable = false), - StructField("name", StringType, nullable = false), - StructField("age", IntegerType, nullable = false), - StructField("embedding", - ArrayType(FloatType, containsNull = false), - nullable = false, - vectorMetadata(s"VECTOR($dim)")) - )) - - // Initial insert - val data1 = Seq( - Row(1, "Alice", 30, Array(1.0f, 2.0f, 3.0f)), - Row(2, "Bob", 25, Array(4.0f, 5.0f, 6.0f)), - Row(3, "Charlie", 35, Array(7.0f, 8.0f, 9.0f)) - ) - val df1 = spark.createDataFrame(spark.sparkContext.parallelize(data1), schema).coalesce(1) - writeDataframe(tableType, tableName, tablePath, df1, saveMode = SaveMode.Overwrite, operation = Some("insert")) - - // Upsert — update Bob's embedding - val data2 = Seq( - Row(2, "Bob", 40, Array(10.0f, 20.0f, 30.0f)) - ) - val df2 = spark.createDataFrame(spark.sparkContext.parallelize(data2), schema).coalesce(1) - writeDataframe(tableType, tableName, tablePath, df2, operation = Some("upsert")) - - // Read merged result - val readDf = spark.read.format("hudi").load(tablePath).select("id", "name", "age", "embedding") - assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim)") - - val rows = readDf.collect().sortBy(_.getInt(0)) - assertEquals(3, rows.length) - assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getSeq[Float](3).toSeq) - assertEquals(Seq(10.0f, 20.0f, 30.0f), rows(1).getSeq[Float](3).toSeq) - assertEquals(40, rows(1).getInt(2), "Bob's age should be updated to 40") - assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getSeq[Float](3).toSeq) - } - @ParameterizedTest @EnumSource(value = classOf[HoodieTableType]) def testVectorProjection(tableType: HoodieTableType): Unit = { @@ -1031,53 +947,6 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { assertEquals(Seq(5.0f, 6.0f, 7.0f, 8.0f), metaRows("2")) } - @ParameterizedTest - @EnumSource(value = classOf[HoodieTableType]) - def testVectorPartitionedTable(tableType: HoodieTableType): Unit = { - val tableName = s"test_lance_vec_part_${tableType.name().toLowerCase}" - val tablePath = s"$basePath/$tableName" - - val dim = 3 - val schema = StructType(Seq( - StructField("id", IntegerType, nullable = false), - StructField("department", StringType, nullable = false), - StructField("age", IntegerType, nullable = false), - StructField("embedding", - ArrayType(FloatType, containsNull = false), - nullable = false, - vectorMetadata(s"VECTOR($dim)")) - )) - val data = Seq( - Row(1, "engineering", 30, Array(1.0f, 2.0f, 3.0f)), - Row(2, "sales", 25, Array(4.0f, 5.0f, 6.0f)), - Row(3, "engineering", 35, Array(7.0f, 8.0f, 9.0f)) - ) - val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema).coalesce(1) - writeDataframe(tableType, tableName, tablePath, df, saveMode = SaveMode.Overwrite, - extraOptions = Map(PARTITIONPATH_FIELD.key() -> "department")) - - // Read back all partitions - val readDf = spark.read.format("hudi").load(tablePath).select("id", "department", "embedding") - assertHudiTypeMetadata(readDf.schema("embedding"), s"VECTOR($dim)") - - val rows = readDf.collect().sortBy(_.getInt(0)) - assertEquals(3, rows.length) - assertEquals(Seq(1.0f, 2.0f, 3.0f), rows(0).getSeq[Float](2).toSeq) - assertEquals("engineering", rows(0).getString(1)) - assertEquals(Seq(4.0f, 5.0f, 6.0f), rows(1).getSeq[Float](2).toSeq) - assertEquals("sales", rows(1).getString(1)) - assertEquals(Seq(7.0f, 8.0f, 9.0f), rows(2).getSeq[Float](2).toSeq) - assertEquals("engineering", rows(2).getString(1)) - - // Read a single partition - val engDf = spark.read.format("hudi").load(tablePath) - .filter("department = 'engineering'").select("id", "embedding") - val engRows = engDf.collect().sortBy(_.getInt(0)) - assertEquals(2, engRows.length) - assertEquals(Seq(1.0f, 2.0f, 3.0f), engRows(0).getSeq[Float](1).toSeq) - assertEquals(Seq(7.0f, 8.0f, 9.0f), engRows(1).getSeq[Float](1).toSeq) - } - private def assertHudiTypeMetadata(field: StructField, expectedDescriptor: String): Unit = { assertTrue(field.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD), s"Expected field ${field.name} to carry ${HoodieSchema.TYPE_METADATA_FIELD} metadata after read") @@ -1089,7 +958,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { new MetadataBuilder().putString(HoodieSchema.TYPE_METADATA_FIELD, descriptor).build() /** Runs `check` against each Lance base file's Arrow schema. */ - private def forEachLanceSchema(tablePath: String)(check: (org.apache.arrow.vector.types.pojo.Schema, String) => Unit): Unit = { + private def validateLanceFileSchema(tablePath: String)(check: (org.apache.arrow.vector.types.pojo.Schema, String) => Unit): Unit = { val metaClient = HoodieTableMetaClient.builder() .setConf(HoodieTestUtils.getDefaultStorageConf) .setBasePath(tablePath) @@ -1125,7 +994,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { } private def assertLanceFooterHasVectorColumns(tablePath: String, expected: String): Unit = { - forEachLanceSchema(tablePath) { (schema, path) => + validateLanceFileSchema(tablePath) { (schema, path) => val meta = schema.getCustomMetadata assertNotNull(meta, s"Lance footer metadata null for $path") val key = HoodieSchema.VECTOR_COLUMNS_METADATA_KEY @@ -1136,7 +1005,7 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { } private def assertLanceFieldIsFixedSizeList(tablePath: String, fieldName: String, expectedDim: Int): Unit = { - forEachLanceSchema(tablePath) { (schema, path) => + validateLanceFileSchema(tablePath) { (schema, path) => val field = schema.findField(fieldName) assertNotNull(field, s"Field $fieldName not found in Lance schema for $path") field.getType match { From 92d75af907d19b48e9cd4808cc41e2d7b4ff3217 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 22 Apr 2026 19:51:48 -0700 Subject: [PATCH 12/15] address ethan comments --- .../io/storage/HoodieSparkLanceReader.java | 4 +- .../io/storage/HoodieSparkLanceWriter.java | 9 +- .../io/storage/VectorConversionUtils.java | 86 ++++++++----------- ...rkFileFormatInternalRowReaderContext.scala | 6 +- .../lance/SparkLanceReaderBase.scala | 12 ++- .../storage/TestHoodieSparkLanceReader.java | 37 ++++++++ 6 files changed, 86 insertions(+), 68 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java index 2c0822233452f..48730ecb6a3db 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java @@ -202,8 +202,8 @@ private ClosableIterator getUnsafeRowIterator(HoodieSchema requestedS @Override public HoodieSchema getSchema() { try { - StructType structType = VectorConversionUtils.restoreVectorMetadataFromArrowSchema( - arrowSchema, LanceArrowUtils.fromArrowSchema(arrowSchema)); + StructType structType = VectorConversionUtils.restoreVectorMetadata( + LanceArrowUtils.fromArrowSchema(arrowSchema)); return HoodieSchemaConversionUtils.convertStructTypeToHoodieSchema(structType, "record", "", false); } catch (Exception e) { throw new HoodieException("Failed to read schema from Lance file: " + path, e); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java index cb8c21b433eb8..8f064b35674e8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java @@ -147,12 +147,9 @@ private HoodieSparkLanceWriter(StoragePath file, * Arrow {@code FixedSizeList} (Lance's vector column encoding) and * {@link LanceArrowWriter} selects its fixed-size-list field writer when serializing values. * - *

Why we auto-attach this metadata: lance-spark requires the - * {@code arrow.fixed-size-list.size} Spark field metadata key to be present as a - * {@code Long} for it to emit a fixed-size list Arrow type. Rather than making - * users manually attach that key alongside the Hudi VECTOR descriptor, the writer - * derives it from the VECTOR dimension so a Hudi table declaring a VECTOR column - * "just works" on the Lance write path. + *

Lance-spark keys vector columns off the {@code .arrow.fixed-size-list.size} + * property (see Lance Spark CREATE TABLE docs); we derive it from the VECTOR dimension so + * users don't have to set it alongside the Hudi descriptor. * *

Currently only FLOAT and DOUBLE element vectors are supported on Lance, matching * lance-spark's {@code VectorUtils.shouldBeFixedSizeList}. Other element types would diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index a9e68557e7131..a2469eedbd20f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -18,21 +18,21 @@ package org.apache.hudi.io.storage; -import org.apache.arrow.vector.types.pojo.Schema; import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.schema.HoodieSchemaType; -import org.apache.arrow.vector.types.FloatingPointPrecision; -import org.apache.arrow.vector.types.pojo.ArrowType; -import org.apache.arrow.vector.types.pojo.Field; import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.types.ArrayType; import org.apache.spark.sql.types.BinaryType$; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.LanceArrowUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; @@ -271,85 +271,71 @@ public static void convertRowVectorColumns(InternalRow row, GenericInternalRow r } /** - * Re-attaches {@link HoodieSchema#TYPE_METADATA_FIELD} to Spark fields that were + * Re-attaches {@link HoodieSchema#TYPE_METADATA_FIELD} to Spark fields that are * Arrow {@code FixedSizeList} in the Lance file. - * {@code LanceArrowUtils.fromArrowSchema} drops all field metadata, so without this - * step VECTOR columns written by Hudi are indistinguishable from plain arrays. - * - *

Only fields whose names appear in the {@link HoodieSchema#VECTOR_COLUMNS_METADATA_KEY} - * footer value are considered. This avoids false-positive annotation of Arrow - * {@code FixedSizeList} columns in externally-written Lance files that - * weren't intended as Hudi VECTORs. Only top-level fields are inspected; nested - * structs are not recursed. - * - * @param arrowSchema original Arrow schema from the Lance file - * @param convertedSpark Spark schema from {@code LanceArrowUtils.fromArrowSchema} - * (same field count and order as {@code arrowSchema}) - * @return StructType with VECTOR metadata restored on eligible fields + * {@code LanceArrowUtils.fromArrowSchema} strips Hudi's VECTOR descriptor during + * Arrow→Spark conversion but preserves the fixed-size-list dimension under the + * lance-spark metadata key {@link LanceArrowUtils#ARROW_FIXED_SIZE_LIST_SIZE_KEY()}. + * This method reads that key together with the Spark {@link ArrayType} element type + * to rebuild the VECTOR descriptor. The Lance footer is not consulted on read — + * the Hudi table schema / Arrow type carry enough information on their own. + * Nested structs are not recursed. */ - public static StructType restoreVectorMetadataFromArrowSchema( - Schema arrowSchema, StructType convertedSpark) { - if (arrowSchema == null || convertedSpark == null) { - return convertedSpark; - } - java.util.Set vectorFieldNames = HoodieSchema.parseVectorColumnNames( - arrowSchema.getCustomMetadata().get(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY)); - if (vectorFieldNames.isEmpty()) { - return convertedSpark; + public static StructType restoreVectorMetadata(StructType convertedSpark) { + if (convertedSpark == null) { + return null; } - List arrowFields = arrowSchema.getFields(); StructField[] sparkFields = convertedSpark.fields(); StructField[] newFields = new StructField[sparkFields.length]; + boolean changed = false; for (int i = 0; i < sparkFields.length; i++) { StructField sf = sparkFields[i]; - if (!vectorFieldNames.contains(sf.name())) { - newFields[i] = sf; - continue; - } - String descriptor = deriveVectorDescriptor(arrowFields.get(i)); + String descriptor = deriveVectorDescriptor(sf); if (descriptor == null) { newFields[i] = sf; } else { + // VECTOR contract: elements are non-nullable. lance-spark's Arrow→Spark + // conversion produces ArrayType(containsNull=true); force containsNull=false + // so the field round-trips through HoodieSchema conversion. + DataType arrayType = DataTypes.createArrayType( + ((ArrayType) sf.dataType()).elementType(), false); newFields[i] = new StructField( sf.name(), - sf.dataType(), + arrayType, sf.nullable(), new MetadataBuilder() .withMetadata(sf.metadata()) .putString(HoodieSchema.TYPE_METADATA_FIELD, descriptor) .build()); + changed = true; } } - return new StructType(newFields); + return changed ? new StructType(newFields) : convertedSpark; } /** - * Derives Hudi's VECTOR type descriptor for an Arrow field if it represents a - * fixed-size list of single- or double-precision floats, otherwise returns null. + * Derives Hudi's VECTOR type descriptor for a Spark field if lance-spark tagged it + * with {@link LanceArrowUtils#ARROW_FIXED_SIZE_LIST_SIZE_KEY()} and its data type is + * {@code ArrayType(Float|Double, containsNull=false)}; otherwise returns null. */ - private static String deriveVectorDescriptor(Field arrowField) { - ArrowType type = arrowField.getType(); - if (!(type instanceof ArrowType.FixedSizeList)) { - return null; - } - int dim = ((ArrowType.FixedSizeList) type).getListSize(); - List children = arrowField.getChildren(); - if (children.size() != 1) { + private static String deriveVectorDescriptor(StructField sf) { + String sizeKey = LanceArrowUtils.ARROW_FIXED_SIZE_LIST_SIZE_KEY(); + if (!sf.metadata().contains(sizeKey)) { return null; } - ArrowType childType = children.get(0).getType(); - if (!(childType instanceof ArrowType.FloatingPoint)) { + if (!(sf.dataType() instanceof ArrayType)) { return null; } - FloatingPointPrecision precision = ((ArrowType.FloatingPoint) childType).getPrecision(); + DataType elemType = ((ArrayType) sf.dataType()).elementType(); HoodieSchema.Vector.VectorElementType elementType; - if (precision == FloatingPointPrecision.SINGLE) { + if (DataTypes.FloatType.equals(elemType)) { elementType = HoodieSchema.Vector.VectorElementType.FLOAT; - } else if (precision == FloatingPointPrecision.DOUBLE) { + } else if (DataTypes.DoubleType.equals(elemType)) { elementType = HoodieSchema.Vector.VectorElementType.DOUBLE; } else { return null; } + int dim = (int) sf.metadata().getLong(sizeKey); return HoodieSchema.createVector(dim, elementType).toTypeDescriptor(); } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 87e044e6d6790..92b963f683906 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -82,9 +82,9 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) // Parquet stores VECTOR as FIXED_LEN_BYTE_ARRAY, so the reader needs BinaryType - // and we decode back to ArrayType below. Lance returns ArrayType natively, so skip. - // Log files are always parquet regardless of the table's base-file format. - val isLanceBaseFile = !FSUtils.isLogFile(filePath) && + // and we decode back to ArrayType below. Lance returns ArrayType natively, so skip + // the rewrite only for Lance base files; log files always go through the rewrite path. + val isLanceBaseFile = FSUtils.isBaseFile(filePath) && tableConfig.getBaseFileFormat == HoodieFileFormat.LANCE val vectorColumnInfo: Map[Int, HoodieSchema.Vector] = if (isLanceBaseFile) { Map.empty diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala index 27d2b8d89b5d4..2375d73b9b192 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala @@ -90,13 +90,11 @@ class SparkLanceReaderBase(enableVectorizedReader: Boolean) extends SparkColumna // Open Lance file reader val lanceReader = LanceFileReader.open(filePath, allocator) - // Get schema from Lance file. lance-spark drops all Arrow field metadata during - // StructType conversion, so re-attach Hudi's VECTOR logical-type descriptor onto - // fields that are encoded as Arrow FixedSizeList. This mirrors - // the Parquet read path, where VECTOR metadata is restored from footer entries. - val arrowSchema = lanceReader.schema() - val fileSchema = VectorConversionUtils.restoreVectorMetadataFromArrowSchema( - arrowSchema, LanceArrowUtils.fromArrowSchema(arrowSchema)) + // Get schema from Lance file. lance-spark strips Hudi's VECTOR descriptor during + // Arrow→Spark conversion but keeps the fixed-size-list dimension on the Spark + // field metadata; rebuild the descriptor from that. + val fileSchema = VectorConversionUtils.restoreVectorMetadata( + LanceArrowUtils.fromArrowSchema(lanceReader.schema())) // Build type change info for schema evolution val (implicitTypeChangeInfo, sparkRequestSchema) = diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/storage/TestHoodieSparkLanceReader.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/storage/TestHoodieSparkLanceReader.java index b2593fc908162..4aef3d8a16b0e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/storage/TestHoodieSparkLanceReader.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/storage/TestHoodieSparkLanceReader.java @@ -27,6 +27,8 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -41,6 +43,9 @@ import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.MetadataBuilder; +import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -648,6 +653,38 @@ public void testReadWithRequestedSchema() throws Exception { } } + @Test + public void testGetSchemaRestoresVectorMetadata() throws Exception { + int dim = 4; + Metadata vectorFieldMetadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(" + dim + ")") + .build(); + StructType schema = new StructType() + .add(new StructField("id", DataTypes.IntegerType, false, Metadata.empty())) + .add(new StructField( + "embedding", + DataTypes.createArrayType(DataTypes.FloatType, false), + false, + vectorFieldMetadata)); + + List rows = new ArrayList<>(); + rows.add(createRow(1, new Object[] {1.0f, 2.0f, 3.0f, 4.0f})); + rows.add(createRow(2, new Object[] {5.0f, 6.0f, 7.0f, 8.0f})); + + StoragePath path = new StoragePath(tempDir.getAbsolutePath() + "/test_vector_schema.lance"); + try (HoodieSparkLanceReader reader = writeAndCreateReader(path, schema, rows)) { + HoodieSchema readSchema = reader.getSchema(); + HoodieSchemaField embedding = readSchema.getField("embedding") + .orElseThrow(() -> new AssertionError("embedding field missing on read schema")); + HoodieSchema fieldSchema = embedding.schema().getNonNullType(); + assertEquals(HoodieSchemaType.VECTOR, fieldSchema.getType(), + "embedding must be restored as VECTOR from the FixedSizeList encoding alone"); + HoodieSchema.Vector vec = (HoodieSchema.Vector) fieldSchema; + assertEquals(dim, vec.getDimension()); + assertEquals(HoodieSchema.Vector.VectorElementType.FLOAT, vec.getVectorElementType()); + } + } + private void assertBloomFilter(HoodieSparkLanceReader reader, Class clazz, String minKey, String maxKey, int keyCount) { BloomFilter bloomFilter = reader.readBloomFilter(); assertInstanceOf(clazz, bloomFilter); From e1d68255d0711b64d1f97e34a1da39e9a45fb8b3 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 22 Apr 2026 23:07:41 -0700 Subject: [PATCH 13/15] fix merge conflict and adress ethan --- .../io/storage/HoodieSparkLanceReader.java | 5 ++- .../io/storage/HoodieSparkLanceWriter.java | 12 +++--- .../io/storage/VectorConversionUtils.java | 38 ++++++++++--------- .../hudi/common/schema/HoodieSchema.java | 3 +- .../lance/SparkLanceReaderBase.scala | 10 ++++- .../hudi/functional/TestLanceDataSource.scala | 11 +++--- 6 files changed, 46 insertions(+), 33 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java index 48730ecb6a3db..905bd74026ad7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceReader.java @@ -202,8 +202,11 @@ private ClosableIterator getUnsafeRowIterator(HoodieSchema requestedS @Override public HoodieSchema getSchema() { try { + Map customMetadata = arrowSchema.getCustomMetadata(); + Set vectorColumnNames = HoodieSchema.parseVectorColumnNames( + customMetadata == null ? null : customMetadata.get(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY)); StructType structType = VectorConversionUtils.restoreVectorMetadata( - LanceArrowUtils.fromArrowSchema(arrowSchema)); + LanceArrowUtils.fromArrowSchema(arrowSchema), vectorColumnNames); return HoodieSchemaConversionUtils.convertStructTypeToHoodieSchema(structType, "record", "", false); } catch (Exception e) { throw new HoodieException("Failed to read schema from Lance file: " + path, e); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java index ff90d74d729b0..64cfb2322bcab 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkLanceWriter.java @@ -127,8 +127,8 @@ private HoodieSparkLanceWriter(StoragePath file, Option bloomFilterOpt, long maxFileSize) { super(file, DEFAULT_BATCH_SIZE, bloomFilterOpt.map(HoodieBloomFilterRowWriteSupport::new)); - this.sparkSchema = sparkSchema; - this.arrowSchema = LanceArrowUtils.toArrowSchema(sparkSchema, DEFAULT_TIMEZONE, true); + this.sparkSchema = enrichSparkSchemaForLanceVectors(sparkSchema); + this.arrowSchema = LanceArrowUtils.toArrowSchema(this.sparkSchema, DEFAULT_TIMEZONE, true); this.fileName = UTF8String.fromString(file.getName()); this.instantTime = UTF8String.fromString(instantTime); this.populateMetaFields = populateMetaFields; @@ -147,9 +147,11 @@ private HoodieSparkLanceWriter(StoragePath file, * Arrow {@code FixedSizeList} (Lance's vector column encoding) and * {@link LanceArrowWriter} selects its fixed-size-list field writer when serializing values. * - *

Lance-spark keys vector columns off the {@code .arrow.fixed-size-list.size} - * property (see Lance Spark CREATE TABLE docs); we derive it from the VECTOR dimension so - * users don't have to set it alongside the Hudi descriptor. + *

Lance-spark keys vector columns off the per-field + * {@link LanceArrowUtils#ARROW_FIXED_SIZE_LIST_SIZE_KEY()} (literal: + * {@code arrow.fixed-size-list.size}) metadata entry (see Lance Spark CREATE TABLE docs); + * we derive it from the VECTOR dimension so users don't have to set it alongside the + * Hudi descriptor. * *

Currently only FLOAT and DOUBLE element vectors are supported on Lance, matching * lance-spark's {@code VectorUtils.shouldBeFixedSizeList}. Other element types would diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index a2469eedbd20f..e332c21cc13ca 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -38,10 +38,10 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.Function; import static org.apache.hudi.common.util.ValidationUtils.checkArgument; @@ -67,7 +67,7 @@ private VectorConversionUtils() { * @return map from field index to Vector schema; empty map if schema is null or has no vectors */ public static Map detectVectorColumns(HoodieSchema schema) { - Map vectorColumnInfo = new HashMap<>(); + Map vectorColumnInfo = new LinkedHashMap<>(); if (schema == null) { return vectorColumnInfo; } @@ -94,17 +94,11 @@ public static String buildVectorColumnsFooterValue(StructType schema) { if (schema == null) { return ""; } - StructField[] fields = schema.fields(); Map detected = detectVectorColumnsFromMetadata(schema); - // Walk fields in ordinal order (not detected.entrySet()) so the footer value - // is emitted in stable, schema-ordered form. detectVectorColumnsFromMetadata - // returns a plain HashMap, whose iteration order is unspecified. + StructField[] fields = schema.fields(); LinkedHashMap named = new LinkedHashMap<>(); - for (int i = 0; i < fields.length; i++) { - HoodieSchema.Vector vec = detected.get(i); - if (vec != null) { - named.put(fields[i].name(), vec); - } + for (Map.Entry entry : detected.entrySet()) { + named.put(fields[entry.getKey()].name(), entry.getValue()); } return HoodieSchema.serializeVectorColumnsMetadata(named); } @@ -118,7 +112,8 @@ public static String buildVectorColumnsFooterValue(StructType schema) { * @return map from field index to Vector schema; empty map if no vectors found */ public static Map detectVectorColumnsFromMetadata(StructType schema) { - Map vectorColumnInfo = new HashMap<>(); + // Use LinkedHashMap so callers iterate in field-ordinal order (stable across JDKs). + Map vectorColumnInfo = new LinkedHashMap<>(); if (schema == null) { return vectorColumnInfo; } @@ -276,21 +271,28 @@ public static void convertRowVectorColumns(InternalRow row, GenericInternalRow r * {@code LanceArrowUtils.fromArrowSchema} strips Hudi's VECTOR descriptor during * Arrow→Spark conversion but preserves the fixed-size-list dimension under the * lance-spark metadata key {@link LanceArrowUtils#ARROW_FIXED_SIZE_LIST_SIZE_KEY()}. - * This method reads that key together with the Spark {@link ArrayType} element type - * to rebuild the VECTOR descriptor. The Lance footer is not consulted on read — - * the Hudi table schema / Arrow type carry enough information on their own. - * Nested structs are not recursed. + * + *

A FixedSizeList alone does not prove the column is a Hudi VECTOR — a + * non-Hudi Lance file could contain one. Callers must pass {@code vectorColumnNames} + * (the set parsed from the {@link HoodieSchema#VECTOR_COLUMNS_METADATA_KEY} footer + * entry) so that only fields known to be Hudi VECTORs are restored. Pass an empty + * set to skip the restore entirely. + * + *

Nested structs are not recursed. */ - public static StructType restoreVectorMetadata(StructType convertedSpark) { + public static StructType restoreVectorMetadata(StructType convertedSpark, Set vectorColumnNames) { if (convertedSpark == null) { return null; } + if (vectorColumnNames == null || vectorColumnNames.isEmpty()) { + return convertedSpark; + } StructField[] sparkFields = convertedSpark.fields(); StructField[] newFields = new StructField[sparkFields.length]; boolean changed = false; for (int i = 0; i < sparkFields.length; i++) { StructField sf = sparkFields[i]; - String descriptor = deriveVectorDescriptor(sf); + String descriptor = vectorColumnNames.contains(sf.name()) ? deriveVectorDescriptor(sf) : null; if (descriptor == null) { newFields[i] = sf; } else { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java index 52095f19bd15f..dff06a4c0e0f0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java @@ -43,6 +43,7 @@ import java.util.Collections; import java.util.EnumSet; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Locale; import java.util.Map; @@ -283,7 +284,7 @@ public static Set parseVectorColumnNames(String footerValue) { if (footerValue == null || footerValue.isEmpty()) { return Collections.emptySet(); } - java.util.LinkedHashSet names = new java.util.LinkedHashSet<>(); + LinkedHashSet names = new LinkedHashSet<>(); int depth = 0; int start = 0; for (int i = 0; i < footerValue.length(); i++) { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala index 2375d73b9b192..9ce3766d22d49 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.lance import org.apache.hudi.SparkAdapterSupport.sparkAdapter +import org.apache.hudi.common.schema.HoodieSchema import org.apache.hudi.common.util import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.io.memory.HoodieArrowAllocator @@ -92,9 +93,14 @@ class SparkLanceReaderBase(enableVectorizedReader: Boolean) extends SparkColumna // Get schema from Lance file. lance-spark strips Hudi's VECTOR descriptor during // Arrow→Spark conversion but keeps the fixed-size-list dimension on the Spark - // field metadata; rebuild the descriptor from that. + // field metadata; rebuild the descriptor from that, cross-referenced with the + // `hoodie.vector.columns` footer entry so non-Hudi fixed-size-lists aren't mis-tagged. + val arrowSchema = lanceReader.schema() + val customMetadata = arrowSchema.getCustomMetadata + val vectorColumnNames = HoodieSchema.parseVectorColumnNames( + if (customMetadata == null) null else customMetadata.get(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY)) val fileSchema = VectorConversionUtils.restoreVectorMetadata( - LanceArrowUtils.fromArrowSchema(lanceReader.schema())) + LanceArrowUtils.fromArrowSchema(arrowSchema), vectorColumnNames) // Build type change info for schema evolution val (implicitTypeChangeInfo, sparkRequestSchema) = diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index d910dac6fee3a..dc8ad8c76bb99 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -790,14 +790,13 @@ class TestLanceDataSource extends HoodieSparkClientTestBase { } /** - * Consolidated vector round-trip test. Covers, parameterized over COW + MOR, - * what used to be separate float-only / double-only / MOR-update-path tests: - * - two non-null VECTOR columns of different element types (FLOAT, DOUBLE) and dims - * - insert then upsert exercises the MOR log-merge path on MOR (COW rewrite on COW) + * Vector round-trip test parameterized over COW + MOR. Covers two non-null VECTOR + * columns of different element types (FLOAT, DOUBLE) and dimensions, and exercises + * the upsert path (MOR log-merge on MOR, file rewrite on COW). * - * Nullable-vector coverage lives in {@code testNullableVectorRoundTrip} because + *

Nullable-vector coverage lives in {@code testNullableVectorRoundTrip} because * merging a null-valued vector through the upsert path currently errors out in - * the Lance reader; that is tracked as a separate follow-up. + * the Lance reader; tracked as a separate follow-up. */ @ParameterizedTest @EnumSource(value = classOf[HoodieTableType]) From 01b67332205a509a4ebf4f7bd0b0ddd965ea13c1 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 22 Apr 2026 23:39:13 -0700 Subject: [PATCH 14/15] fix minor issue --- .../hudi/io/storage/VectorConversionUtils.java | 6 +++--- .../datasources/lance/SparkLanceReaderBase.scala | 15 +++++++++------ 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java index e332c21cc13ca..a8cc02f58dad3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -274,9 +274,9 @@ public static void convertRowVectorColumns(InternalRow row, GenericInternalRow r * *

A FixedSizeList alone does not prove the column is a Hudi VECTOR — a * non-Hudi Lance file could contain one. Callers must pass {@code vectorColumnNames} - * (the set parsed from the {@link HoodieSchema#VECTOR_COLUMNS_METADATA_KEY} footer - * entry) so that only fields known to be Hudi VECTORs are restored. Pass an empty - * set to skip the restore entirely. + * (derived from the Hudi schema's VECTOR-tagged fields, e.g. via + * {@link #detectVectorColumnsFromMetadata(StructType)}) so that only fields known to + * be Hudi VECTORs are restored. Pass an empty set to skip the restore entirely. * *

Nested structs are not recursed. */ diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala index 9ce3766d22d49..183eea6fc5baf 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/lance/SparkLanceReaderBase.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.lance import org.apache.hudi.SparkAdapterSupport.sparkAdapter -import org.apache.hudi.common.schema.HoodieSchema import org.apache.hudi.common.util import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.io.memory.HoodieArrowAllocator @@ -93,12 +92,16 @@ class SparkLanceReaderBase(enableVectorizedReader: Boolean) extends SparkColumna // Get schema from Lance file. lance-spark strips Hudi's VECTOR descriptor during // Arrow→Spark conversion but keeps the fixed-size-list dimension on the Spark - // field metadata; rebuild the descriptor from that, cross-referenced with the - // `hoodie.vector.columns` footer entry so non-Hudi fixed-size-lists aren't mis-tagged. + // field metadata; rebuild the descriptor from that, using requiredSchema + // as the source of truth for which columns are Hudi VECTORs — so non-Hudi fixed-size-lists aren't mis-tagged. val arrowSchema = lanceReader.schema() - val customMetadata = arrowSchema.getCustomMetadata - val vectorColumnNames = HoodieSchema.parseVectorColumnNames( - if (customMetadata == null) null else customMetadata.get(HoodieSchema.VECTOR_COLUMNS_METADATA_KEY)) + val vectorColumnNames: java.util.Set[String] = VectorConversionUtils + .detectVectorColumnsFromMetadata(requiredSchema) + .keySet() + .asScala + .map(i => requiredSchema.fields(i).name) + .toSet + .asJava val fileSchema = VectorConversionUtils.restoreVectorMetadata( LanceArrowUtils.fromArrowSchema(arrowSchema), vectorColumnNames) From a8f2c55d9c5dcd3a4bbdb602853de26edcddb2a7 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Thu, 23 Apr 2026 13:47:06 -0700 Subject: [PATCH 15/15] fix(lance): reorder imports in TestLanceDataSource to satisfy scalastyle Co-Authored-By: Claude Opus 4.7 --- .../scala/org/apache/hudi/functional/TestLanceDataSource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala index cdcf61a8c4ac5..01172d58706e9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLanceDataSource.scala @@ -31,9 +31,9 @@ import org.apache.hudi.io.storage.HoodieSparkLanceReader import org.apache.hudi.storage.StoragePath import org.apache.hudi.testutils.HoodieSparkClientTestBase -import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession} import org.apache.arrow.memory.RootAllocator import org.apache.arrow.vector.types.pojo.ArrowType +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession} import org.apache.spark.sql.types._ import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotNull, assertTrue}