From c46bfb96fa1c41131c71d3c445138735819b31cf Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Fri, 13 Feb 2026 09:03:48 -0800 Subject: [PATCH 01/13] futher changes for getting parquet to write hudi vectors --- .../io/storage/HoodieSparkParquetReader.java | 78 +++++++- .../row/HoodieRowParquetWriteSupport.java | 60 +++++- ...rkFileFormatInternalRowReaderContext.scala | 98 ++++++++- .../convert/InternalSchemaConverter.java | 1 + .../AvroSchemaConverterWithTimestampNTZ.java | 6 + .../functional/TestVectorDataSource.scala | 186 ++++++++++++++++++ 6 files changed, 421 insertions(+), 8 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestVectorDataSource.scala diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index 160a731cece41..fe87cb393f463 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -24,6 +24,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieSparkRecord; 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.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.FileFormatUtils; import org.apache.hudi.common.util.Option; @@ -47,8 +49,10 @@ import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.avro.HoodieSparkSchemaConverters; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.catalyst.util.RebaseDateTime; import org.apache.spark.sql.execution.datasources.parquet.HoodieParquetReadSupport; import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters; @@ -57,13 +61,19 @@ import org.apache.spark.sql.execution.datasources.parquet.SparkBasicSchemaEvolution; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.sources.Filter; +import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Set; import scala.Option$; @@ -142,10 +152,27 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSchema, List readFilters) throws IOException { HoodieSchema nonNullSchema = requestedSchema.getNonNullType(); StructType structSchema = HoodieInternalRowUtils.getCachedSchema(nonNullSchema); + + // Detect vector columns: ordinal → dimension + Map vectorColumnInfo = detectVectorColumns(nonNullSchema); + + // For vector columns, replace ArrayType(FloatType) with BinaryType in the read schema + // so SparkBasicSchemaEvolution sees matching types (file has FIXED_LEN_BYTE_ARRAY → BinaryType) + StructType readStructSchema = structSchema; + if (!vectorColumnInfo.isEmpty()) { + StructField[] fields = structSchema.fields().clone(); + for (Map.Entry entry : vectorColumnInfo.entrySet()) { + int idx = entry.getKey(); + StructField orig = fields[idx]; + fields[idx] = new StructField(orig.name(), DataTypes.BinaryType, orig.nullable(), Metadata.empty()); + } + readStructSchema = new StructType(fields); + } + Option messageSchema = Option.of(getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(nonNullSchema)); boolean enableTimestampFieldRepair = storage.getConf().getBoolean(ENABLE_LOGICAL_TIMESTAMP_REPAIR, true); StructType dataStructType = convertToStruct(enableTimestampFieldRepair ? SchemaRepair.repairLogicalTypes(getFileSchema(), messageSchema) : getFileSchema()); - SparkBasicSchemaEvolution evolution = new SparkBasicSchemaEvolution(dataStructType, structSchema, SQLConf.get().sessionLocalTimeZone()); + SparkBasicSchemaEvolution evolution = new SparkBasicSchemaEvolution(dataStructType, readStructSchema, SQLConf.get().sessionLocalTimeZone()); String readSchemaJson = evolution.getRequestSchema().json(); SQLConf sqlConf = SQLConf.get(); storage.getConf().set(ParquetReadSupport.PARQUET_READ_SCHEMA, readSchemaJson); @@ -184,10 +211,59 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc UnsafeProjection projection = evolution.generateUnsafeProjection(); ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader); CloseableMappingIterator projectedIterator = new CloseableMappingIterator<>(parquetReaderIterator, projection::apply); + + if (!vectorColumnInfo.isEmpty()) { + // Post-process: convert binary VECTOR columns back to float arrays + UnsafeProjection vectorProjection = UnsafeProjection.create(structSchema); + int numFields = readStructSchema.fields().length; + StructType finalReadSchema = readStructSchema; + CloseableMappingIterator vectorIterator = + new CloseableMappingIterator<>(projectedIterator, row -> { + GenericInternalRow converted = new GenericInternalRow(numFields); + for (int i = 0; i < numFields; i++) { + if (row.isNullAt(i)) { + converted.setNullAt(i); + } else if (vectorColumnInfo.containsKey(i)) { + byte[] bytes = row.getBinary(i); + int dim = vectorColumnInfo.get(i); + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + float[] floats = new float[dim]; + for (int j = 0; j < dim; j++) { + floats[j] = buffer.getFloat(); + } + converted.update(i, new GenericArrayData(floats)); + } else { + converted.update(i, row.get(i, finalReadSchema.apply(i).dataType())); + } + } + return vectorProjection.apply(converted); + }); + readerIterators.add(vectorIterator); + return vectorIterator; + } + readerIterators.add(projectedIterator); return projectedIterator; } + /** + * Detects vector columns in the schema and returns a map of ordinal to dimension. + */ + private static Map detectVectorColumns(HoodieSchema schema) { + Map vectorColumnInfo = new HashMap<>(); + if (schema == null) { + return vectorColumnInfo; + } + List fields = schema.getFields(); + for (int i = 0; i < fields.size(); i++) { + HoodieSchema fieldSchema = fields.get(i).schema().getNonNullType(); + if (fieldSchema.getType() == HoodieSchemaType.VECTOR) { + vectorColumnInfo.put(i, ((HoodieSchema.Vector) fieldSchema).getDimension()); + } + } + return vectorColumnInfo; + } + private MessageType getFileSchema() { if (fileSchemaOption.isEmpty()) { MessageType messageType = ((ParquetUtils) parquetUtils).readMessageType(storage, path); 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 9d0da7f534afa..655d5d6de2657 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 @@ -26,6 +26,7 @@ import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.schema.HoodieSchema.TimePrecision; +import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.Option; @@ -69,6 +70,8 @@ import org.apache.spark.unsafe.types.UTF8String; import org.apache.spark.util.VersionUtils; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -182,13 +185,43 @@ public void write(InternalRow row) { @Override public WriteSupport.FinalizedWriteContext finalizeWrite() { - Map extraMetadata = - bloomFilterWriteSupportOpt.map(HoodieBloomFilterWriteSupport::finalizeMetadata) - .orElse(Collections.emptyMap()); + Map extraMetadata = new HashMap<>(); + bloomFilterWriteSupportOpt.ifPresent(bf -> extraMetadata.putAll(bf.finalizeMetadata())); + + String vectorColumnsStr = buildVectorColumnsMetadata(schema); + if (!vectorColumnsStr.isEmpty()) { + extraMetadata.put("hoodie.vector.columns", vectorColumnsStr); + } return new WriteSupport.FinalizedWriteContext(extraMetadata); } + /** + * Scans the schema for VECTOR fields and builds metadata string. + * Format: "colName:dimension:elementType,..." e.g. "embedding:128:FLOAT" + */ + private static String buildVectorColumnsMetadata(HoodieSchema hoodieSchema) { + if (hoodieSchema == null || !hoodieSchema.hasFields()) { + return ""; + } + StringBuilder sb = new StringBuilder(); + for (HoodieSchemaField field : hoodieSchema.getFields()) { + HoodieSchema fieldSchema = field.schema().getNonNullType(); + if (fieldSchema.getType() == HoodieSchemaType.VECTOR) { + HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) fieldSchema; + if (sb.length() > 0) { + sb.append(","); + } + sb.append(field.name()) + .append(":") + .append(vectorSchema.getDimension()) + .append(":") + .append(vectorSchema.getVectorElementType().getDataType()); + } + } + return sb.toString(); + } + public void add(UTF8String recordKey) { this.bloomFilterWriteSupportOpt.ifPresent(bloomFilterWriteSupport -> bloomFilterWriteSupport.addKey(recordKey)); @@ -305,6 +338,19 @@ private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { } recordConsumer.addBinary(Binary.fromReusedByteArray(fixedLengthBytes, 0, numBytes)); }; + } else if (dataType instanceof ArrayType + && resolvedSchema != null + && resolvedSchema.getType() == HoodieSchemaType.VECTOR) { + HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) resolvedSchema; + int dimension = vectorSchema.getDimension(); + return (row, ordinal) -> { + ArrayData array = row.getArray(ordinal); + ByteBuffer buffer = ByteBuffer.allocate(dimension * 4).order(ByteOrder.LITTLE_ENDIAN); + for (int i = 0; i < dimension; i++) { + buffer.putFloat(array.getFloat(i)); + } + recordConsumer.addBinary(Binary.fromReusedByteArray(buffer.array())); + }; } else if (dataType instanceof ArrayType) { ValueWriter elementWriter = makeWriter(resolvedSchema == null ? null : resolvedSchema.getElementType(), ((ArrayType) dataType).elementType()); if (!writeLegacyListFormat) { @@ -518,6 +564,14 @@ private Type convertField(HoodieSchema fieldSchema, StructField structField, Typ .as(LogicalTypeAnnotation.decimalType(scale, precision)) .length(Decimal.minBytesForPrecision()[precision]) .named(structField.name()); + } else if (dataType instanceof ArrayType + && resolvedSchema != null + && resolvedSchema.getType() == HoodieSchemaType.VECTOR) { + HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) resolvedSchema; + int fixedSize = vectorSchema.getDimension() + * vectorSchema.getVectorElementType().getElementSize(); + return Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .length(fixedSize).named(structField.name()); } else if (dataType instanceof ArrayType) { ArrayType arrayType = (ArrayType) dataType; DataType elementType = arrayType.elementType(); 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 e4d97a2df6a29..9c9122a13a386 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 @@ -25,7 +25,7 @@ import org.apache.hudi.avro.{AvroSchemaUtils, HoodieAvroUtils} 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.schema.{HoodieSchema, HoodieSchemaUtils} +import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaType, HoodieSchemaUtils} import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.table.read.buffer.PositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME import org.apache.hudi.common.util.ValidationUtils.checkState @@ -41,7 +41,9 @@ import org.apache.spark.sql.execution.datasources.{PartitionedFile, SparkColumna import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{LongType, MetadataBuilder, StructField, StructType} +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.types.{BinaryType, LongType, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} import scala.collection.JavaConverters._ @@ -81,7 +83,17 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR assert(getRecordContext.supportsParquetRowIndex()) } val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) - val (readSchema, readFilters) = getSchemaAndFiltersForRead(structType, hasRowIndexField) + + // 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) + val parquetReadStructType = if (vectorColumnInfo.nonEmpty) { + SparkFileFormatInternalRowReaderContext.replaceVectorColumnsWithBinary(structType, vectorColumnInfo) + } else { + structType + } + + val (readSchema, readFilters) = getSchemaAndFiltersForRead(parquetReadStructType, hasRowIndexField) if (FSUtils.isLogFile(filePath)) { // NOTE: now only primary key based filtering is supported for log files new HoodieSparkFileReaderFactory(storage).newParquetFileReader(filePath) @@ -100,9 +112,16 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR } else { org.apache.hudi.common.util.Option.empty[org.apache.parquet.schema.MessageType]() } - new CloseableInternalRowIterator(baseFileReader.read(fileInfo, + val rawIterator = new CloseableInternalRowIterator(baseFileReader.read(fileInfo, readSchema, StructType(Seq.empty), getSchemaHandler.getInternalSchemaOpt, readFilters, storage.getConf.asInstanceOf[StorageConfiguration[Configuration]], tableSchemaOpt)) + + // Post-process: convert binary VECTOR columns back to float arrays + if (vectorColumnInfo.nonEmpty) { + SparkFileFormatInternalRowReaderContext.wrapWithVectorConversion(rawIterator, vectorColumnInfo, readSchema) + } else { + rawIterator + } } } @@ -290,4 +309,75 @@ object SparkFileFormatInternalRowReaderContext { field.name.equals(ROW_INDEX_TEMPORARY_COLUMN_NAME) } + /** + * Detects VECTOR columns from HoodieSchema. + * @return Map of ordinal to dimension for VECTOR fields. + */ + private[hudi] def detectVectorColumns(schema: HoodieSchema): Map[Int, Int] = { + if (schema == null) return Map.empty + import scala.collection.JavaConverters._ + schema.getFields.asScala.zipWithIndex.flatMap { case (field, idx) => + val fieldSchema = field.schema().getNonNullType + if (fieldSchema.getType == HoodieSchemaType.VECTOR) { + Some(idx -> fieldSchema.asInstanceOf[HoodieSchema.Vector].getDimension) + } else { + None + } + }.toMap + } + + /** + * Replaces ArrayType(FloatType) with BinaryType for VECTOR columns so the Parquet reader + * can read FIXED_LEN_BYTE_ARRAY data without type mismatch. + */ + private[hudi] def replaceVectorColumnsWithBinary(structType: StructType, vectorColumns: Map[Int, Int]): StructType = { + StructType(structType.fields.zipWithIndex.map { case (field, idx) => + if (vectorColumns.contains(idx)) { + StructField(field.name, BinaryType, field.nullable, org.apache.spark.sql.types.Metadata.empty) + } else { + field + } + }) + } + + /** + * Wraps an iterator to convert binary VECTOR columns back to ArrayType(FloatType). + * Unpacks little-endian float bytes from FIXED_LEN_BYTE_ARRAY into GenericArrayData. + */ + private[hudi] def wrapWithVectorConversion( + iterator: ClosableIterator[InternalRow], + vectorColumns: Map[Int, Int], + readSchema: StructType): ClosableIterator[InternalRow] = { + val numFields = readSchema.fields.length + new ClosableIterator[InternalRow] { + override def hasNext: Boolean = iterator.hasNext + override def next(): InternalRow = { + val row = iterator.next() + val result = new GenericInternalRow(numFields) + var i = 0 + while (i < numFields) { + if (row.isNullAt(i)) { + result.setNullAt(i) + } else if (vectorColumns.contains(i)) { + val bytes = row.getBinary(i) + val dim = vectorColumns(i) + val buffer = java.nio.ByteBuffer.wrap(bytes).order(java.nio.ByteOrder.LITTLE_ENDIAN) + val floats = new Array[Float](dim) + var j = 0 + while (j < dim) { + floats(j) = buffer.getFloat() + j += 1 + } + result.update(i, new GenericArrayData(floats)) + } else { + result.update(i, row.get(i, readSchema(i).dataType)) + } + i += 1 + } + result + } + override def close(): Unit = iterator.close() + } + } + } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java index 9783ecac1b5a5..a6e010f409762 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java @@ -313,6 +313,7 @@ private static Type visitPrimitiveToBuildInternalType(HoodieSchema schema) { case ENUM: return Types.StringType.get(); case FIXED: + case VECTOR: return Types.FixedType.getFixed(schema.getFixedSize()); case BYTES: return Types.BinaryType.get(); diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java index 4f8d88d0f6d91..7611202ed3c9c 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java @@ -248,6 +248,12 @@ private Type convertField(String fieldName, HoodieSchema schema, Type.Repetition builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition).length(schema.getFixedSize()); } break; + case VECTOR: + HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) schema; + int fixedSize = vectorSchema.getDimension() + * vectorSchema.getVectorElementType().getElementSize(); + builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition).length(fixedSize); + break; case UNION: return convertUnion(fieldName, schema, repetition, schemaPath); default: 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 new file mode 100644 index 0000000000000..4f0cd734a109f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestVectorDataSource.scala @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.functional + +import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaType} +import org.apache.hudi.testutils.HoodieSparkClientTestBase + +import org.apache.spark.sql.{Row, SaveMode, SparkSession} +import org.apache.spark.sql.types._ +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import org.junit.jupiter.api.Assertions._ + +import java.util.Collections + +/** + * End-to-end tests for vector column support in Hudi. + * Tests round-trip data correctness through Spark DataFrames. + */ +class TestVectorDataSource extends HoodieSparkClientTestBase { + + var spark: SparkSession = null + + @BeforeEach override def setUp(): Unit = { + initPath() + initSparkContexts() + spark = sqlContext.sparkSession + initTestDataGenerator() + initHoodieStorage() + } + + @AfterEach override def tearDown(): Unit = { + cleanupSparkContexts() + cleanupTestDataGenerator() + cleanupFileSystem() + } + + @Test + def testVectorRoundTrip(): Unit = { + // 1. Create schema with vector metadata + val typeMetadataStr = HoodieSchema.buildTypeMetadata( + Collections.singletonMap("vector.dimension", "128")) + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchemaType.VECTOR.name()) + .putString(HoodieSchema.TYPE_METADATA_PROPS_FIELD, typeMetadataStr) + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, metadata), + StructField("label", StringType, nullable = true) + )) + + // 2. Generate test data (128-dim float vectors) + val random = new scala.util.Random(42) + val data = (0 until 100).map { i => + val embedding = Array.fill(128)(random.nextFloat()) + Row(s"key_$i", embedding.toSeq, s"label_$i") + } + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + // 3. Write as COW Hudi table + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "vector_test_table") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath) + + // 4. Read back + val readDf = spark.read.format("hudi").load(basePath) + + // 5. Verify row count + assertEquals(100, readDf.count()) + + // 6. Verify schema preserved + val embeddingField = readDf.schema("embedding") + assertTrue(embeddingField.dataType.isInstanceOf[ArrayType]) + val arrayType = embeddingField.dataType.asInstanceOf[ArrayType] + assertEquals(FloatType, arrayType.elementType) + assertFalse(arrayType.containsNull) + + // 7. Verify vector metadata preserved + val readMetadata = embeddingField.metadata + assertTrue(readMetadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) + assertEquals("VECTOR", readMetadata.getString(HoodieSchema.TYPE_METADATA_FIELD)) + + // 8. Verify float values match exactly + val originalRows = df.select("id", "embedding").collect() + .map(r => (r.getString(0), r.getSeq[Float](1))) + .toMap + + val readRows = readDf.select("id", "embedding").collect() + .map(r => (r.getString(0), r.getSeq[Float](1))) + .toMap + + originalRows.foreach { case (id, origEmbedding) => + val readEmbedding = readRows(id) + assertEquals(128, readEmbedding.size, s"Vector size mismatch for $id") + + origEmbedding.zip(readEmbedding).zipWithIndex.foreach { + case ((orig, read), idx) => + assertEquals(orig, read, 1e-9f, + s"Vector mismatch at $id index $idx: orig=$orig read=$read") + } + } + } + + + @Test + def testNullableVectorField(): Unit = { + // Vector column itself nullable (entire array can be null) + val typeMetadataStr = HoodieSchema.buildTypeMetadata( + Collections.singletonMap("vector.dimension", "32")) + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchemaType.VECTOR.name()) + .putString(HoodieSchema.TYPE_METADATA_PROPS_FIELD, typeMetadataStr) + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = true, metadata) // nullable = true + )) + + val data = Seq( + Row("key_1", Array.fill(32)(0.5f).toSeq), + Row("key_2", null), // null vector + Row("key_3", Array.fill(32)(1.0f).toSeq) + ) + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "nullable_vector_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/nullable") + + val readDf = spark.read.format("hudi").load(basePath + "/nullable") + val readRows = readDf.select("id", "embedding").collect() + + // Verify null handling + val key2Row = readRows.find(_.getString(0) == "key_2").get + assertTrue(key2Row.isNullAt(1), "Null vector not preserved") + + // Verify non-null vectors preserved correctly + val key1Row = readRows.find(_.getString(0) == "key_1").get + assertFalse(key1Row.isNullAt(1)) + val key1Embedding = key1Row.getSeq[Float](1) + assertEquals(32, key1Embedding.size) + assertTrue(key1Embedding.forall(_ == 0.5f)) + + val key3Row = readRows.find(_.getString(0) == "key_3").get + assertFalse(key3Row.isNullAt(1)) + val key3Embedding = key3Row.getSeq[Float](1) + assertEquals(32, key3Embedding.size) + assertTrue(key3Embedding.forall(_ == 1.0f)) + } +} From 5b266075b6808d2530423aab6a7cbcf584740a75 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Fri, 13 Feb 2026 10:46:25 -0800 Subject: [PATCH 02/13] trying to resolve type issues during read/write --- .../org/apache/hudi/internal/schema/Type.java | 3 +- .../apache/hudi/internal/schema/Types.java | 61 +++++++++++++++++++ .../convert/InternalSchemaConverter.java | 14 ++++- 3 files changed, 76 insertions(+), 2 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java index da67b952258e6..13fcba74b3550 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java @@ -73,7 +73,8 @@ enum TypeID { TIME_MILLIS(Integer.class), TIMESTAMP_MILLIS(Long.class), LOCAL_TIMESTAMP_MILLIS(Long.class), - LOCAL_TIMESTAMP_MICROS(Long.class); + LOCAL_TIMESTAMP_MICROS(Long.class), + VECTOR(ByteBuffer.class); private final String name; private final Class classTag; diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java index 86ec29da9c459..f4f4abb71adbe 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java @@ -298,6 +298,67 @@ public int hashCode() { } } + /** + * Vector type that preserves dimension, element type, and storage backing + * through InternalSchema round-trips. + */ + public static class VectorType extends PrimitiveType { + private final int dimension; + private final String elementType; + private final String storageBacking; + + public static VectorType get(int dimension, String elementType, String storageBacking) { + return new VectorType(dimension, elementType, storageBacking); + } + + private VectorType(int dimension, String elementType, String storageBacking) { + this.dimension = dimension; + this.elementType = elementType; + this.storageBacking = storageBacking; + } + + public int getDimension() { + return dimension; + } + + public String getElementType() { + return elementType; + } + + public String getStorageBacking() { + return storageBacking; + } + + @Override + public TypeID typeId() { + return TypeID.VECTOR; + } + + @Override + public String toString() { + return String.format("vector[%d, %s, %s]", dimension, elementType, storageBacking); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof VectorType)) { + return false; + } + + VectorType that = (VectorType) o; + return dimension == that.dimension + && Objects.equals(elementType, that.elementType) + && Objects.equals(storageBacking, that.storageBacking); + } + + @Override + public int hashCode() { + return Objects.hash(VectorType.class, dimension, elementType, storageBacking); + } + } + public abstract static class DecimalBase extends PrimitiveType { protected final int scale; diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java index a6e010f409762..cb7d3f4957837 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java @@ -313,8 +313,13 @@ private static Type visitPrimitiveToBuildInternalType(HoodieSchema schema) { case ENUM: return Types.StringType.get(); case FIXED: - case VECTOR: return Types.FixedType.getFixed(schema.getFixedSize()); + case VECTOR: + HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) schema; + return Types.VectorType.get( + vectorSchema.getDimension(), + vectorSchema.getVectorElementType().getDataType(), + vectorSchema.getStorageBacking()); case BYTES: return Types.BinaryType.get(); case UUID: @@ -539,6 +544,13 @@ private static HoodieSchema visitInternalPrimitiveToBuildHoodiePrimitiveType(Typ return HoodieSchema.createFixed(name, null, null, fixed.getFixedSize()); } + case VECTOR: { + Types.VectorType vector = (Types.VectorType) primitive; + return HoodieSchema.createVector( + vector.getDimension(), + HoodieSchema.Vector.VectorElementType.fromString(vector.getElementType())); + } + case DECIMAL: case DECIMAL_FIXED: { Types.DecimalTypeFixed decimal = (Types.DecimalTypeFixed) primitive; From d8a7dca520972d497107ab7915bc933700a289cc Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Tue, 24 Feb 2026 15:58:48 -0800 Subject: [PATCH 03/13] finally able to write from spark to hudi to parquet vectors and then read them back --- .../row/HoodieRowParquetWriteSupport.java | 1 - .../convert/InternalSchemaConverter.java | 2 +- ...HoodieFileGroupReaderBasedFileFormat.scala | 129 +++++++++++++++++- .../functional/TestVectorDataSource.scala | 17 +-- 4 files changed, 129 insertions(+), 20 deletions(-) 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 655d5d6de2657..ae187aea3396c 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 @@ -73,7 +73,6 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.Map; diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java index cb7d3f4957837..9eb3f3ce31946 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java @@ -319,7 +319,7 @@ private static Type visitPrimitiveToBuildInternalType(HoodieSchema schema) { return Types.VectorType.get( vectorSchema.getDimension(), vectorSchema.getVectorElementType().getDataType(), - vectorSchema.getStorageBacking()); + vectorSchema.getStorageBacking().getBacking()); case BYTES: return Types.BinaryType.get(); case UUID: 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 8d83c9f4223a9..e7ba328d87625 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 @@ -41,24 +41,27 @@ import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.Job +import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaType} import org.apache.parquet.schema.{HoodieSchemaRepair, MessageType} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.util.GenericArrayData import org.apache.spark.sql.execution.datasources.{OutputWriterFactory, PartitionedFile, SparkColumnarFileReader} import org.apache.spark.sql.execution.datasources.orc.OrcUtils import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector} import org.apache.spark.sql.hudi.MultipleColumnarFileFormatReader import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{BinaryType, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnarBatchUtils} import org.apache.spark.util.SerializableConfiguration import java.io.Closeable +import java.nio.{ByteBuffer, ByteOrder} import scala.collection.JavaConverters.mapAsJavaMapConverter @@ -128,6 +131,13 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, * */ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { + // Vector columns are stored as FIXED_LEN_BYTE_ARRAY in Parquet but read as ArrayType in Spark. + // The binary→array conversion requires row-level access, so disable vectorized batch reading. + if (detectVectorColumns(schema).nonEmpty) { + supportVectorizedRead = false + supportReturningBatch = false + return false + } val conf = sparkSession.sessionState.conf val parquetBatchSupported = ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && supportBatchWithTableSchema val orcBatchSupported = conf.orcVectorizedReaderEnabled && @@ -398,21 +408,112 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, } } + /** + * Detects vector columns in a StructType by checking for VECTOR metadata. + * Returns a map of field index → (dimension, elementType). + */ + private def detectVectorColumns(schema: StructType): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = { + schema.fields.zipWithIndex.flatMap { case (field, idx) => + if (field.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) { + val typeStr = field.metadata.getString(HoodieSchema.TYPE_METADATA_FIELD) + if (typeStr.startsWith("VECTOR")) { + val typeDescriptor = HoodieSchema.parseTypeString(typeStr) + if (typeDescriptor.getType == HoodieSchemaType.VECTOR) { + val dimension = typeDescriptor.getParam(0).toInt + val elementType = if (typeDescriptor.getParams.size() > 1) + HoodieSchema.Vector.VectorElementType.fromString(typeDescriptor.getParam(1)) + else HoodieSchema.Vector.VectorElementType.FLOAT + Some(idx -> (dimension, elementType)) + } else None + } else None + } else None + }.toMap + } + + /** + * Replaces vector ArrayType fields with BinaryType so the Parquet reader sees + * a type matching the file's FIXED_LEN_BYTE_ARRAY. + */ + private def replaceVectorFieldsWithBinary(schema: StructType, vectorCols: Map[Int, _]): StructType = { + StructType(schema.fields.zipWithIndex.map { case (field, idx) => + if (vectorCols.contains(idx)) { + StructField(field.name, BinaryType, field.nullable) + } else field + }) + } + + /** + * Wraps an iterator to convert binary VECTOR columns back to typed arrays. + * The read schema has BinaryType for vector columns; the target schema has ArrayType. + */ + private def wrapWithVectorConversion(iter: Iterator[InternalRow], + readSchema: StructType, + targetSchema: StructType, + vectorCols: Map[Int, (Int, HoodieSchema.Vector.VectorElementType)]): Iterator[InternalRow] = { + val numFields = readSchema.fields.length + val vectorProjection = UnsafeProjection.create(targetSchema) + iter.map { row => + val converted = new GenericInternalRow(numFields) + for (i <- 0 until numFields) { + if (row.isNullAt(i)) { + converted.setNullAt(i) + } else if (vectorCols.contains(i)) { + val (dim, elemType) = vectorCols(i) + val bytes = row.getBinary(i) + val buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN) + elemType match { + case HoodieSchema.Vector.VectorElementType.FLOAT => + val arr = new Array[Float](dim) + var j = 0 + while (j < dim) { arr(j) = buffer.getFloat(); j += 1 } + converted.update(i, new GenericArrayData(arr)) + case HoodieSchema.Vector.VectorElementType.DOUBLE => + val arr = new Array[Double](dim) + var j = 0 + while (j < dim) { arr(j) = buffer.getDouble(); j += 1 } + converted.update(i, new GenericArrayData(arr)) + case HoodieSchema.Vector.VectorElementType.INT8 => + val arr = new Array[Byte](dim) + buffer.get(arr) + converted.update(i, new GenericArrayData(arr)) + } + } else { + converted.update(i, row.get(i, readSchema.apply(i).dataType)) + } + } + vectorProjection.apply(converted).asInstanceOf[InternalRow] + } + } + // executor private def readBaseFile(file: PartitionedFile, parquetFileReader: SparkColumnarFileReader, requestedSchema: StructType, remainingPartitionSchema: StructType, fixedPartitionIndexes: Set[Int], requiredSchema: StructType, partitionSchema: StructType, outputSchema: StructType, filters: Seq[Filter], storageConf: StorageConfiguration[Configuration]): Iterator[InternalRow] = { - if (remainingPartitionSchema.fields.length == partitionSchema.fields.length) { + // Detect vector columns in requiredSchema and create modified schemas with BinaryType + val vectorCols = detectVectorColumns(requiredSchema) + val hasVectors = vectorCols.nonEmpty + + val modifiedRequiredSchema = if (hasVectors) replaceVectorFieldsWithBinary(requiredSchema, vectorCols) else requiredSchema + val modifiedOutputSchema = if (hasVectors) replaceVectorFieldsWithBinary(outputSchema, vectorCols) else outputSchema + val modifiedRequestedSchema = if (hasVectors) { + val requestedVectorCols = detectVectorColumns(requestedSchema) + replaceVectorFieldsWithBinary(requestedSchema, requestedVectorCols) + } else requestedSchema + + // Detect vector columns in the full output schema for post-read conversion + val outputVectorCols = if (hasVectors) detectVectorColumns(outputSchema) else Map.empty[Int, (Int, HoodieSchema.Vector.VectorElementType)] + + val rawIter = if (remainingPartitionSchema.fields.length == partitionSchema.fields.length) { //none of partition fields are read from the file, so the reader will do the appending for us - parquetFileReader.read(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) + parquetFileReader.read(file, modifiedRequiredSchema, partitionSchema, internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) } else if (remainingPartitionSchema.fields.length == 0) { //we read all of the partition fields from the file val pfileUtils = sparkAdapter.getSparkPartitionedFileUtils //we need to modify the partitioned file so that the partition values are empty val modifiedFile = pfileUtils.createPartitionedFile(InternalRow.empty, pfileUtils.getPathFromPartitionedFile(file), file.start, file.length) //and we pass an empty schema for the partition schema - parquetFileReader.read(modifiedFile, outputSchema, new StructType(), internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) + parquetFileReader.read(modifiedFile, modifiedOutputSchema, new StructType(), internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) } else { //need to do an additional projection here. The case in mind is that partition schema is "a,b,c" mandatoryFields is "a,c", //then we will read (dataSchema + a + c) and append b. So the final schema will be (data schema + a + c +b) @@ -420,8 +521,22 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val pfileUtils = sparkAdapter.getSparkPartitionedFileUtils val partitionValues = getFixedPartitionValues(file.partitionValues, partitionSchema, fixedPartitionIndexes) val modifiedFile = pfileUtils.createPartitionedFile(partitionValues, pfileUtils.getPathFromPartitionedFile(file), file.start, file.length) - val iter = parquetFileReader.read(modifiedFile, requestedSchema, remainingPartitionSchema, internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) - projectIter(iter, StructType(requestedSchema.fields ++ remainingPartitionSchema.fields), outputSchema) + val iter = parquetFileReader.read(modifiedFile, modifiedRequestedSchema, remainingPartitionSchema, internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) + projectIter(iter, StructType(modifiedRequestedSchema.fields ++ remainingPartitionSchema.fields), modifiedOutputSchema) + } + + if (hasVectors) { + // The raw iterator has BinaryType for vector columns; convert back to ArrayType + val readSchema = if (remainingPartitionSchema.fields.length == partitionSchema.fields.length) { + StructType(modifiedRequiredSchema.fields ++ partitionSchema.fields) + } else if (remainingPartitionSchema.fields.length == 0) { + modifiedOutputSchema + } else { + modifiedOutputSchema + } + wrapWithVectorConversion(rawIter, readSchema, outputSchema, outputVectorCols) + } else { + rawIter } } 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 4f0cd734a109f..ab4d5cb22a526 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 @@ -26,8 +26,6 @@ import org.apache.spark.sql.types._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.Assertions._ -import java.util.Collections - /** * End-to-end tests for vector column support in Hudi. * Tests round-trip data correctness through Spark DataFrames. @@ -53,11 +51,8 @@ class TestVectorDataSource extends HoodieSparkClientTestBase { @Test def testVectorRoundTrip(): Unit = { // 1. Create schema with vector metadata - val typeMetadataStr = HoodieSchema.buildTypeMetadata( - Collections.singletonMap("vector.dimension", "128")) val metadata = new MetadataBuilder() - .putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchemaType.VECTOR.name()) - .putString(HoodieSchema.TYPE_METADATA_PROPS_FIELD, typeMetadataStr) + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(128)") .build() val schema = StructType(Seq( @@ -104,7 +99,10 @@ class TestVectorDataSource extends HoodieSparkClientTestBase { // 7. Verify vector metadata preserved val readMetadata = embeddingField.metadata assertTrue(readMetadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) - assertEquals("VECTOR", readMetadata.getString(HoodieSchema.TYPE_METADATA_FIELD)) + val typeDescriptor = HoodieSchema.parseTypeString( + readMetadata.getString(HoodieSchema.TYPE_METADATA_FIELD)) + assertEquals(HoodieSchemaType.VECTOR, typeDescriptor.getType) + assertEquals("128", typeDescriptor.getParam(0)) // 8. Verify float values match exactly val originalRows = df.select("id", "embedding").collect() @@ -131,11 +129,8 @@ class TestVectorDataSource extends HoodieSparkClientTestBase { @Test def testNullableVectorField(): Unit = { // Vector column itself nullable (entire array can be null) - val typeMetadataStr = HoodieSchema.buildTypeMetadata( - Collections.singletonMap("vector.dimension", "32")) val metadata = new MetadataBuilder() - .putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchemaType.VECTOR.name()) - .putString(HoodieSchema.TYPE_METADATA_PROPS_FIELD, typeMetadataStr) + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(32)") .build() val schema = StructType(Seq( From aa3be52fa63bdf312969b5b6113b889f012b5238 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Fri, 6 Mar 2026 20:48:51 -0800 Subject: [PATCH 04/13] get things working --- .../io/storage/HoodieSparkParquetReader.java | 61 ++++++++++++----- .../row/HoodieRowParquetWriteSupport.java | 38 ++--------- ...rkFileFormatInternalRowReaderContext.scala | 43 ++++++++---- .../convert/InternalSchemaConverter.java | 4 +- ...HoodieFileGroupReaderBasedFileFormat.scala | 68 +++++++++---------- .../functional/TestVectorDataSource.scala | 7 +- 6 files changed, 115 insertions(+), 106 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index fe87cb393f463..9f0f9886f6202 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -76,6 +76,8 @@ import java.util.Map; import java.util.Set; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; + import scala.Option$; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; @@ -153,16 +155,15 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc HoodieSchema nonNullSchema = requestedSchema.getNonNullType(); StructType structSchema = HoodieInternalRowUtils.getCachedSchema(nonNullSchema); - // Detect vector columns: ordinal → dimension - Map vectorColumnInfo = detectVectorColumns(nonNullSchema); + // Detect vector columns: ordinal → (dimension, elementType) + Map vectorColumnInfo = detectVectorColumns(nonNullSchema); // For vector columns, replace ArrayType(FloatType) with BinaryType in the read schema // so SparkBasicSchemaEvolution sees matching types (file has FIXED_LEN_BYTE_ARRAY → BinaryType) StructType readStructSchema = structSchema; if (!vectorColumnInfo.isEmpty()) { StructField[] fields = structSchema.fields().clone(); - for (Map.Entry entry : vectorColumnInfo.entrySet()) { - int idx = entry.getKey(); + for (int idx : vectorColumnInfo.keySet()) { StructField orig = fields[idx]; fields[idx] = new StructField(orig.name(), DataTypes.BinaryType, orig.nullable(), Metadata.empty()); } @@ -213,7 +214,7 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc CloseableMappingIterator projectedIterator = new CloseableMappingIterator<>(parquetReaderIterator, projection::apply); if (!vectorColumnInfo.isEmpty()) { - // Post-process: convert binary VECTOR columns back to float arrays + // Post-process: convert binary VECTOR columns back to typed arrays UnsafeProjection vectorProjection = UnsafeProjection.create(structSchema); int numFields = readStructSchema.fields().length; StructType finalReadSchema = readStructSchema; @@ -224,14 +225,7 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc if (row.isNullAt(i)) { converted.setNullAt(i); } else if (vectorColumnInfo.containsKey(i)) { - byte[] bytes = row.getBinary(i); - int dim = vectorColumnInfo.get(i); - ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); - float[] floats = new float[dim]; - for (int j = 0; j < dim; j++) { - floats[j] = buffer.getFloat(); - } - converted.update(i, new GenericArrayData(floats)); + converted.update(i, convertBinaryToVectorArray(row.getBinary(i), vectorColumnInfo.get(i))); } else { converted.update(i, row.get(i, finalReadSchema.apply(i).dataType())); } @@ -247,10 +241,10 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc } /** - * Detects vector columns in the schema and returns a map of ordinal to dimension. + * Detects vector columns in the schema and returns a map of ordinal to Vector schema. */ - private static Map detectVectorColumns(HoodieSchema schema) { - Map vectorColumnInfo = new HashMap<>(); + private static Map detectVectorColumns(HoodieSchema schema) { + Map vectorColumnInfo = new HashMap<>(); if (schema == null) { return vectorColumnInfo; } @@ -258,12 +252,45 @@ private static Map detectVectorColumns(HoodieSchema schema) { for (int i = 0; i < fields.size(); i++) { HoodieSchema fieldSchema = fields.get(i).schema().getNonNullType(); if (fieldSchema.getType() == HoodieSchemaType.VECTOR) { - vectorColumnInfo.put(i, ((HoodieSchema.Vector) fieldSchema).getDimension()); + vectorColumnInfo.put(i, (HoodieSchema.Vector) fieldSchema); } } return vectorColumnInfo; } + /** + * Converts binary bytes from a FIXED_LEN_BYTE_ARRAY parquet column back to a typed array + * based on the vector's element type and dimension. + */ + private static GenericArrayData convertBinaryToVectorArray(byte[] bytes, HoodieSchema.Vector vectorSchema) { + int dim = vectorSchema.getDimension(); + int expectedSize = dim * vectorSchema.getVectorElementType().getElementSize(); + checkArgument(bytes.length == expectedSize, + "Vector byte array length mismatch: expected " + expectedSize + " but got " + bytes.length); + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + switch (vectorSchema.getVectorElementType()) { + case FLOAT: + float[] floats = new float[dim]; + for (int j = 0; j < dim; j++) { + floats[j] = buffer.getFloat(); + } + return new GenericArrayData(floats); + case DOUBLE: + double[] doubles = new double[dim]; + for (int j = 0; j < dim; j++) { + doubles[j] = buffer.getDouble(); + } + return new GenericArrayData(doubles); + case INT8: + byte[] int8s = new byte[dim]; + buffer.get(int8s); + return new GenericArrayData(int8s); + default: + throw new UnsupportedOperationException( + "Unsupported vector element type: " + vectorSchema.getVectorElementType()); + } + } + private MessageType getFileSchema() { if (fileSchemaOption.isEmpty()) { MessageType messageType = ((ParquetUtils) parquetUtils).readMessageType(storage, path); 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 ae187aea3396c..fdd31fb2d7974 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 @@ -26,7 +26,6 @@ import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.schema.HoodieSchema.TimePrecision; -import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.Option; @@ -73,6 +72,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -184,43 +184,13 @@ public void write(InternalRow row) { @Override public WriteSupport.FinalizedWriteContext finalizeWrite() { - Map extraMetadata = new HashMap<>(); - bloomFilterWriteSupportOpt.ifPresent(bf -> extraMetadata.putAll(bf.finalizeMetadata())); - - String vectorColumnsStr = buildVectorColumnsMetadata(schema); - if (!vectorColumnsStr.isEmpty()) { - extraMetadata.put("hoodie.vector.columns", vectorColumnsStr); - } + Map extraMetadata = + bloomFilterWriteSupportOpt.map(HoodieBloomFilterWriteSupport::finalizeMetadata) + .orElse(Collections.emptyMap()); return new WriteSupport.FinalizedWriteContext(extraMetadata); } - /** - * Scans the schema for VECTOR fields and builds metadata string. - * Format: "colName:dimension:elementType,..." e.g. "embedding:128:FLOAT" - */ - private static String buildVectorColumnsMetadata(HoodieSchema hoodieSchema) { - if (hoodieSchema == null || !hoodieSchema.hasFields()) { - return ""; - } - StringBuilder sb = new StringBuilder(); - for (HoodieSchemaField field : hoodieSchema.getFields()) { - HoodieSchema fieldSchema = field.schema().getNonNullType(); - if (fieldSchema.getType() == HoodieSchemaType.VECTOR) { - HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) fieldSchema; - if (sb.length() > 0) { - sb.append(","); - } - sb.append(field.name()) - .append(":") - .append(vectorSchema.getDimension()) - .append(":") - .append(vectorSchema.getVectorElementType().getDataType()); - } - } - return sb.toString(); - } - public void add(UTF8String recordKey) { this.bloomFilterWriteSupportOpt.ifPresent(bloomFilterWriteSupport -> bloomFilterWriteSupport.addKey(recordKey)); 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 9c9122a13a386..74a9cf3be86be 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 @@ -311,15 +311,16 @@ object SparkFileFormatInternalRowReaderContext { /** * Detects VECTOR columns from HoodieSchema. - * @return Map of ordinal to dimension for VECTOR fields. + * @return Map of ordinal to (dimension, elementType) for VECTOR fields. */ - private[hudi] def detectVectorColumns(schema: HoodieSchema): Map[Int, Int] = { + private[hudi] def detectVectorColumns(schema: HoodieSchema): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = { if (schema == null) return Map.empty import scala.collection.JavaConverters._ schema.getFields.asScala.zipWithIndex.flatMap { case (field, idx) => val fieldSchema = field.schema().getNonNullType if (fieldSchema.getType == HoodieSchemaType.VECTOR) { - Some(idx -> fieldSchema.asInstanceOf[HoodieSchema.Vector].getDimension) + val v = fieldSchema.asInstanceOf[HoodieSchema.Vector] + Some(idx -> (v.getDimension, v.getVectorElementType)) } else { None } @@ -327,10 +328,10 @@ object SparkFileFormatInternalRowReaderContext { } /** - * Replaces ArrayType(FloatType) with BinaryType for VECTOR columns so the Parquet reader + * Replaces ArrayType with BinaryType for VECTOR columns so the Parquet reader * can read FIXED_LEN_BYTE_ARRAY data without type mismatch. */ - private[hudi] def replaceVectorColumnsWithBinary(structType: StructType, vectorColumns: Map[Int, Int]): StructType = { + private[hudi] def replaceVectorColumnsWithBinary(structType: StructType, vectorColumns: Map[Int, _]): StructType = { StructType(structType.fields.zipWithIndex.map { case (field, idx) => if (vectorColumns.contains(idx)) { StructField(field.name, BinaryType, field.nullable, org.apache.spark.sql.types.Metadata.empty) @@ -341,12 +342,12 @@ object SparkFileFormatInternalRowReaderContext { } /** - * Wraps an iterator to convert binary VECTOR columns back to ArrayType(FloatType). - * Unpacks little-endian float bytes from FIXED_LEN_BYTE_ARRAY into GenericArrayData. + * Wraps an iterator to convert binary VECTOR columns back to typed arrays. + * Unpacks little-endian bytes from FIXED_LEN_BYTE_ARRAY into GenericArrayData. */ private[hudi] def wrapWithVectorConversion( iterator: ClosableIterator[InternalRow], - vectorColumns: Map[Int, Int], + vectorColumns: Map[Int, (Int, HoodieSchema.Vector.VectorElementType)], readSchema: StructType): ClosableIterator[InternalRow] = { val numFields = readSchema.fields.length new ClosableIterator[InternalRow] { @@ -359,16 +360,28 @@ object SparkFileFormatInternalRowReaderContext { if (row.isNullAt(i)) { result.setNullAt(i) } else if (vectorColumns.contains(i)) { + val (dim, elemType) = vectorColumns(i) val bytes = row.getBinary(i) - val dim = vectorColumns(i) + val expectedSize = dim * elemType.getElementSize + require(bytes.length == expectedSize, + s"Vector byte array length mismatch: expected $expectedSize but got ${bytes.length}") val buffer = java.nio.ByteBuffer.wrap(bytes).order(java.nio.ByteOrder.LITTLE_ENDIAN) - val floats = new Array[Float](dim) - var j = 0 - while (j < dim) { - floats(j) = buffer.getFloat() - j += 1 + elemType match { + case HoodieSchema.Vector.VectorElementType.FLOAT => + val arr = new Array[Float](dim) + var j = 0 + while (j < dim) { arr(j) = buffer.getFloat(); j += 1 } + result.update(i, new GenericArrayData(arr)) + case HoodieSchema.Vector.VectorElementType.DOUBLE => + val arr = new Array[Double](dim) + var j = 0 + while (j < dim) { arr(j) = buffer.getDouble(); j += 1 } + result.update(i, new GenericArrayData(arr)) + case HoodieSchema.Vector.VectorElementType.INT8 => + val arr = new Array[Byte](dim) + buffer.get(arr) + result.update(i, new GenericArrayData(arr)) } - result.update(i, new GenericArrayData(floats)) } else { result.update(i, row.get(i, readSchema(i).dataType)) } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java index 9eb3f3ce31946..0432dbeb45793 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java @@ -318,8 +318,8 @@ private static Type visitPrimitiveToBuildInternalType(HoodieSchema schema) { HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) schema; return Types.VectorType.get( vectorSchema.getDimension(), - vectorSchema.getVectorElementType().getDataType(), - vectorSchema.getStorageBacking().getBacking()); + vectorSchema.getVectorElementType().name(), + vectorSchema.getStorageBacking().name()); case BYTES: return Types.BinaryType.get(); case UUID: 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 e7ba328d87625..333c93cd98e41 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 @@ -18,13 +18,13 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.hudi.{HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieSchemaConversionUtils, HoodieSparkUtils, HoodieTableSchema, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} -import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.cdc.{CDCFileGroupIterator, HoodieCDCFileGroupSplit, HoodieCDCFileIndex} import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.config.{HoodieMemoryConfig, TypedProperties} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieFileFormat +import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaType} import org.apache.hudi.common.schema.HoodieSchemaUtils import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, ParquetTableSchemaResolver} import org.apache.hudi.common.table.read.HoodieFileGroupReader @@ -41,7 +41,6 @@ import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.Job -import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaType} import org.apache.parquet.schema.{HoodieSchemaRepair, MessageType} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging @@ -136,32 +135,33 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, if (detectVectorColumns(schema).nonEmpty) { supportVectorizedRead = false supportReturningBatch = false - return false - } - val conf = sparkSession.sessionState.conf - val parquetBatchSupported = ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && supportBatchWithTableSchema - val orcBatchSupported = conf.orcVectorizedReaderEnabled && - schema.forall(s => OrcUtils.supportColumnarReads( - s.dataType, sparkSession.sessionState.conf.orcVectorizedReaderNestedColumnEnabled)) - // TODO: Implement columnar batch reading https://github.com/apache/hudi/issues/17736 - val lanceBatchSupported = false - - val supportBatch = if (isMultipleBaseFileFormatsEnabled) { - parquetBatchSupported && orcBatchSupported - } else if (hoodieFileFormat == HoodieFileFormat.PARQUET) { - parquetBatchSupported - } else if (hoodieFileFormat == HoodieFileFormat.ORC) { - orcBatchSupported - } else if (hoodieFileFormat == HoodieFileFormat.LANCE) { - lanceBatchSupported + false } else { - throw new HoodieNotSupportedException("Unsupported file format: " + hoodieFileFormat) + val conf = sparkSession.sessionState.conf + val parquetBatchSupported = ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && supportBatchWithTableSchema + val orcBatchSupported = conf.orcVectorizedReaderEnabled && + schema.forall(s => OrcUtils.supportColumnarReads( + s.dataType, sparkSession.sessionState.conf.orcVectorizedReaderNestedColumnEnabled)) + // TODO: Implement columnar batch reading https://github.com/apache/hudi/issues/17736 + val lanceBatchSupported = false + + val supportBatch = if (isMultipleBaseFileFormatsEnabled) { + parquetBatchSupported && orcBatchSupported + } else if (hoodieFileFormat == HoodieFileFormat.PARQUET) { + parquetBatchSupported + } else if (hoodieFileFormat == HoodieFileFormat.ORC) { + orcBatchSupported + } else if (hoodieFileFormat == HoodieFileFormat.LANCE) { + lanceBatchSupported + } else { + throw new HoodieNotSupportedException("Unsupported file format: " + hoodieFileFormat) + } + supportVectorizedRead = !isIncremental && !isBootstrap && supportBatch + supportReturningBatch = !isMOR && supportVectorizedRead + logInfo(s"supportReturningBatch: $supportReturningBatch, supportVectorizedRead: $supportVectorizedRead, isIncremental: $isIncremental, " + + s"isBootstrap: $isBootstrap, superSupportBatch: $supportBatch") + supportReturningBatch } - supportVectorizedRead = !isIncremental && !isBootstrap && supportBatch - supportReturningBatch = !isMOR && supportVectorizedRead - logInfo(s"supportReturningBatch: $supportReturningBatch, supportVectorizedRead: $supportVectorizedRead, isIncremental: $isIncremental, " + - s"isBootstrap: $isBootstrap, superSupportBatch: $supportBatch") - supportReturningBatch } //for partition columns that we read from the file, we don't want them to be constant column vectors so we @@ -417,13 +417,10 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, if (field.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) { val typeStr = field.metadata.getString(HoodieSchema.TYPE_METADATA_FIELD) if (typeStr.startsWith("VECTOR")) { - val typeDescriptor = HoodieSchema.parseTypeString(typeStr) - if (typeDescriptor.getType == HoodieSchemaType.VECTOR) { - val dimension = typeDescriptor.getParam(0).toInt - val elementType = if (typeDescriptor.getParams.size() > 1) - HoodieSchema.Vector.VectorElementType.fromString(typeDescriptor.getParam(1)) - else HoodieSchema.Vector.VectorElementType.FLOAT - Some(idx -> (dimension, elementType)) + val parsed = HoodieSchema.parseTypeDescriptor(typeStr) + if (parsed.getType == HoodieSchemaType.VECTOR) { + val vectorSchema = parsed.asInstanceOf[HoodieSchema.Vector] + Some(idx -> (vectorSchema.getDimension, vectorSchema.getVectorElementType)) } else None } else None } else None @@ -460,6 +457,9 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, } else if (vectorCols.contains(i)) { val (dim, elemType) = vectorCols(i) val bytes = row.getBinary(i) + val expectedSize = dim * elemType.getElementSize + require(bytes.length == expectedSize, + s"Vector byte array length mismatch: expected $expectedSize but got ${bytes.length}") val buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN) elemType match { case HoodieSchema.Vector.VectorElementType.FLOAT => @@ -529,8 +529,6 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, // The raw iterator has BinaryType for vector columns; convert back to ArrayType val readSchema = if (remainingPartitionSchema.fields.length == partitionSchema.fields.length) { StructType(modifiedRequiredSchema.fields ++ partitionSchema.fields) - } else if (remainingPartitionSchema.fields.length == 0) { - modifiedOutputSchema } else { modifiedOutputSchema } 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 ab4d5cb22a526..10d2bc95986bb 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 @@ -99,10 +99,11 @@ class TestVectorDataSource extends HoodieSparkClientTestBase { // 7. Verify vector metadata preserved val readMetadata = embeddingField.metadata assertTrue(readMetadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) - val typeDescriptor = HoodieSchema.parseTypeString( + val parsedSchema = HoodieSchema.parseTypeDescriptor( readMetadata.getString(HoodieSchema.TYPE_METADATA_FIELD)) - assertEquals(HoodieSchemaType.VECTOR, typeDescriptor.getType) - assertEquals("128", typeDescriptor.getParam(0)) + assertEquals(HoodieSchemaType.VECTOR, parsedSchema.getType) + val vectorSchema = parsedSchema.asInstanceOf[HoodieSchema.Vector] + assertEquals(128, vectorSchema.getDimension) // 8. Verify float values match exactly val originalRows = df.select("id", "embedding").collect() From eff69bf2fc378e272dbc726b6beb9099bb8250b9 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Fri, 6 Mar 2026 21:41:05 -0800 Subject: [PATCH 05/13] fix vector write path to support all element types, add column projection test - Write path (HoodieRowParquetWriteSupport.makeWriter) now switches on VectorElementType (FLOAT/DOUBLE/INT8) instead of hardcoding float, matching the read paths - Remove redundant detectVectorColumns call in readBaseFile by reusing vectorCols from requiredSchema for requestedSchema - Add testColumnProjectionWithVector covering 3 scenarios: exclude vector, vector-only, and all columns Co-Authored-By: Claude Opus 4.6 (1M context) --- .../row/HoodieRowParquetWriteSupport.java | 24 +++++++- ...HoodieFileGroupReaderBasedFileFormat.scala | 11 ++-- .../functional/TestVectorDataSource.scala | 61 +++++++++++++++++++ 3 files changed, 88 insertions(+), 8 deletions(-) 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 fdd31fb2d7974..6b1f089e950e7 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 @@ -312,11 +312,29 @@ private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { && resolvedSchema.getType() == HoodieSchemaType.VECTOR) { HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) resolvedSchema; int dimension = vectorSchema.getDimension(); + int elementSize = vectorSchema.getVectorElementType().getElementSize(); + HoodieSchema.Vector.VectorElementType elemType = vectorSchema.getVectorElementType(); return (row, ordinal) -> { ArrayData array = row.getArray(ordinal); - ByteBuffer buffer = ByteBuffer.allocate(dimension * 4).order(ByteOrder.LITTLE_ENDIAN); - for (int i = 0; i < dimension; i++) { - buffer.putFloat(array.getFloat(i)); + ByteBuffer buffer = ByteBuffer.allocate(dimension * elementSize).order(ByteOrder.LITTLE_ENDIAN); + switch (elemType) { + case FLOAT: + for (int i = 0; i < dimension; i++) { + buffer.putFloat(array.getFloat(i)); + } + break; + case DOUBLE: + for (int i = 0; i < dimension; i++) { + buffer.putDouble(array.getDouble(i)); + } + break; + case INT8: + for (int i = 0; i < dimension; i++) { + buffer.put(array.getByte(i)); + } + break; + default: + throw new UnsupportedOperationException("Unsupported vector element type: " + elemType); } recordConsumer.addBinary(Binary.fromReusedByteArray(buffer.array())); }; 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 333c93cd98e41..352bbf02d880c 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 @@ -496,12 +496,13 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val modifiedRequiredSchema = if (hasVectors) replaceVectorFieldsWithBinary(requiredSchema, vectorCols) else requiredSchema val modifiedOutputSchema = if (hasVectors) replaceVectorFieldsWithBinary(outputSchema, vectorCols) else outputSchema - val modifiedRequestedSchema = if (hasVectors) { - val requestedVectorCols = detectVectorColumns(requestedSchema) - replaceVectorFieldsWithBinary(requestedSchema, requestedVectorCols) - } else requestedSchema + // requestedSchema = requiredSchema + mandatory partition fields appended at end, + // so vector columns are at the same indices as in requiredSchema — reuse vectorCols + val modifiedRequestedSchema = if (hasVectors) replaceVectorFieldsWithBinary(requestedSchema, vectorCols) else requestedSchema - // Detect vector columns in the full output schema for post-read conversion + // Detect vector columns in the full output schema for post-read conversion. + // Output schema may have different indices than requiredSchema (e.g. partition columns interleaved), + // so we must detect separately here. val outputVectorCols = if (hasVectors) detectVectorColumns(outputSchema) else Map.empty[Int, (Int, HoodieSchema.Vector.VectorElementType)] val rawIter = if (remainingPartitionSchema.fields.length == partitionSchema.fields.length) { 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 10d2bc95986bb..469d5ca8b0710 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 @@ -179,4 +179,65 @@ class TestVectorDataSource extends HoodieSparkClientTestBase { assertEquals(32, key3Embedding.size) assertTrue(key3Embedding.forall(_ == 1.0f)) } + + @Test + def testColumnProjectionWithVector(): Unit = { + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(16)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, metadata), + StructField("label", StringType, nullable = true), + StructField("score", IntegerType, nullable = true) + )) + + val data = (0 until 10).map { i => + Row(s"key_$i", Array.fill(16)(i.toFloat).toSeq, s"label_$i", i * 10) + } + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "projection_vector_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/projection") + + // Read only non-vector columns (vector column excluded) + val nonVectorDf = spark.read.format("hudi").load(basePath + "/projection") + .select("id", "label", "score") + assertEquals(10, nonVectorDf.count()) + val row0 = nonVectorDf.filter("id = 'key_0'").collect()(0) + assertEquals("label_0", row0.getString(1)) + assertEquals(0, row0.getInt(2)) + + // Read only the vector column with id + val vectorOnlyDf = spark.read.format("hudi").load(basePath + "/projection") + .select("id", "embedding") + assertEquals(10, vectorOnlyDf.count()) + val vecRow = vectorOnlyDf.filter("id = 'key_5'").collect()(0) + val embedding = vecRow.getSeq[Float](1) + assertEquals(16, embedding.size) + assertTrue(embedding.forall(_ == 5.0f)) + + // Read all columns including vector + val allDf = spark.read.format("hudi").load(basePath + "/projection") + .select("id", "embedding", "label", "score") + assertEquals(10, allDf.count()) + val allRow = allDf.filter("id = 'key_3'").collect()(0) + assertEquals("label_3", allRow.getString(2)) + assertEquals(30, allRow.getInt(3)) + val allEmbedding = allRow.getSeq[Float](1) + assertEquals(16, allEmbedding.size) + assertTrue(allEmbedding.forall(_ == 3.0f)) + } + } From 9fe1c02ee135b636aa68c67eaeab79c8e23c0b27 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Fri, 13 Mar 2026 10:42:05 -0700 Subject: [PATCH 06/13] Address PR review comments for vector Parquet read/write support - Use VectorLogicalType.VECTOR_BYTE_ORDER instead of hardcoded ByteOrder.LITTLE_ENDIAN in all 4 locations (write support, reader, Scala reader context, file group format) - Add Math.multiplyExact overflow guard for dimension * elementSize in HoodieRowParquetWriteSupport - Remove unnecessary array clone in HoodieSparkParquetReader - Add clarifying comment on non-vector column else branch - Fix misleading "float arrays" comment to "typed arrays" - Move inline JavaConverters import to top-level in SparkFileFormatInternalRowReaderContext - Import Metadata at top level instead of fully-qualified reference - Consolidate duplicate detectVectorColumns, replaceVectorColumnsWithBinary, and convertBinaryToVectorArray into SparkFileFormatInternalRowReaderContext companion object; HoodieFileGroupReaderBasedFileFormat now delegates - Add Javadoc on VectorType explaining it's needed for InternalSchema type hierarchy (cannot reuse HoodieSchema.Vector) - Clean up unused imports (ByteOrder, ByteBuffer, GenericArrayData, StructField, BinaryType, HoodieSchemaType) Co-Authored-By: Claude Opus 4.6 (1M context) --- .../io/storage/HoodieSparkParquetReader.java | 7 +- .../row/HoodieRowParquetWriteSupport.java | 4 +- ...rkFileFormatInternalRowReaderContext.scala | 79 ++++++++++++------- .../apache/hudi/internal/schema/Types.java | 5 ++ ...HoodieFileGroupReaderBasedFileFormat.scala | 61 ++------------ 5 files changed, 70 insertions(+), 86 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index 9f0f9886f6202..bec7a67430863 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -68,7 +68,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.ByteOrder; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -162,7 +161,8 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc // so SparkBasicSchemaEvolution sees matching types (file has FIXED_LEN_BYTE_ARRAY → BinaryType) StructType readStructSchema = structSchema; if (!vectorColumnInfo.isEmpty()) { - StructField[] fields = structSchema.fields().clone(); + StructField[] fields = new StructField[structSchema.fields().length]; + System.arraycopy(structSchema.fields(), 0, fields, 0, fields.length); for (int idx : vectorColumnInfo.keySet()) { StructField orig = fields[idx]; fields[idx] = new StructField(orig.name(), DataTypes.BinaryType, orig.nullable(), Metadata.empty()); @@ -227,6 +227,7 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc } else if (vectorColumnInfo.containsKey(i)) { converted.update(i, convertBinaryToVectorArray(row.getBinary(i), vectorColumnInfo.get(i))); } else { + // Non-vector column: copy value as-is using the read schema's data type converted.update(i, row.get(i, finalReadSchema.apply(i).dataType())); } } @@ -267,7 +268,7 @@ private static GenericArrayData convertBinaryToVectorArray(byte[] bytes, HoodieS int expectedSize = dim * vectorSchema.getVectorElementType().getElementSize(); checkArgument(bytes.length == expectedSize, "Vector byte array length mismatch: expected " + expectedSize + " but got " + bytes.length); - ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER); switch (vectorSchema.getVectorElementType()) { case FLOAT: float[] floats = new float[dim]; 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 6b1f089e950e7..e5983273aa9ea 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 @@ -70,7 +70,6 @@ import org.apache.spark.util.VersionUtils; import java.nio.ByteBuffer; -import java.nio.ByteOrder; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -316,7 +315,8 @@ private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { HoodieSchema.Vector.VectorElementType elemType = vectorSchema.getVectorElementType(); return (row, ordinal) -> { ArrayData array = row.getArray(ordinal); - ByteBuffer buffer = ByteBuffer.allocate(dimension * elementSize).order(ByteOrder.LITTLE_ENDIAN); + int bufferSize = Math.multiplyExact(dimension, elementSize); + ByteBuffer buffer = ByteBuffer.allocate(bufferSize).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER); switch (elemType) { case FLOAT: for (int i = 0; i < dimension; 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 74a9cf3be86be..326a29f000382 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 @@ -43,7 +43,7 @@ import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.GenericArrayData -import org.apache.spark.sql.types.{BinaryType, LongType, MetadataBuilder, StructField, StructType} +import org.apache.spark.sql.types.{BinaryType, LongType, Metadata, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} import scala.collection.JavaConverters._ @@ -116,7 +116,7 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR readSchema, StructType(Seq.empty), getSchemaHandler.getInternalSchemaOpt, readFilters, storage.getConf.asInstanceOf[StorageConfiguration[Configuration]], tableSchemaOpt)) - // Post-process: convert binary VECTOR columns back to float arrays + // Post-process: convert binary VECTOR columns back to typed arrays if (vectorColumnInfo.nonEmpty) { SparkFileFormatInternalRowReaderContext.wrapWithVectorConversion(rawIterator, vectorColumnInfo, readSchema) } else { @@ -315,7 +315,6 @@ object SparkFileFormatInternalRowReaderContext { */ private[hudi] def detectVectorColumns(schema: HoodieSchema): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = { if (schema == null) return Map.empty - import scala.collection.JavaConverters._ schema.getFields.asScala.zipWithIndex.flatMap { case (field, idx) => val fieldSchema = field.schema().getNonNullType if (fieldSchema.getType == HoodieSchemaType.VECTOR) { @@ -327,23 +326,69 @@ object SparkFileFormatInternalRowReaderContext { }.toMap } + /** + * Detects VECTOR columns from Spark StructType metadata. + * @return Map of ordinal to (dimension, elementType) for VECTOR fields. + */ + def detectVectorColumnsFromMetadata(schema: StructType): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = { + schema.fields.zipWithIndex.flatMap { case (field, idx) => + if (field.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) { + val typeStr = field.metadata.getString(HoodieSchema.TYPE_METADATA_FIELD) + if (typeStr.startsWith("VECTOR")) { + val parsed = HoodieSchema.parseTypeDescriptor(typeStr) + if (parsed.getType == HoodieSchemaType.VECTOR) { + val vectorSchema = parsed.asInstanceOf[HoodieSchema.Vector] + Some(idx -> (vectorSchema.getDimension, vectorSchema.getVectorElementType)) + } else None + } else None + } else None + }.toMap + } + /** * Replaces ArrayType with BinaryType for VECTOR columns so the Parquet reader * can read FIXED_LEN_BYTE_ARRAY data without type mismatch. */ - private[hudi] def replaceVectorColumnsWithBinary(structType: StructType, vectorColumns: Map[Int, _]): StructType = { + def replaceVectorColumnsWithBinary(structType: StructType, vectorColumns: Map[Int, _]): StructType = { StructType(structType.fields.zipWithIndex.map { case (field, idx) => if (vectorColumns.contains(idx)) { - StructField(field.name, BinaryType, field.nullable, org.apache.spark.sql.types.Metadata.empty) + StructField(field.name, BinaryType, field.nullable, Metadata.empty) } else { field } }) } + /** + * Converts binary bytes from a FIXED_LEN_BYTE_ARRAY parquet column back to a typed array + * based on the vector's element type and dimension. + */ + def convertBinaryToVectorArray(bytes: Array[Byte], dim: Int, elemType: HoodieSchema.Vector.VectorElementType): GenericArrayData = { + val expectedSize = dim * elemType.getElementSize + require(bytes.length == expectedSize, + s"Vector byte array length mismatch: expected $expectedSize but got ${bytes.length}") + val buffer = java.nio.ByteBuffer.wrap(bytes).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER) + elemType match { + case HoodieSchema.Vector.VectorElementType.FLOAT => + val arr = new Array[Float](dim) + var j = 0 + while (j < dim) { arr(j) = buffer.getFloat(); j += 1 } + new GenericArrayData(arr) + case HoodieSchema.Vector.VectorElementType.DOUBLE => + val arr = new Array[Double](dim) + var j = 0 + while (j < dim) { arr(j) = buffer.getDouble(); j += 1 } + new GenericArrayData(arr) + case HoodieSchema.Vector.VectorElementType.INT8 => + val arr = new Array[Byte](dim) + buffer.get(arr) + new GenericArrayData(arr) + } + } + /** * Wraps an iterator to convert binary VECTOR columns back to typed arrays. - * Unpacks little-endian bytes from FIXED_LEN_BYTE_ARRAY into GenericArrayData. + * Unpacks bytes from FIXED_LEN_BYTE_ARRAY into GenericArrayData using the canonical vector byte order. */ private[hudi] def wrapWithVectorConversion( iterator: ClosableIterator[InternalRow], @@ -361,27 +406,7 @@ object SparkFileFormatInternalRowReaderContext { result.setNullAt(i) } else if (vectorColumns.contains(i)) { val (dim, elemType) = vectorColumns(i) - val bytes = row.getBinary(i) - val expectedSize = dim * elemType.getElementSize - require(bytes.length == expectedSize, - s"Vector byte array length mismatch: expected $expectedSize but got ${bytes.length}") - val buffer = java.nio.ByteBuffer.wrap(bytes).order(java.nio.ByteOrder.LITTLE_ENDIAN) - elemType match { - case HoodieSchema.Vector.VectorElementType.FLOAT => - val arr = new Array[Float](dim) - var j = 0 - while (j < dim) { arr(j) = buffer.getFloat(); j += 1 } - result.update(i, new GenericArrayData(arr)) - case HoodieSchema.Vector.VectorElementType.DOUBLE => - val arr = new Array[Double](dim) - var j = 0 - while (j < dim) { arr(j) = buffer.getDouble(); j += 1 } - result.update(i, new GenericArrayData(arr)) - case HoodieSchema.Vector.VectorElementType.INT8 => - val arr = new Array[Byte](dim) - buffer.get(arr) - result.update(i, new GenericArrayData(arr)) - } + result.update(i, convertBinaryToVectorArray(row.getBinary(i), dim, elemType)) } else { result.update(i, row.get(i, readSchema(i).dataType)) } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java index f4f4abb71adbe..bcb6d32bf16e2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java @@ -301,6 +301,11 @@ public int hashCode() { /** * Vector type that preserves dimension, element type, and storage backing * through InternalSchema round-trips. + * + *

This class is part of the InternalSchema type system (separate from HoodieSchema) + * and follows the same pattern as {@link FixedType}, {@link DecimalTypeFixed}, etc. + * It cannot be replaced with {@code HoodieSchema.Vector} because they belong to + * different type hierarchies. */ public static class VectorType extends PrimitiveType { private final int dimension; 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 352bbf02d880c..4943ed633cf45 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 @@ -24,7 +24,7 @@ import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.config.{HoodieMemoryConfig, TypedProperties} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieFileFormat -import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaType} +import org.apache.hudi.common.schema.HoodieSchema import org.apache.hudi.common.schema.HoodieSchemaUtils import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, ParquetTableSchemaResolver} import org.apache.hudi.common.table.read.HoodieFileGroupReader @@ -48,19 +48,17 @@ import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjecti import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeProjection} -import org.apache.spark.sql.catalyst.util.GenericArrayData import org.apache.spark.sql.execution.datasources.{OutputWriterFactory, PartitionedFile, SparkColumnarFileReader} import org.apache.spark.sql.execution.datasources.orc.OrcUtils import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector} import org.apache.spark.sql.hudi.MultipleColumnarFileFormatReader import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{BinaryType, StructField, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnarBatchUtils} import org.apache.spark.util.SerializableConfiguration import java.io.Closeable -import java.nio.{ByteBuffer, ByteOrder} import scala.collection.JavaConverters.mapAsJavaMapConverter @@ -408,36 +406,11 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, } } - /** - * Detects vector columns in a StructType by checking for VECTOR metadata. - * Returns a map of field index → (dimension, elementType). - */ - private def detectVectorColumns(schema: StructType): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = { - schema.fields.zipWithIndex.flatMap { case (field, idx) => - if (field.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) { - val typeStr = field.metadata.getString(HoodieSchema.TYPE_METADATA_FIELD) - if (typeStr.startsWith("VECTOR")) { - val parsed = HoodieSchema.parseTypeDescriptor(typeStr) - if (parsed.getType == HoodieSchemaType.VECTOR) { - val vectorSchema = parsed.asInstanceOf[HoodieSchema.Vector] - Some(idx -> (vectorSchema.getDimension, vectorSchema.getVectorElementType)) - } else None - } else None - } else None - }.toMap - } + private def detectVectorColumns(schema: StructType): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = + SparkFileFormatInternalRowReaderContext.detectVectorColumnsFromMetadata(schema) - /** - * Replaces vector ArrayType fields with BinaryType so the Parquet reader sees - * a type matching the file's FIXED_LEN_BYTE_ARRAY. - */ - private def replaceVectorFieldsWithBinary(schema: StructType, vectorCols: Map[Int, _]): StructType = { - StructType(schema.fields.zipWithIndex.map { case (field, idx) => - if (vectorCols.contains(idx)) { - StructField(field.name, BinaryType, field.nullable) - } else field - }) - } + private def replaceVectorFieldsWithBinary(schema: StructType, vectorCols: Map[Int, _]): StructType = + SparkFileFormatInternalRowReaderContext.replaceVectorColumnsWithBinary(schema, vectorCols) /** * Wraps an iterator to convert binary VECTOR columns back to typed arrays. @@ -456,27 +429,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, converted.setNullAt(i) } else if (vectorCols.contains(i)) { val (dim, elemType) = vectorCols(i) - val bytes = row.getBinary(i) - val expectedSize = dim * elemType.getElementSize - require(bytes.length == expectedSize, - s"Vector byte array length mismatch: expected $expectedSize but got ${bytes.length}") - val buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN) - elemType match { - case HoodieSchema.Vector.VectorElementType.FLOAT => - val arr = new Array[Float](dim) - var j = 0 - while (j < dim) { arr(j) = buffer.getFloat(); j += 1 } - converted.update(i, new GenericArrayData(arr)) - case HoodieSchema.Vector.VectorElementType.DOUBLE => - val arr = new Array[Double](dim) - var j = 0 - while (j < dim) { arr(j) = buffer.getDouble(); j += 1 } - converted.update(i, new GenericArrayData(arr)) - case HoodieSchema.Vector.VectorElementType.INT8 => - val arr = new Array[Byte](dim) - buffer.get(arr) - converted.update(i, new GenericArrayData(arr)) - } + converted.update(i, SparkFileFormatInternalRowReaderContext.convertBinaryToVectorArray(row.getBinary(i), dim, elemType)) } else { converted.update(i, row.get(i, readSchema.apply(i).dataType)) } From 8c555b8aafe49b35236a8abd04decea2bcb739a3 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Fri, 13 Mar 2026 12:05:31 -0700 Subject: [PATCH 07/13] Add comprehensive vector test coverage for all element types and table types New tests added to TestVectorDataSource: - testDoubleVectorRoundTrip: DOUBLE element type end-to-end (64-dim) - testInt8VectorRoundTrip: INT8/byte element type end-to-end (256-dim) - testMultipleVectorColumns: two vector columns (float + double) in same schema with selective nulls and per-column projection - testMorTableWithVectors: MOR table type with insert + upsert, verifying merged view returns correct vectors - testCowUpsertWithVectors: COW upsert (update existing + insert new) verifying vector values after merge - testLargeDimensionVector: 1536-dim float vectors (OpenAI embedding size) to exercise large buffer allocation - testSmallDimensionVector: 2-dim vectors with edge values (Float.MaxValue) to verify boundary handling - testVectorWithNonVectorArrayColumn: vector column alongside a regular ArrayType(StringType) to ensure non-vector arrays are not incorrectly treated as vectors - testMorWithMultipleUpserts: MOR with 3 successive upsert batches of DOUBLE vectors, verifying the latest value wins per key Co-Authored-By: Claude Opus 4.6 (1M context) --- .../functional/TestVectorDataSource.scala | 565 ++++++++++++++++++ 1 file changed, 565 insertions(+) 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 469d5ca8b0710..c206a07ba2a5d 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 @@ -240,4 +240,569 @@ class TestVectorDataSource extends HoodieSparkClientTestBase { assertTrue(allEmbedding.forall(_ == 3.0f)) } + @Test + def testDoubleVectorRoundTrip(): Unit = { + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(64, DOUBLE)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(DoubleType, containsNull = false), + nullable = false, metadata), + StructField("label", StringType, nullable = true) + )) + + val random = new scala.util.Random(123) + val data = (0 until 50).map { i => + val embedding = Array.fill(64)(random.nextDouble()) + Row(s"key_$i", embedding.toSeq, s"label_$i") + } + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "double_vector_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/double_vec") + + val readDf = spark.read.format("hudi").load(basePath + "/double_vec") + assertEquals(50, readDf.count()) + + // Verify schema: ArrayType(DoubleType) + val embField = readDf.schema("embedding") + val arrType = embField.dataType.asInstanceOf[ArrayType] + assertEquals(DoubleType, arrType.elementType) + + // Verify metadata preserved with DOUBLE element type + val readMeta = embField.metadata + assertTrue(readMeta.contains(HoodieSchema.TYPE_METADATA_FIELD)) + val parsed = HoodieSchema.parseTypeDescriptor( + readMeta.getString(HoodieSchema.TYPE_METADATA_FIELD)) + assertEquals(HoodieSchemaType.VECTOR, parsed.getType) + val vecSchema = parsed.asInstanceOf[HoodieSchema.Vector] + assertEquals(64, vecSchema.getDimension) + assertEquals(HoodieSchema.Vector.VectorElementType.DOUBLE, vecSchema.getVectorElementType) + + // Verify actual values + val origMap = df.select("id", "embedding").collect() + .map(r => (r.getString(0), r.getSeq[Double](1))).toMap + val readMap = readDf.select("id", "embedding").collect() + .map(r => (r.getString(0), r.getSeq[Double](1))).toMap + + origMap.foreach { case (id, orig) => + val read = readMap(id) + assertEquals(64, read.size, s"Dimension mismatch for $id") + orig.zip(read).zipWithIndex.foreach { case ((o, r), idx) => + assertEquals(o, r, 1e-15, s"Double mismatch at $id[$idx]") + } + } + } + + @Test + def testInt8VectorRoundTrip(): Unit = { + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(256, INT8)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(ByteType, containsNull = false), + nullable = false, metadata) + )) + + val random = new scala.util.Random(99) + val data = (0 until 30).map { i => + val embedding = Array.fill(256)((random.nextInt(256) - 128).toByte) + Row(s"key_$i", embedding.toSeq) + } + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "int8_vector_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/int8_vec") + + val readDf = spark.read.format("hudi").load(basePath + "/int8_vec") + assertEquals(30, readDf.count()) + + // Verify schema: ArrayType(ByteType) + val embField = readDf.schema("embedding") + val arrType = embField.dataType.asInstanceOf[ArrayType] + assertEquals(ByteType, arrType.elementType) + + // Verify metadata + val readMeta = embField.metadata + val parsed = HoodieSchema.parseTypeDescriptor( + readMeta.getString(HoodieSchema.TYPE_METADATA_FIELD)) + assertEquals(HoodieSchemaType.VECTOR, parsed.getType) + val vecSchema = parsed.asInstanceOf[HoodieSchema.Vector] + assertEquals(256, vecSchema.getDimension) + assertEquals(HoodieSchema.Vector.VectorElementType.INT8, vecSchema.getVectorElementType) + + // Verify byte values + val origMap = df.select("id", "embedding").collect() + .map(r => (r.getString(0), r.getSeq[Byte](1))).toMap + val readMap = readDf.select("id", "embedding").collect() + .map(r => (r.getString(0), r.getSeq[Byte](1))).toMap + + origMap.foreach { case (id, orig) => + val read = readMap(id) + assertEquals(256, read.size, s"Dimension mismatch for $id") + assertArrayEquals(orig.toArray, read.toArray, s"INT8 vector mismatch for $id") + } + } + + @Test + def testMultipleVectorColumns(): Unit = { + val floatMeta = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(8)") + .build() + val doubleMeta = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(4, DOUBLE)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("vec_float", ArrayType(FloatType, containsNull = false), + nullable = false, floatMeta), + StructField("label", StringType, nullable = true), + StructField("vec_double", ArrayType(DoubleType, containsNull = false), + nullable = true, doubleMeta) + )) + + val data = (0 until 20).map { i => + Row( + s"key_$i", + Array.fill(8)(i.toFloat).toSeq, + s"label_$i", + if (i % 3 == 0) null else Array.fill(4)(i.toDouble).toSeq + ) + } + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "multi_vector_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/multi_vec") + + val readDf = spark.read.format("hudi").load(basePath + "/multi_vec") + assertEquals(20, readDf.count()) + + // Verify both vector columns present with correct types + val floatField = readDf.schema("vec_float") + assertEquals(FloatType, floatField.dataType.asInstanceOf[ArrayType].elementType) + val doubleField = readDf.schema("vec_double") + assertEquals(DoubleType, doubleField.dataType.asInstanceOf[ArrayType].elementType) + + // Verify data: row with both vectors + val row5 = readDf.select("id", "vec_float", "vec_double") + .filter("id = 'key_5'").collect()(0) + val fVec = row5.getSeq[Float](1) + assertEquals(8, fVec.size) + assertTrue(fVec.forall(_ == 5.0f)) + val dVec = row5.getSeq[Double](2) + assertEquals(4, dVec.size) + assertTrue(dVec.forall(_ == 5.0)) + + // Verify data: row with null double vector (i=0, i%3==0) + val row0 = readDf.select("id", "vec_float", "vec_double") + .filter("id = 'key_0'").collect()(0) + assertFalse(row0.isNullAt(1)) + assertTrue(row0.isNullAt(2), "Expected null double vector for key_0") + + // Verify projection: select only one vector column + val floatOnlyDf = readDf.select("id", "vec_float") + assertEquals(20, floatOnlyDf.count()) + val doubleOnlyDf = readDf.select("id", "vec_double") + assertEquals(20, doubleOnlyDf.count()) + } + + @Test + def testMorTableWithVectors(): Unit = { + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(16)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, metadata), + StructField("ts", LongType, nullable = false) + )) + + // Initial insert + val data1 = (0 until 20).map { i => + Row(s"key_$i", Array.fill(16)(1.0f).toSeq, i.toLong) + } + + val df1 = spark.createDataFrame( + spark.sparkContext.parallelize(data1), + schema + ) + + df1.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "mor_vector_test") + .option(TABLE_TYPE.key, "MERGE_ON_READ") + .mode(SaveMode.Overwrite) + .save(basePath + "/mor_vec") + + // Upsert: update some vectors with new values + val data2 = (0 until 10).map { i => + Row(s"key_$i", Array.fill(16)(2.0f).toSeq, 100L + i) + } + + val df2 = spark.createDataFrame( + spark.sparkContext.parallelize(data2), + schema + ) + + df2.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "mor_vector_test") + .option(TABLE_TYPE.key, "MERGE_ON_READ") + .mode(SaveMode.Append) + .save(basePath + "/mor_vec") + + // Read the merged view + val readDf = spark.read.format("hudi").load(basePath + "/mor_vec") + assertEquals(20, readDf.count()) + + // Updated rows (key_0 through key_9) should have new vectors + val updatedRow = readDf.select("id", "embedding") + .filter("id = 'key_5'").collect()(0) + val updatedVec = updatedRow.getSeq[Float](1) + assertEquals(16, updatedVec.size) + assertTrue(updatedVec.forall(_ == 2.0f), "Updated vector should have value 2.0") + + // Non-updated rows (key_10 through key_19) should keep original vectors + val origRow = readDf.select("id", "embedding") + .filter("id = 'key_15'").collect()(0) + val origVec = origRow.getSeq[Float](1) + assertEquals(16, origVec.size) + assertTrue(origVec.forall(_ == 1.0f), "Non-updated vector should have value 1.0") + } + + @Test + def testCowUpsertWithVectors(): Unit = { + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(8)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, metadata), + StructField("ts", LongType, nullable = false), + StructField("name", StringType, nullable = true) + )) + + // Initial write + val data1 = (0 until 10).map { i => + Row(s"key_$i", Array.fill(8)(0.0f).toSeq, i.toLong, s"name_$i") + } + + spark.createDataFrame(spark.sparkContext.parallelize(data1), schema) + .write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "cow_upsert_vec_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/cow_upsert") + + // Upsert: update vectors for existing keys + add new keys + val data2 = Seq( + Row("key_0", Array.fill(8)(9.9f).toSeq, 100L, "updated_0"), + Row("key_5", Array.fill(8)(5.5f).toSeq, 100L, "updated_5"), + Row("key_10", Array.fill(8)(10.0f).toSeq, 100L, "new_10") + ) + + spark.createDataFrame(spark.sparkContext.parallelize(data2), schema) + .write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "cow_upsert_vec_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Append) + .save(basePath + "/cow_upsert") + + val readDf = spark.read.format("hudi").load(basePath + "/cow_upsert") + assertEquals(11, readDf.count()) + + // Verify updated key_0 + val r0 = readDf.select("id", "embedding", "name") + .filter("id = 'key_0'").collect()(0) + assertTrue(r0.getSeq[Float](1).forall(_ == 9.9f)) + assertEquals("updated_0", r0.getString(2)) + + // Verify non-updated key_3 + val r3 = readDf.select("id", "embedding", "name") + .filter("id = 'key_3'").collect()(0) + assertTrue(r3.getSeq[Float](1).forall(_ == 0.0f)) + assertEquals("name_3", r3.getString(2)) + + // Verify new key_10 + val r10 = readDf.select("id", "embedding", "name") + .filter("id = 'key_10'").collect()(0) + assertTrue(r10.getSeq[Float](1).forall(_ == 10.0f)) + assertEquals("new_10", r10.getString(2)) + } + + @Test + def testLargeDimensionVector(): Unit = { + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(1536)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, metadata) + )) + + val random = new scala.util.Random(7) + val data = (0 until 5).map { i => + Row(s"key_$i", Array.fill(1536)(random.nextFloat()).toSeq) + } + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "large_dim_vec_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/large_dim") + + val readDf = spark.read.format("hudi").load(basePath + "/large_dim") + assertEquals(5, readDf.count()) + + // Verify dimension preserved + val readMeta = readDf.schema("embedding").metadata + val vecSchema = HoodieSchema.parseTypeDescriptor( + readMeta.getString(HoodieSchema.TYPE_METADATA_FIELD)).asInstanceOf[HoodieSchema.Vector] + assertEquals(1536, vecSchema.getDimension) + + // Verify values + val origMap = df.select("id", "embedding").collect() + .map(r => (r.getString(0), r.getSeq[Float](1))).toMap + val readMap = readDf.select("id", "embedding").collect() + .map(r => (r.getString(0), r.getSeq[Float](1))).toMap + + origMap.foreach { case (id, orig) => + val read = readMap(id) + assertEquals(1536, read.size) + orig.zip(read).foreach { case (o, r) => + assertEquals(o, r, 1e-9f, s"Mismatch in $id") + } + } + } + + @Test + def testSmallDimensionVector(): Unit = { + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(2)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("coords", ArrayType(FloatType, containsNull = false), + nullable = false, metadata) + )) + + val data = Seq( + Row("a", Seq(1.0f, 2.0f)), + Row("b", Seq(-1.5f, 3.14f)), + Row("c", Seq(0.0f, Float.MaxValue)) + ) + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "small_dim_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/small_dim") + + val readDf = spark.read.format("hudi").load(basePath + "/small_dim") + assertEquals(3, readDf.count()) + + val rowA = readDf.select("id", "coords").filter("id = 'a'").collect()(0) + val coordsA = rowA.getSeq[Float](1) + assertEquals(2, coordsA.size) + assertEquals(1.0f, coordsA(0), 1e-9f) + assertEquals(2.0f, coordsA(1), 1e-9f) + + val rowC = readDf.select("id", "coords").filter("id = 'c'").collect()(0) + val coordsC = rowC.getSeq[Float](1) + assertEquals(Float.MaxValue, coordsC(1), 1e-30f) + } + + @Test + def testVectorWithNonVectorArrayColumn(): Unit = { + val vectorMeta = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(4)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, vectorMeta), + StructField("tags", ArrayType(StringType, containsNull = true), + nullable = true) + )) + + val data = Seq( + Row("k1", Seq(1.0f, 2.0f, 3.0f, 4.0f), Seq("tag1", "tag2")), + Row("k2", Seq(5.0f, 6.0f, 7.0f, 8.0f), null), + Row("k3", Seq(0.1f, 0.2f, 0.3f, 0.4f), Seq("tag3")) + ) + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "mixed_array_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/mixed_array") + + val readDf = spark.read.format("hudi").load(basePath + "/mixed_array") + assertEquals(3, readDf.count()) + + // Vector column should be ArrayType(FloatType) with vector metadata + val embField = readDf.schema("embedding") + assertTrue(embField.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) + assertEquals(FloatType, embField.dataType.asInstanceOf[ArrayType].elementType) + + // Non-vector array column should be ArrayType(StringType) without vector metadata + val tagsField = readDf.schema("tags") + assertFalse(tagsField.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) + assertEquals(StringType, tagsField.dataType.asInstanceOf[ArrayType].elementType) + + // Verify vector data preserved + val row1 = readDf.select("id", "embedding", "tags") + .filter("id = 'k1'").collect()(0) + val emb = row1.getSeq[Float](1) + assertEquals(Seq(1.0f, 2.0f, 3.0f, 4.0f), emb) + assertEquals(Seq("tag1", "tag2"), row1.getSeq[String](2)) + + // Verify null tags preserved + val row2 = readDf.select("id", "embedding", "tags") + .filter("id = 'k2'").collect()(0) + assertFalse(row2.isNullAt(1)) + assertTrue(row2.isNullAt(2)) + } + + @Test + def testMorWithMultipleUpserts(): Unit = { + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(4, DOUBLE)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(DoubleType, containsNull = false), + nullable = false, metadata), + StructField("ts", LongType, nullable = false) + )) + + // Insert batch 1 + val batch1 = (0 until 10).map { i => + Row(s"key_$i", Array.fill(4)(1.0).toSeq, 1L) + } + spark.createDataFrame(spark.sparkContext.parallelize(batch1), schema) + .write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "mor_multi_upsert_test") + .option(TABLE_TYPE.key, "MERGE_ON_READ") + .mode(SaveMode.Overwrite) + .save(basePath + "/mor_multi") + + // Upsert batch 2: update key_0..key_4 + val batch2 = (0 until 5).map { i => + Row(s"key_$i", Array.fill(4)(2.0).toSeq, 2L) + } + spark.createDataFrame(spark.sparkContext.parallelize(batch2), schema) + .write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "mor_multi_upsert_test") + .option(TABLE_TYPE.key, "MERGE_ON_READ") + .mode(SaveMode.Append) + .save(basePath + "/mor_multi") + + // Upsert batch 3: update key_0..key_2 again + val batch3 = (0 until 3).map { i => + Row(s"key_$i", Array.fill(4)(3.0).toSeq, 3L) + } + spark.createDataFrame(spark.sparkContext.parallelize(batch3), schema) + .write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "mor_multi_upsert_test") + .option(TABLE_TYPE.key, "MERGE_ON_READ") + .mode(SaveMode.Append) + .save(basePath + "/mor_multi") + + val readDf = spark.read.format("hudi").load(basePath + "/mor_multi") + assertEquals(10, readDf.count()) + + // key_0: updated 3 times → should have value 3.0 + val r0 = readDf.select("id", "embedding").filter("id = 'key_0'").collect()(0) + assertTrue(r0.getSeq[Double](1).forall(_ == 3.0), "key_0 should have latest value 3.0") + + // key_3: updated once (batch 2) → should have value 2.0 + val r3 = readDf.select("id", "embedding").filter("id = 'key_3'").collect()(0) + assertTrue(r3.getSeq[Double](1).forall(_ == 2.0), "key_3 should have value 2.0") + + // key_7: never updated → should have value 1.0 + val r7 = readDf.select("id", "embedding").filter("id = 'key_7'").collect()(0) + assertTrue(r7.getSeq[Double](1).forall(_ == 1.0), "key_7 should have original value 1.0") + } + + private def assertArrayEquals(expected: Array[Byte], actual: Array[Byte], message: String): Unit = { + assertEquals(expected.length, actual.length, s"$message: length mismatch") + expected.zip(actual).zipWithIndex.foreach { case ((e, a), idx) => + assertEquals(e, a, s"$message: mismatch at index $idx") + } + } } From 35223ec3a6ac7f54280b58e364b9b4e6daef4f91 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Mon, 16 Mar 2026 19:51:48 -0700 Subject: [PATCH 08/13] Handle vector schema checks and skip vector column stats --- ...dieSchemaComparatorForSchemaEvolution.java | 10 +++++ .../HoodieSchemaCompatibilityChecker.java | 20 ++++++++++ .../metadata/HoodieTableMetadataUtil.java | 7 +++- ...dieSchemaComparatorForSchemaEvolution.java | 18 +++++++++ .../schema/TestHoodieSchemaCompatibility.java | 38 +++++++++++++++++++ .../metadata/TestHoodieTableMetadataUtil.java | 9 +++++ 6 files changed, 100 insertions(+), 2 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaComparatorForSchemaEvolution.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaComparatorForSchemaEvolution.java index e3f28165f0db2..c38cf22863704 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaComparatorForSchemaEvolution.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaComparatorForSchemaEvolution.java @@ -164,6 +164,8 @@ protected boolean schemaEqualsInternal(HoodieSchema s1, HoodieSchema s2) { return mapSchemaEquals(s1, s2); case FIXED: return fixedSchemaEquals(s1, s2); + case VECTOR: + return vectorSchemaEquals(s1, s2); case UNION: return unionSchemaEquals(s1, s2); case STRING: @@ -289,6 +291,14 @@ private boolean fixedSchemaEquals(HoodieSchema s1, HoodieSchema s2) { return s1.getName().equals(s2.getName()) && s1.getFixedSize() == s2.getFixedSize(); } + private static boolean vectorSchemaEquals(HoodieSchema s1, HoodieSchema s2) { + HoodieSchema.Vector v1 = (HoodieSchema.Vector) s1; + HoodieSchema.Vector v2 = (HoodieSchema.Vector) s2; + return v1.getDimension() == v2.getDimension() + && v1.getVectorElementType() == v2.getVectorElementType() + && v1.getStorageBacking() == v2.getStorageBacking(); + } + private static boolean decimalSchemaEquals(HoodieSchema s1, HoodieSchema s2) { HoodieSchema.Decimal d1 = (HoodieSchema.Decimal) s1; HoodieSchema.Decimal d2 = (HoodieSchema.Decimal) s2; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibilityChecker.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibilityChecker.java index 66935b6d17452..726a75d561009 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibilityChecker.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibilityChecker.java @@ -310,6 +310,8 @@ private SchemaCompatibilityResult calculateCompatibility(final HoodieSchema read case FIXED: result = result.mergedWith(checkSchemaNames(reader, writer, locations)); return result.mergedWith(checkFixedSize(reader, writer, locations)); + case VECTOR: + return result.mergedWith(checkVectorCompatibility(reader, writer, locations)); case DECIMAL: return result.mergedWith(checkDecimalWidening(reader, writer, locations)); case ENUM: @@ -377,6 +379,7 @@ private SchemaCompatibilityResult calculateCompatibility(final HoodieSchema read case MAP: return result.mergedWith(typeMismatch(reader, writer, locations)); case FIXED: + case VECTOR: return result.mergedWith(typeMismatch(reader, writer, locations)); case ENUM: return result.mergedWith(typeMismatch(reader, writer, locations)); @@ -457,6 +460,23 @@ private SchemaCompatibilityResult checkFixedSize(final HoodieSchema reader, fina return checkDecimalWidening(reader, writer, locations); } + private SchemaCompatibilityResult checkVectorCompatibility(final HoodieSchema reader, final HoodieSchema writer, + final Deque locations) { + HoodieSchema.Vector readerVector = (HoodieSchema.Vector) reader; + HoodieSchema.Vector writerVector = (HoodieSchema.Vector) writer; + if (readerVector.getDimension() != writerVector.getDimension() + || readerVector.getVectorElementType() != writerVector.getVectorElementType() + || readerVector.getStorageBacking() != writerVector.getStorageBacking()) { + String message = String.format("Vector field '%s' expected dimension: %d, elementType: %s, storageBacking: %s, found: dimension: %d, elementType: %s, storageBacking: %s", + getLocationName(locations, reader.getType()), + writerVector.getDimension(), writerVector.getVectorElementType(), writerVector.getStorageBacking(), + readerVector.getDimension(), readerVector.getVectorElementType(), readerVector.getStorageBacking()); + return SchemaCompatibilityResult.incompatible(SchemaIncompatibilityType.TYPE_MISMATCH, reader, writer, + message, asList(locations)); + } + return SchemaCompatibilityResult.compatible(); + } + private SchemaCompatibilityResult checkDecimalWidening(final HoodieSchema reader, final HoodieSchema writer, final Deque locations) { boolean isReaderDecimal = reader.getType() == HoodieSchemaType.DECIMAL; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index bfa0c803fe64e..2babb429260e9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -284,11 +284,14 @@ public static Map> collectColumnRa String fieldName = fieldNameFieldPair.getKey(); HoodieSchemaField field = fieldNameFieldPair.getValue(); HoodieSchema fieldSchema = field.schema().getNonNullType(); + if (!isColumnTypeSupported(fieldSchema, Option.of(record.getRecordType()), indexVersion)) { + return; + } ColumnStats colStats = allColumnStats.computeIfAbsent(fieldName, ignored -> new ColumnStats(getValueMetadata(fieldSchema, indexVersion))); Object fieldValue = collectColumnRangeFieldValue(record, colStats.valueMetadata, fieldName, fieldSchema, recordSchema, properties); colStats.valueCount++; - if (fieldValue != null && isColumnTypeSupported(fieldSchema, Option.of(record.getRecordType()), indexVersion)) { + if (fieldValue != null) { // Set the min value of the field if (colStats.minValue == null || ConvertingGenericData.INSTANCE.compare(fieldValue, colStats.minValue, fieldSchema.toAvroSchema()) < 0) { @@ -2056,7 +2059,7 @@ private static boolean isColumnTypeSupportedV2(HoodieSchema schema) { // Check for precision and scale if the schema has a logical decimal type. return type != HoodieSchemaType.RECORD && type != HoodieSchemaType.MAP && type != HoodieSchemaType.ARRAY && type != HoodieSchemaType.ENUM - && type != HoodieSchemaType.BLOB; + && type != HoodieSchemaType.BLOB && type != HoodieSchemaType.VECTOR; } public static Set getInflightMetadataPartitions(HoodieTableConfig tableConfig) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaComparatorForSchemaEvolution.java b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaComparatorForSchemaEvolution.java index 29d53f0c146ce..09724cc54e914 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaComparatorForSchemaEvolution.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaComparatorForSchemaEvolution.java @@ -611,4 +611,22 @@ void testBlobFieldEquality() { // BLOB with fields in different order should not be equal (order matters) assertFalse(HoodieSchemaComparatorForSchemaEvolution.schemaEquals(blob1, blobWithDifferentOrder)); } + + @Test + void testVectorSchemaEquality() { + assertTrue(HoodieSchemaComparatorForSchemaEvolution.schemaEquals( + HoodieSchema.createVector(16, HoodieSchema.Vector.VectorElementType.FLOAT), + HoodieSchema.createVector(16, HoodieSchema.Vector.VectorElementType.FLOAT) + )); + + assertFalse(HoodieSchemaComparatorForSchemaEvolution.schemaEquals( + HoodieSchema.createVector(16, HoodieSchema.Vector.VectorElementType.FLOAT), + HoodieSchema.createVector(16, HoodieSchema.Vector.VectorElementType.DOUBLE) + )); + + assertFalse(HoodieSchemaComparatorForSchemaEvolution.schemaEquals( + HoodieSchema.createVector(16, HoodieSchema.Vector.VectorElementType.FLOAT), + HoodieSchema.createVector(32, HoodieSchema.Vector.VectorElementType.FLOAT) + )); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaCompatibility.java b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaCompatibility.java index 6ff89d6e8586d..f3313db6ffd8f 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaCompatibility.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaCompatibility.java @@ -39,6 +39,7 @@ import static org.apache.hudi.common.schema.TestHoodieSchemaUtils.EVOLVED_SCHEMA; import static org.apache.hudi.common.schema.TestHoodieSchemaUtils.SIMPLE_SCHEMA; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -814,4 +815,41 @@ public void testBlobAreSchemasProjectionEquivalent() { HoodieSchema stringSchema = HoodieSchema.create(HoodieSchemaType.STRING); assertFalse(HoodieSchemaCompatibility.areSchemasProjectionEquivalent(blob1, stringSchema)); } + + @Test + public void testVectorSchemaCompatibility() { + HoodieSchema incomingSchema = HoodieSchema.createRecord("vector_record", null, null, Arrays.asList( + HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.STRING), null, null), + HoodieSchemaField.of("embedding", HoodieSchema.createVector(4, HoodieSchema.Vector.VectorElementType.DOUBLE), null, null) + )); + + HoodieSchema tableSchema = HoodieSchema.createRecord("vector_record", null, null, Arrays.asList( + HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.STRING), null, null), + HoodieSchemaField.of("embedding", HoodieSchema.createVector(4, HoodieSchema.Vector.VectorElementType.DOUBLE), null, null) + )); + + HoodieSchemaCompatibilityChecker.SchemaPairCompatibility compatibility = + HoodieSchemaCompatibilityChecker.checkReaderWriterCompatibility(incomingSchema, tableSchema, false); + assertEquals(HoodieSchemaCompatibilityChecker.SchemaCompatibilityType.COMPATIBLE, compatibility.getType()); + assertDoesNotThrow(() -> HoodieSchemaCompatibility.checkValidEvolution(incomingSchema, tableSchema)); + } + + @Test + public void testVectorSchemaCompatibilityRejectsElementTypeEvolution() { + HoodieSchema incomingSchema = HoodieSchema.createRecord("vector_record", null, null, Arrays.asList( + HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.STRING), null, null), + HoodieSchemaField.of("embedding", HoodieSchema.createVector(4, HoodieSchema.Vector.VectorElementType.DOUBLE), null, null) + )); + + HoodieSchema tableSchema = HoodieSchema.createRecord("vector_record", null, null, Arrays.asList( + HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.STRING), null, null), + HoodieSchemaField.of("embedding", HoodieSchema.createVector(4, HoodieSchema.Vector.VectorElementType.FLOAT), null, null) + )); + + HoodieSchemaCompatibilityChecker.SchemaPairCompatibility compatibility = + HoodieSchemaCompatibilityChecker.checkReaderWriterCompatibility(incomingSchema, tableSchema, false); + assertEquals(HoodieSchemaCompatibilityChecker.SchemaCompatibilityType.INCOMPATIBLE, compatibility.getType()); + assertThrows(SchemaBackwardsCompatibilityException.class, + () -> HoodieSchemaCompatibility.checkValidEvolution(incomingSchema, tableSchema)); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java b/hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java index 171bd92315579..b0093315306d1 100644 --- a/hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java +++ b/hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java @@ -322,4 +322,13 @@ void testIsTimestampMillisField() { assertFalse(HoodieTableMetadataUtil.isTimestampMillisField(stringSchema), "Should return false for string"); } + + @Test + void testVectorColumnsAreNotSupportedForV2ColumnStats() { + HoodieSchema vectorSchema = HoodieSchema.createNullable(HoodieSchema.createVector(128)); + HoodieSchema stringSchema = HoodieSchema.createNullable(HoodieSchema.create(HoodieSchemaType.STRING)); + + assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(vectorSchema, Option.empty(), HoodieIndexVersion.V2)); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(stringSchema, Option.empty(), HoodieIndexVersion.V2)); + } } From 09a032c335279bb74fd40f275d7b081037dc61ee Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Tue, 17 Mar 2026 10:47:32 -0700 Subject: [PATCH 09/13] Extract VectorConversionUtils, add write-path dimension validation, fix hot-path allocation - Create shared VectorConversionUtils utility class to eliminate duplicated vector conversion logic across HoodieSparkParquetReader, SparkFileFormatInternalRowReaderContext, and HoodieFileGroupReaderBasedFileFormat - Add explicit dimension validation in HoodieRowParquetWriteSupport to prevent silent data corruption when array length doesn't match declared vector dimension - Reuse GenericInternalRow in HoodieSparkParquetReader's vector post-processing loop to reduce GC pressure on large scans --- .../io/storage/HoodieSparkParquetReader.java | 82 +------- .../io/storage/VectorConversionUtils.java | 176 ++++++++++++++++++ .../row/HoodieRowParquetWriteSupport.java | 2 + ...rkFileFormatInternalRowReaderContext.scala | 83 ++------- ...HoodieFileGroupReaderBasedFileFormat.scala | 9 +- 5 files changed, 206 insertions(+), 146 deletions(-) create mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index bec7a67430863..8de9392410ed7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -24,8 +24,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieSparkRecord; 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.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.FileFormatUtils; import org.apache.hudi.common.util.Option; @@ -52,7 +50,6 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; -import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.catalyst.util.RebaseDateTime; import org.apache.spark.sql.execution.datasources.parquet.HoodieParquetReadSupport; import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters; @@ -61,22 +58,16 @@ import org.apache.spark.sql.execution.datasources.parquet.SparkBasicSchemaEvolution; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.sources.Filter; -import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; -import static org.apache.hudi.common.util.ValidationUtils.checkArgument; - import scala.Option$; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; @@ -154,21 +145,14 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc HoodieSchema nonNullSchema = requestedSchema.getNonNullType(); StructType structSchema = HoodieInternalRowUtils.getCachedSchema(nonNullSchema); - // Detect vector columns: ordinal → (dimension, elementType) - Map vectorColumnInfo = detectVectorColumns(nonNullSchema); + // Detect vector columns: ordinal → Vector schema + Map vectorColumnInfo = VectorConversionUtils.detectVectorColumns(nonNullSchema); // For vector columns, replace ArrayType(FloatType) with BinaryType in the read schema // so SparkBasicSchemaEvolution sees matching types (file has FIXED_LEN_BYTE_ARRAY → BinaryType) - StructType readStructSchema = structSchema; - if (!vectorColumnInfo.isEmpty()) { - StructField[] fields = new StructField[structSchema.fields().length]; - System.arraycopy(structSchema.fields(), 0, fields, 0, fields.length); - for (int idx : vectorColumnInfo.keySet()) { - StructField orig = fields[idx]; - fields[idx] = new StructField(orig.name(), DataTypes.BinaryType, orig.nullable(), Metadata.empty()); - } - readStructSchema = new StructType(fields); - } + StructType readStructSchema = vectorColumnInfo.isEmpty() + ? structSchema + : VectorConversionUtils.replaceVectorColumnsWithBinary(structSchema, vectorColumnInfo); Option messageSchema = Option.of(getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(nonNullSchema)); boolean enableTimestampFieldRepair = storage.getConf().getBoolean(ENABLE_LOGICAL_TIMESTAMP_REPAIR, true); @@ -218,14 +202,15 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc UnsafeProjection vectorProjection = UnsafeProjection.create(structSchema); int numFields = readStructSchema.fields().length; StructType finalReadSchema = readStructSchema; + // Reuse a single GenericInternalRow across iterations; UnsafeProjection.apply() copies the data + GenericInternalRow converted = new GenericInternalRow(numFields); CloseableMappingIterator vectorIterator = new CloseableMappingIterator<>(projectedIterator, row -> { - GenericInternalRow converted = new GenericInternalRow(numFields); for (int i = 0; i < numFields; i++) { if (row.isNullAt(i)) { converted.setNullAt(i); } else if (vectorColumnInfo.containsKey(i)) { - converted.update(i, convertBinaryToVectorArray(row.getBinary(i), vectorColumnInfo.get(i))); + converted.update(i, VectorConversionUtils.convertBinaryToVectorArray(row.getBinary(i), vectorColumnInfo.get(i))); } else { // Non-vector column: copy value as-is using the read schema's data type converted.update(i, row.get(i, finalReadSchema.apply(i).dataType())); @@ -241,57 +226,6 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc return projectedIterator; } - /** - * Detects vector columns in the schema and returns a map of ordinal to Vector schema. - */ - private static Map detectVectorColumns(HoodieSchema schema) { - Map vectorColumnInfo = new HashMap<>(); - if (schema == null) { - return vectorColumnInfo; - } - List fields = schema.getFields(); - for (int i = 0; i < fields.size(); i++) { - HoodieSchema fieldSchema = fields.get(i).schema().getNonNullType(); - if (fieldSchema.getType() == HoodieSchemaType.VECTOR) { - vectorColumnInfo.put(i, (HoodieSchema.Vector) fieldSchema); - } - } - return vectorColumnInfo; - } - - /** - * Converts binary bytes from a FIXED_LEN_BYTE_ARRAY parquet column back to a typed array - * based on the vector's element type and dimension. - */ - private static GenericArrayData convertBinaryToVectorArray(byte[] bytes, HoodieSchema.Vector vectorSchema) { - int dim = vectorSchema.getDimension(); - int expectedSize = dim * vectorSchema.getVectorElementType().getElementSize(); - checkArgument(bytes.length == expectedSize, - "Vector byte array length mismatch: expected " + expectedSize + " but got " + bytes.length); - ByteBuffer buffer = ByteBuffer.wrap(bytes).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER); - switch (vectorSchema.getVectorElementType()) { - case FLOAT: - float[] floats = new float[dim]; - for (int j = 0; j < dim; j++) { - floats[j] = buffer.getFloat(); - } - return new GenericArrayData(floats); - case DOUBLE: - double[] doubles = new double[dim]; - for (int j = 0; j < dim; j++) { - doubles[j] = buffer.getDouble(); - } - return new GenericArrayData(doubles); - case INT8: - byte[] int8s = new byte[dim]; - buffer.get(int8s); - return new GenericArrayData(int8s); - default: - throw new UnsupportedOperationException( - "Unsupported vector element type: " + vectorSchema.getVectorElementType()); - } - } - private MessageType getFileSchema() { if (fileSchemaOption.isEmpty()) { MessageType messageType = ((ParquetUtils) parquetUtils).readMessageType(storage, path); 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 new file mode 100644 index 0000000000000..a2dba86ee4367 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; + +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.types.BinaryType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; + +/** + * Shared utility methods for vector column handling during Parquet read/write. + * + * Vectors are stored as Parquet FIXED_LEN_BYTE_ARRAY columns. On read, Spark maps these + * to BinaryType. This class provides the canonical conversion between the binary + * representation and Spark's typed ArrayData (float[], double[], byte[]). + * + * All byte buffers use little-endian order ({@link HoodieSchema.VectorLogicalType#VECTOR_BYTE_ORDER}) + * for compatibility with common vector search libraries (FAISS, ScaNN, etc.) and to match + * native x86/ARM byte order for zero-copy reads. + */ +public final class VectorConversionUtils { + + private VectorConversionUtils() { + } + + /** + * Detects VECTOR columns in a HoodieSchema record and returns a map of field ordinal + * to the corresponding {@link HoodieSchema.Vector} schema. + * + * @param schema a HoodieSchema of type RECORD (or null) + * @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<>(); + if (schema == null) { + return vectorColumnInfo; + } + List fields = schema.getFields(); + for (int i = 0; i < fields.size(); i++) { + HoodieSchema fieldSchema = fields.get(i).schema().getNonNullType(); + if (fieldSchema.getType() == HoodieSchemaType.VECTOR) { + vectorColumnInfo.put(i, (HoodieSchema.Vector) fieldSchema); + } + } + return vectorColumnInfo; + } + + /** + * Detects VECTOR columns from Spark StructType metadata annotations. + * Fields with metadata key {@link HoodieSchema#TYPE_METADATA_FIELD} starting with "VECTOR" + * are parsed and included. + * + * @param schema Spark StructType + * @return map from field index to Vector schema; empty map if no vectors found + */ + public static Map detectVectorColumnsFromMetadata(StructType schema) { + Map vectorColumnInfo = new HashMap<>(); + if (schema == null) { + return vectorColumnInfo; + } + StructField[] fields = schema.fields(); + for (int i = 0; i < fields.length; i++) { + StructField field = fields[i]; + if (field.metadata().contains(HoodieSchema.TYPE_METADATA_FIELD)) { + String typeStr = field.metadata().getString(HoodieSchema.TYPE_METADATA_FIELD); + if (typeStr.startsWith("VECTOR")) { + HoodieSchema parsed = HoodieSchema.parseTypeDescriptor(typeStr); + if (parsed.getType() == HoodieSchemaType.VECTOR) { + vectorColumnInfo.put(i, (HoodieSchema.Vector) parsed); + } + } + } + } + return vectorColumnInfo; + } + + /** + * Replaces ArrayType with BinaryType for VECTOR columns so the Parquet reader + * can read FIXED_LEN_BYTE_ARRAY data without type mismatch. + * + * @param structType the original Spark schema + * @param vectorColumns map of ordinal to vector info (only the key set is used) + * @return a new StructType with vector columns replaced by BinaryType + */ + public static StructType replaceVectorColumnsWithBinary(StructType structType, Map vectorColumns) { + StructField[] fields = structType.fields(); + StructField[] newFields = new StructField[fields.length]; + for (int i = 0; i < fields.length; i++) { + if (vectorColumns.containsKey(i)) { + newFields[i] = new StructField(fields[i].name(), BinaryType$.MODULE$, fields[i].nullable(), Metadata.empty()); + } else { + newFields[i] = fields[i]; + } + } + return new StructType(newFields); + } + + /** + * Converts binary bytes from a FIXED_LEN_BYTE_ARRAY Parquet column back to a typed array + * based on the vector's element type and dimension. + * + * @param bytes raw bytes read from Parquet + * @param vectorSchema the vector schema describing dimension and element type + * @return a GenericArrayData containing the decoded float[], double[], or byte[] array + * @throws IllegalArgumentException if byte array length doesn't match expected size + */ + public static GenericArrayData convertBinaryToVectorArray(byte[] bytes, HoodieSchema.Vector vectorSchema) { + return convertBinaryToVectorArray(bytes, vectorSchema.getDimension(), vectorSchema.getVectorElementType()); + } + + /** + * Converts binary bytes from a FIXED_LEN_BYTE_ARRAY Parquet column back to a typed array. + * + * @param bytes raw bytes read from Parquet + * @param dim vector dimension (number of elements) + * @param elemType element type (FLOAT, DOUBLE, or INT8) + * @return a GenericArrayData containing the decoded float[], double[], or byte[] array + * @throws IllegalArgumentException if byte array length doesn't match expected size + */ + public static GenericArrayData convertBinaryToVectorArray(byte[] bytes, int dim, + HoodieSchema.Vector.VectorElementType elemType) { + int expectedSize = dim * elemType.getElementSize(); + checkArgument(bytes.length == expectedSize, + "Vector byte array length mismatch: expected " + expectedSize + " but got " + bytes.length); + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER); + switch (elemType) { + case FLOAT: + float[] floats = new float[dim]; + for (int j = 0; j < dim; j++) { + floats[j] = buffer.getFloat(); + } + return new GenericArrayData(floats); + case DOUBLE: + double[] doubles = new double[dim]; + for (int j = 0; j < dim; j++) { + doubles[j] = buffer.getDouble(); + } + return new GenericArrayData(doubles); + case INT8: + byte[] int8s = new byte[dim]; + buffer.get(int8s); + return new GenericArrayData(int8s); + default: + throw new UnsupportedOperationException( + "Unsupported vector element type: " + elemType); + } + } +} 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 e5983273aa9ea..bca1892b0e015 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 @@ -315,6 +315,8 @@ private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { HoodieSchema.Vector.VectorElementType elemType = vectorSchema.getVectorElementType(); return (row, ordinal) -> { ArrayData array = row.getArray(ordinal); + ValidationUtils.checkArgument(array.numElements() == dimension, + () -> String.format("Vector dimension mismatch: schema expects %d elements but got %d", dimension, array.numElements())); int bufferSize = Math.multiplyExact(dimension, elementSize); ByteBuffer buffer = ByteBuffer.allocate(bufferSize).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER); switch (elemType) { 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 326a29f000382..02c52dfdd8daa 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 @@ -25,12 +25,12 @@ import org.apache.hudi.avro.{AvroSchemaUtils, HoodieAvroUtils} 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.schema.{HoodieSchema, HoodieSchemaType, HoodieSchemaUtils} +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 import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.common.util.collection.{CachingIterator, ClosableIterator, Pair => HPair} -import org.apache.hudi.io.storage.{HoodieSparkFileReaderFactory, HoodieSparkParquetReader} +import org.apache.hudi.io.storage.{HoodieSparkFileReaderFactory, HoodieSparkParquetReader, VectorConversionUtils} import org.apache.hudi.storage.{HoodieStorage, StorageConfiguration, StoragePath} import org.apache.hudi.util.CloseableInternalRowIterator import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter @@ -42,8 +42,7 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.catalyst.util.GenericArrayData -import org.apache.spark.sql.types.{BinaryType, LongType, Metadata, MetadataBuilder, StructField, StructType} +import org.apache.spark.sql.types.{LongType, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} import scala.collection.JavaConverters._ @@ -311,79 +310,30 @@ object SparkFileFormatInternalRowReaderContext { /** * Detects VECTOR columns from HoodieSchema. - * @return Map of ordinal to (dimension, elementType) for VECTOR fields. + * Delegates to [[VectorConversionUtils.detectVectorColumns]]. + * @return Map of ordinal to Vector schema for VECTOR fields. */ - private[hudi] def detectVectorColumns(schema: HoodieSchema): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = { - if (schema == null) return Map.empty - schema.getFields.asScala.zipWithIndex.flatMap { case (field, idx) => - val fieldSchema = field.schema().getNonNullType - if (fieldSchema.getType == HoodieSchemaType.VECTOR) { - val v = fieldSchema.asInstanceOf[HoodieSchema.Vector] - Some(idx -> (v.getDimension, v.getVectorElementType)) - } else { - None - } - }.toMap + private[hudi] def detectVectorColumns(schema: HoodieSchema): Map[Int, HoodieSchema.Vector] = { + VectorConversionUtils.detectVectorColumns(schema).asScala.map { case (k, v) => (k.intValue(), v) }.toMap } /** * Detects VECTOR columns from Spark StructType metadata. - * @return Map of ordinal to (dimension, elementType) for VECTOR fields. + * Delegates to [[VectorConversionUtils.detectVectorColumnsFromMetadata]]. + * @return Map of ordinal to Vector schema for VECTOR fields. */ - def detectVectorColumnsFromMetadata(schema: StructType): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = { - schema.fields.zipWithIndex.flatMap { case (field, idx) => - if (field.metadata.contains(HoodieSchema.TYPE_METADATA_FIELD)) { - val typeStr = field.metadata.getString(HoodieSchema.TYPE_METADATA_FIELD) - if (typeStr.startsWith("VECTOR")) { - val parsed = HoodieSchema.parseTypeDescriptor(typeStr) - if (parsed.getType == HoodieSchemaType.VECTOR) { - val vectorSchema = parsed.asInstanceOf[HoodieSchema.Vector] - Some(idx -> (vectorSchema.getDimension, vectorSchema.getVectorElementType)) - } else None - } else None - } else None - }.toMap + def detectVectorColumnsFromMetadata(schema: StructType): Map[Int, HoodieSchema.Vector] = { + VectorConversionUtils.detectVectorColumnsFromMetadata(schema).asScala.map { case (k, v) => (k.intValue(), v) }.toMap } /** * Replaces ArrayType with BinaryType for VECTOR columns so the Parquet reader * can read FIXED_LEN_BYTE_ARRAY data without type mismatch. + * Delegates to [[VectorConversionUtils.replaceVectorColumnsWithBinary]]. */ def replaceVectorColumnsWithBinary(structType: StructType, vectorColumns: Map[Int, _]): StructType = { - StructType(structType.fields.zipWithIndex.map { case (field, idx) => - if (vectorColumns.contains(idx)) { - StructField(field.name, BinaryType, field.nullable, Metadata.empty) - } else { - field - } - }) - } - - /** - * Converts binary bytes from a FIXED_LEN_BYTE_ARRAY parquet column back to a typed array - * based on the vector's element type and dimension. - */ - def convertBinaryToVectorArray(bytes: Array[Byte], dim: Int, elemType: HoodieSchema.Vector.VectorElementType): GenericArrayData = { - val expectedSize = dim * elemType.getElementSize - require(bytes.length == expectedSize, - s"Vector byte array length mismatch: expected $expectedSize but got ${bytes.length}") - val buffer = java.nio.ByteBuffer.wrap(bytes).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER) - elemType match { - case HoodieSchema.Vector.VectorElementType.FLOAT => - val arr = new Array[Float](dim) - var j = 0 - while (j < dim) { arr(j) = buffer.getFloat(); j += 1 } - new GenericArrayData(arr) - case HoodieSchema.Vector.VectorElementType.DOUBLE => - val arr = new Array[Double](dim) - var j = 0 - while (j < dim) { arr(j) = buffer.getDouble(); j += 1 } - new GenericArrayData(arr) - case HoodieSchema.Vector.VectorElementType.INT8 => - val arr = new Array[Byte](dim) - buffer.get(arr) - new GenericArrayData(arr) - } + val javaMap = vectorColumns.map { case (k, v) => (Integer.valueOf(k), v.asInstanceOf[AnyRef]) }.asJava + VectorConversionUtils.replaceVectorColumnsWithBinary(structType, javaMap) } /** @@ -392,7 +342,7 @@ object SparkFileFormatInternalRowReaderContext { */ private[hudi] def wrapWithVectorConversion( iterator: ClosableIterator[InternalRow], - vectorColumns: Map[Int, (Int, HoodieSchema.Vector.VectorElementType)], + vectorColumns: Map[Int, HoodieSchema.Vector], readSchema: StructType): ClosableIterator[InternalRow] = { val numFields = readSchema.fields.length new ClosableIterator[InternalRow] { @@ -405,8 +355,7 @@ object SparkFileFormatInternalRowReaderContext { if (row.isNullAt(i)) { result.setNullAt(i) } else if (vectorColumns.contains(i)) { - val (dim, elemType) = vectorColumns(i) - result.update(i, convertBinaryToVectorArray(row.getBinary(i), dim, elemType)) + result.update(i, VectorConversionUtils.convertBinaryToVectorArray(row.getBinary(i), vectorColumns(i))) } else { result.update(i, row.get(i, readSchema(i).dataType)) } 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 4943ed633cf45..fdec37e0bc68f 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 @@ -406,7 +406,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, } } - private def detectVectorColumns(schema: StructType): Map[Int, (Int, HoodieSchema.Vector.VectorElementType)] = + private def detectVectorColumns(schema: StructType): Map[Int, HoodieSchema.Vector] = SparkFileFormatInternalRowReaderContext.detectVectorColumnsFromMetadata(schema) private def replaceVectorFieldsWithBinary(schema: StructType, vectorCols: Map[Int, _]): StructType = @@ -419,7 +419,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, private def wrapWithVectorConversion(iter: Iterator[InternalRow], readSchema: StructType, targetSchema: StructType, - vectorCols: Map[Int, (Int, HoodieSchema.Vector.VectorElementType)]): Iterator[InternalRow] = { + vectorCols: Map[Int, HoodieSchema.Vector]): Iterator[InternalRow] = { val numFields = readSchema.fields.length val vectorProjection = UnsafeProjection.create(targetSchema) iter.map { row => @@ -428,8 +428,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, if (row.isNullAt(i)) { converted.setNullAt(i) } else if (vectorCols.contains(i)) { - val (dim, elemType) = vectorCols(i) - converted.update(i, SparkFileFormatInternalRowReaderContext.convertBinaryToVectorArray(row.getBinary(i), dim, elemType)) + converted.update(i, org.apache.hudi.io.storage.VectorConversionUtils.convertBinaryToVectorArray(row.getBinary(i), vectorCols(i))) } else { converted.update(i, row.get(i, readSchema.apply(i).dataType)) } @@ -456,7 +455,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, // Detect vector columns in the full output schema for post-read conversion. // Output schema may have different indices than requiredSchema (e.g. partition columns interleaved), // so we must detect separately here. - val outputVectorCols = if (hasVectors) detectVectorColumns(outputSchema) else Map.empty[Int, (Int, HoodieSchema.Vector.VectorElementType)] + val outputVectorCols = if (hasVectors) detectVectorColumns(outputSchema) else Map.empty[Int, HoodieSchema.Vector] val rawIter = if (remainingPartitionSchema.fields.length == partitionSchema.fields.length) { //none of partition fields are read from the file, so the reader will do the appending for us From 959bcd8a247da9beacbb06fb59fe75abbf939edd Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Tue, 17 Mar 2026 12:53:26 -0700 Subject: [PATCH 10/13] fix(vector): replace existential type Map[Int, _] with Map[Int, HoodieSchema.Vector] to fix Scala 2.12 type inference error --- .../apache/hudi/SparkFileFormatInternalRowReaderContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 02c52dfdd8daa..863855ccc35c3 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 @@ -331,7 +331,7 @@ object SparkFileFormatInternalRowReaderContext { * can read FIXED_LEN_BYTE_ARRAY data without type mismatch. * Delegates to [[VectorConversionUtils.replaceVectorColumnsWithBinary]]. */ - def replaceVectorColumnsWithBinary(structType: StructType, vectorColumns: Map[Int, _]): StructType = { + def replaceVectorColumnsWithBinary(structType: StructType, vectorColumns: Map[Int, HoodieSchema.Vector]): StructType = { val javaMap = vectorColumns.map { case (k, v) => (Integer.valueOf(k), v.asInstanceOf[AnyRef]) }.asJava VectorConversionUtils.replaceVectorColumnsWithBinary(structType, javaMap) } From f8ce228427149e83028f8b72a19e07a895a961d9 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 18 Mar 2026 16:25:33 -0700 Subject: [PATCH 11/13] minor comilation issue fix --- .../parquet/HoodieFileGroupReaderBasedFileFormat.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 fdec37e0bc68f..fe24b70d90626 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 @@ -409,7 +409,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, private def detectVectorColumns(schema: StructType): Map[Int, HoodieSchema.Vector] = SparkFileFormatInternalRowReaderContext.detectVectorColumnsFromMetadata(schema) - private def replaceVectorFieldsWithBinary(schema: StructType, vectorCols: Map[Int, _]): StructType = + private def replaceVectorFieldsWithBinary(schema: StructType, vectorCols: Map[Int, HoodieSchema.Vector]): StructType = SparkFileFormatInternalRowReaderContext.replaceVectorColumnsWithBinary(schema, vectorCols) /** From fd8fca7c20f60e2c4216b6a0e9f049cb544b9614 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 18 Mar 2026 17:21:49 -0700 Subject: [PATCH 12/13] further self review fixes --- .../io/storage/HoodieSparkParquetReader.java | 11 +-- .../io/storage/VectorConversionUtils.java | 29 ++++++ .../row/HoodieRowParquetWriteSupport.java | 6 +- ...rkFileFormatInternalRowReaderContext.scala | 17 +--- .../HoodieSchemaCompatibilityChecker.java | 2 + .../apache/hudi/internal/schema/Types.java | 4 + .../convert/TestInternalSchemaConverter.java | 55 +++++++++++ .../AvroSchemaConverterWithTimestampNTZ.java | 6 ++ ...HoodieFileGroupReaderBasedFileFormat.scala | 43 +++++---- .../functional/TestVectorDataSource.scala | 94 +++++++++++++++++++ 10 files changed, 226 insertions(+), 41 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index 8de9392410ed7..06948245ba057 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -206,16 +206,7 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc GenericInternalRow converted = new GenericInternalRow(numFields); CloseableMappingIterator vectorIterator = new CloseableMappingIterator<>(projectedIterator, row -> { - for (int i = 0; i < numFields; i++) { - if (row.isNullAt(i)) { - converted.setNullAt(i); - } else if (vectorColumnInfo.containsKey(i)) { - converted.update(i, VectorConversionUtils.convertBinaryToVectorArray(row.getBinary(i), vectorColumnInfo.get(i))); - } else { - // Non-vector column: copy value as-is using the read schema's data type - converted.update(i, row.get(i, finalReadSchema.apply(i).dataType())); - } - } + VectorConversionUtils.convertRowVectorColumns(row, converted, finalReadSchema, vectorColumnInfo); return vectorProjection.apply(converted); }); readerIterators.add(vectorIterator); 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 a2dba86ee4367..6cea8b7aba7d0 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 @@ -28,6 +28,9 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; + import java.nio.ByteBuffer; import java.util.HashMap; import java.util.List; @@ -173,4 +176,30 @@ public static GenericArrayData convertBinaryToVectorArray(byte[] bytes, int dim, "Unsupported vector element type: " + elemType); } } + + /** + * Converts vector columns in a row from binary (BinaryType) back to typed arrays, + * copying non-vector columns as-is. The caller must supply a pre-allocated + * {@link GenericInternalRow} for reuse across iterations to reduce GC pressure. + * + * @param row the source row (with BinaryType for vector columns) + * @param result a pre-allocated GenericInternalRow to write into (reused across calls) + * @param readSchema the Spark schema of the source row (BinaryType for vector columns) + * @param vectorColumns map of ordinal to Vector schema for vector columns + */ + public static void convertRowVectorColumns(InternalRow row, GenericInternalRow result, + StructType readSchema, + Map vectorColumns) { + int numFields = readSchema.fields().length; + for (int i = 0; i < numFields; i++) { + if (row.isNullAt(i)) { + result.setNullAt(i); + } else if (vectorColumns.containsKey(i)) { + result.update(i, convertBinaryToVectorArray(row.getBinary(i), vectorColumns.get(i))); + } else { + // Non-vector column: copy value as-is using the read schema's data type + result.update(i, row.get(i, readSchema.apply(i).dataType())); + } + } + } } 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 bca1892b0e015..17661a018befa 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 @@ -311,14 +311,14 @@ private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { && resolvedSchema.getType() == HoodieSchemaType.VECTOR) { HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) resolvedSchema; int dimension = vectorSchema.getDimension(); - int elementSize = vectorSchema.getVectorElementType().getElementSize(); HoodieSchema.Vector.VectorElementType elemType = vectorSchema.getVectorElementType(); + int bufferSize = Math.multiplyExact(dimension, elemType.getElementSize()); + ByteBuffer buffer = ByteBuffer.allocate(bufferSize).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER); return (row, ordinal) -> { ArrayData array = row.getArray(ordinal); ValidationUtils.checkArgument(array.numElements() == dimension, () -> String.format("Vector dimension mismatch: schema expects %d elements but got %d", dimension, array.numElements())); - int bufferSize = Math.multiplyExact(dimension, elementSize); - ByteBuffer buffer = ByteBuffer.allocate(bufferSize).order(HoodieSchema.VectorLogicalType.VECTOR_BYTE_ORDER); + buffer.clear(); switch (elemType) { case FLOAT: for (int i = 0; i < dimension; 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 863855ccc35c3..76c9542fddacf 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 @@ -345,22 +345,15 @@ object SparkFileFormatInternalRowReaderContext { vectorColumns: Map[Int, HoodieSchema.Vector], readSchema: StructType): ClosableIterator[InternalRow] = { val numFields = readSchema.fields.length + // Reuse a single GenericInternalRow across iterations to reduce GC pressure + val result = new GenericInternalRow(numFields) + val javaVectorColumns: java.util.Map[Integer, HoodieSchema.Vector] = + vectorColumns.map { case (k, v) => (Integer.valueOf(k), v) }.asJava new ClosableIterator[InternalRow] { override def hasNext: Boolean = iterator.hasNext override def next(): InternalRow = { val row = iterator.next() - val result = new GenericInternalRow(numFields) - var i = 0 - while (i < numFields) { - if (row.isNullAt(i)) { - result.setNullAt(i) - } else if (vectorColumns.contains(i)) { - result.update(i, VectorConversionUtils.convertBinaryToVectorArray(row.getBinary(i), vectorColumns(i))) - } else { - result.update(i, row.get(i, readSchema(i).dataType)) - } - i += 1 - } + VectorConversionUtils.convertRowVectorColumns(row, result, readSchema, javaVectorColumns) result } override def close(): Unit = iterator.close() diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibilityChecker.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibilityChecker.java index 726a75d561009..6897fc96ab1bb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibilityChecker.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibilityChecker.java @@ -460,6 +460,8 @@ private SchemaCompatibilityResult checkFixedSize(final HoodieSchema reader, fina return checkDecimalWidening(reader, writer, locations); } + // Convention: "expected" = writer (existing data), "found" = reader (evolved schema). + // This matches checkFixedSize, checkDecimalWidening, checkTimeCompatibility, etc. private SchemaCompatibilityResult checkVectorCompatibility(final HoodieSchema reader, final HoodieSchema writer, final Deque locations) { HoodieSchema.Vector readerVector = (HoodieSchema.Vector) reader; diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java index bcb6d32bf16e2..aaac7b51928db 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java @@ -18,6 +18,7 @@ package org.apache.hudi.internal.schema; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.internal.schema.Type.NestedType; import org.apache.hudi.internal.schema.Type.PrimitiveType; @@ -317,6 +318,9 @@ public static VectorType get(int dimension, String elementType, String storageBa } private VectorType(int dimension, String elementType, String storageBacking) { + // Validate that the strings correspond to known enum values to fail fast on typos + HoodieSchema.Vector.VectorElementType.fromString(elementType); + HoodieSchema.Vector.StorageBacking.fromString(storageBacking); this.dimension = dimension; this.elementType = elementType; this.storageBacking = storageBacking; diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestInternalSchemaConverter.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestInternalSchemaConverter.java index 1ec7927813f0a..5da64b4050ad1 100644 --- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestInternalSchemaConverter.java +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestInternalSchemaConverter.java @@ -22,6 +22,8 @@ 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.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Types; import org.junit.jupiter.api.Test; @@ -111,4 +113,57 @@ public void testCollectColumnNames() { assertEquals(expectedOutput.size(), fieldNames.size()); assertTrue(fieldNames.containsAll(expectedOutput)); } + + @Test + public void testVectorTypeRoundTrip() { + // Create a HoodieSchema with a VECTOR field + HoodieSchema vectorField = HoodieSchema.createVector(128, HoodieSchema.Vector.VectorElementType.FLOAT); + HoodieSchema recordSchema = HoodieSchema.createRecord("vectorRecord", null, null, Arrays.asList( + HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.STRING), null, null), + HoodieSchemaField.of("embedding", vectorField, null, null) + )); + + // HoodieSchema → InternalSchema + InternalSchema internalSchema = InternalSchemaConverter.convert(recordSchema); + + // Verify the InternalSchema has a VectorType + Types.VectorType internalVectorType = (Types.VectorType) internalSchema.findType("embedding"); + assertEquals(128, internalVectorType.getDimension()); + assertEquals("FLOAT", internalVectorType.getElementType()); + + // InternalSchema → HoodieSchema + HoodieSchema roundTripped = InternalSchemaConverter.convert(internalSchema, "vectorRecord"); + + // Verify vector properties survived the round-trip + HoodieSchemaField embeddingField = roundTripped.getFields().stream() + .filter(f -> f.name().equals("embedding")) + .findFirst().get(); + HoodieSchema embeddingSchema = embeddingField.schema().getNonNullType(); + assertEquals(HoodieSchemaType.VECTOR, embeddingSchema.getType()); + HoodieSchema.Vector roundTrippedVector = (HoodieSchema.Vector) embeddingSchema; + assertEquals(128, roundTrippedVector.getDimension()); + assertEquals(HoodieSchema.Vector.VectorElementType.FLOAT, roundTrippedVector.getVectorElementType()); + } + + @Test + public void testVectorTypeRoundTripDouble() { + HoodieSchema vectorField = HoodieSchema.createVector(64, HoodieSchema.Vector.VectorElementType.DOUBLE); + HoodieSchema recordSchema = HoodieSchema.createRecord("vectorDoubleRecord", null, null, Arrays.asList( + HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.INT), null, null), + HoodieSchemaField.of("vec", vectorField, null, null) + )); + + InternalSchema internalSchema = InternalSchemaConverter.convert(recordSchema); + Types.VectorType internalVec = (Types.VectorType) internalSchema.findType("vec"); + assertEquals(64, internalVec.getDimension()); + assertEquals("DOUBLE", internalVec.getElementType()); + + HoodieSchema roundTripped = InternalSchemaConverter.convert(internalSchema, "vectorDoubleRecord"); + HoodieSchemaField vecField = roundTripped.getFields().stream() + .filter(f -> f.name().equals("vec")) + .findFirst().get(); + HoodieSchema.Vector rtVec = (HoodieSchema.Vector) vecField.schema().getNonNullType(); + assertEquals(64, rtVec.getDimension()); + assertEquals(HoodieSchema.Vector.VectorElementType.DOUBLE, rtVec.getVectorElementType()); + } } diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java index 7611202ed3c9c..010a0b7055aea 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java @@ -249,6 +249,12 @@ private Type convertField(String fieldName, HoodieSchema schema, Type.Repetition } break; case VECTOR: + // Vectors are stored as bare FIXED_LEN_BYTE_ARRAY without a Parquet logical type annotation. + // Vector semantics (dimension, element type) are resolved from HoodieSchema (the table's + // stored schema), not from the Parquet file schema. The reverse direction + // (FIXED_LEN_BYTE_ARRAY → HoodieSchema) currently maps to generic FIXED; this is + // acceptable because the read path detects vectors from the HoodieSchema, not from Parquet. + // TODO: Consider adding VectorLogicalTypeAnnotation for fully self-describing Parquet files. HoodieSchema.Vector vectorSchema = (HoodieSchema.Vector) schema; int fixedSize = vectorSchema.getDimension() * vectorSchema.getVectorElementType().getElementSize(); 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 fe24b70d90626..ae7904b6d5a1b 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 @@ -37,10 +37,10 @@ import org.apache.hudi.io.IOUtils import org.apache.hudi.io.storage.HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR import org.apache.hudi.storage.StorageConfiguration import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.Job +import org.apache.hudi.io.storage.VectorConversionUtils import org.apache.parquet.schema.{HoodieSchemaRepair, MessageType} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging @@ -59,7 +59,6 @@ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnarBatchUtils} import org.apache.spark.util.SerializableConfiguration import java.io.Closeable - import scala.collection.JavaConverters.mapAsJavaMapConverter trait HoodieFormatTrait { @@ -116,6 +115,22 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, */ private var supportReturningBatch = false + /** + * Cached result of vector column detection keyed by schema identity. + * Avoids re-parsing metadata on repeated supportBatch / readBaseFile calls with the same schema. + */ + @transient private var cachedVectorDetection: (StructType, Map[Int, HoodieSchema.Vector]) = _ + + private def detectVectorColumnsCached(schema: StructType): Map[Int, HoodieSchema.Vector] = { + if (cachedVectorDetection != null && (cachedVectorDetection._1 eq schema)) { + cachedVectorDetection._2 + } else { + val result = detectVectorColumns(schema) + cachedVectorDetection = (schema, result) + result + } + } + /** * Checks if the file format supports vectorized reading, please refer to SPARK-40918. * @@ -130,7 +145,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { // Vector columns are stored as FIXED_LEN_BYTE_ARRAY in Parquet but read as ArrayType in Spark. // The binary→array conversion requires row-level access, so disable vectorized batch reading. - if (detectVectorColumns(schema).nonEmpty) { + if (detectVectorColumnsCached(schema).nonEmpty) { supportVectorizedRead = false supportReturningBatch = false false @@ -422,17 +437,12 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, vectorCols: Map[Int, HoodieSchema.Vector]): Iterator[InternalRow] = { val numFields = readSchema.fields.length val vectorProjection = UnsafeProjection.create(targetSchema) + // Reuse a single GenericInternalRow across iterations; vectorProjection.apply() copies the data + val converted = new GenericInternalRow(numFields) + val javaVectorCols: java.util.Map[Integer, HoodieSchema.Vector] = + vectorCols.map { case (k, v) => (Integer.valueOf(k), v) }.asJava iter.map { row => - val converted = new GenericInternalRow(numFields) - for (i <- 0 until numFields) { - if (row.isNullAt(i)) { - converted.setNullAt(i) - } else if (vectorCols.contains(i)) { - converted.update(i, org.apache.hudi.io.storage.VectorConversionUtils.convertBinaryToVectorArray(row.getBinary(i), vectorCols(i))) - } else { - converted.update(i, row.get(i, readSchema.apply(i).dataType)) - } - } + VectorConversionUtils.convertRowVectorColumns(row, converted, readSchema, javaVectorCols) vectorProjection.apply(converted).asInstanceOf[InternalRow] } } @@ -448,9 +458,10 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val modifiedRequiredSchema = if (hasVectors) replaceVectorFieldsWithBinary(requiredSchema, vectorCols) else requiredSchema val modifiedOutputSchema = if (hasVectors) replaceVectorFieldsWithBinary(outputSchema, vectorCols) else outputSchema - // requestedSchema = requiredSchema + mandatory partition fields appended at end, - // so vector columns are at the same indices as in requiredSchema — reuse vectorCols - val modifiedRequestedSchema = if (hasVectors) replaceVectorFieldsWithBinary(requestedSchema, vectorCols) else requestedSchema + // Detect from requestedSchema independently — don't assume ordinals match requiredSchema + // (requestedSchema = requiredSchema + mandatory partition fields; ordinals may differ) + val requestedVectorCols = if (hasVectors) detectVectorColumns(requestedSchema) else Map.empty[Int, HoodieSchema.Vector] + val modifiedRequestedSchema = if (hasVectors) replaceVectorFieldsWithBinary(requestedSchema, requestedVectorCols) else requestedSchema // Detect vector columns in the full output schema for post-read conversion. // Output schema may have different indices than requiredSchema (e.g. partition columns interleaved), 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 c206a07ba2a5d..07d2782dd5522 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 @@ -799,6 +799,100 @@ class TestVectorDataSource extends HoodieSparkClientTestBase { assertTrue(r7.getSeq[Double](1).forall(_ == 1.0), "key_7 should have original value 1.0") } + @Test + def testDimensionMismatchOnWrite(): Unit = { + // Schema declares VECTOR(8) but data has arrays of length 4 + val metadata = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(8)") + .build() + + val schema = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, metadata) + )) + + val data = Seq( + Row("key_1", Seq(1.0f, 2.0f, 3.0f, 4.0f)) // only 4 elements, schema says 8 + ) + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + schema + ) + + val ex = assertThrows(classOf[Exception], () => { + df.write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(TABLE_NAME.key, "dim_mismatch_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/dim_mismatch") + }) + // The root cause should mention dimension mismatch + var cause: Throwable = ex + var foundMismatch = false + while (cause != null && !foundMismatch) { + if (cause.getMessage != null && cause.getMessage.contains("dimension mismatch")) { + foundMismatch = true + } + cause = cause.getCause + } + assertTrue(foundMismatch, + s"Expected 'dimension mismatch' in exception chain, got: ${ex.getMessage}") + } + + @Test + def testSchemaEvolutionRejectsDimensionChange(): Unit = { + // Write initial table with VECTOR(4) + val metadata4 = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(4)") + .build() + + val schema4 = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, metadata4), + StructField("ts", LongType, nullable = false) + )) + + val data1 = Seq(Row("key_1", Seq(1.0f, 2.0f, 3.0f, 4.0f), 1L)) + spark.createDataFrame(spark.sparkContext.parallelize(data1), schema4) + .write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "schema_evolve_dim_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Overwrite) + .save(basePath + "/schema_evolve_dim") + + // Now try to write with VECTOR(8) — different dimension should be rejected + val metadata8 = new MetadataBuilder() + .putString(HoodieSchema.TYPE_METADATA_FIELD, "VECTOR(8)") + .build() + + val schema8 = StructType(Seq( + StructField("id", StringType, nullable = false), + StructField("embedding", ArrayType(FloatType, containsNull = false), + nullable = false, metadata8), + StructField("ts", LongType, nullable = false) + )) + + val data2 = Seq(Row("key_2", Seq(1.0f, 2.0f, 3.0f, 4.0f, 5.0f, 6.0f, 7.0f, 8.0f), 2L)) + + assertThrows(classOf[Exception], () => { + spark.createDataFrame(spark.sparkContext.parallelize(data2), schema8) + .write.format("hudi") + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(TABLE_NAME.key, "schema_evolve_dim_test") + .option(TABLE_TYPE.key, "COPY_ON_WRITE") + .mode(SaveMode.Append) + .save(basePath + "/schema_evolve_dim") + }) + } + private def assertArrayEquals(expected: Array[Byte], actual: Array[Byte], message: String): Unit = { assertEquals(expected.length, actual.length, s"$message: length mismatch") expected.zip(actual).zipWithIndex.foreach { case ((e, a), idx) => From 106fb29fab335f616c82195e610a1c6db50bbe19 Mon Sep 17 00:00:00 2001 From: Rahil Chertara Date: Wed, 18 Mar 2026 19:12:28 -0700 Subject: [PATCH 13/13] fix(vector): fix Scala import ordering style violations - Move VectorConversionUtils import into hudi group (was misplaced in 3rdParty) - Add blank line between hudi and 3rdParty import groups - Add blank line between java and scala import groups Co-Authored-By: Claude Opus 4.6 (1M context) --- .../parquet/HoodieFileGroupReaderBasedFileFormat.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 ae7904b6d5a1b..7468ea8b37077 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 @@ -35,12 +35,13 @@ import org.apache.hudi.exception.HoodieNotSupportedException import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.io.IOUtils import org.apache.hudi.io.storage.HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR +import org.apache.hudi.io.storage.VectorConversionUtils import org.apache.hudi.storage.StorageConfiguration import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.Job -import org.apache.hudi.io.storage.VectorConversionUtils import org.apache.parquet.schema.{HoodieSchemaRepair, MessageType} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging @@ -59,6 +60,7 @@ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnarBatchUtils} import org.apache.spark.util.SerializableConfiguration import java.io.Closeable + import scala.collection.JavaConverters.mapAsJavaMapConverter trait HoodieFormatTrait {