-
Notifications
You must be signed in to change notification settings - Fork 2.5k
feat(vector): Support writing VECTOR to parquet and avro formats using Spark #18328
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
c46bfb9
5b26607
d8a7dca
aa3be52
eff69bf
9fe1c02
8c555b8
35223ec
09a032c
959bcd8
f8ce228
fd8fca7
106fb29
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,205 @@ | ||
| /* | ||
| * 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 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; | ||
| 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<Integer, HoodieSchema.Vector> detectVectorColumns(HoodieSchema schema) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just checking, As we are using integer ordinal position in the schema, can you check if things end to end with projections and schema evolution?
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I believe i have tests for this in the pr but will check |
||
| Map<Integer, HoodieSchema.Vector> vectorColumnInfo = new HashMap<>(); | ||
| if (schema == null) { | ||
| return vectorColumnInfo; | ||
| } | ||
| List<HoodieSchemaField> 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<Integer, HoodieSchema.Vector> detectVectorColumnsFromMetadata(StructType schema) { | ||
| Map<Integer, HoodieSchema.Vector> 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<Integer, ?> 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); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * 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<Integer, HoodieSchema.Vector> 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())); | ||
| } | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -69,6 +69,7 @@ | |
| import org.apache.spark.unsafe.types.UTF8String; | ||
| import org.apache.spark.util.VersionUtils; | ||
|
|
||
| import java.nio.ByteBuffer; | ||
| import java.util.Arrays; | ||
| import java.util.Collections; | ||
| import java.util.HashMap; | ||
|
|
@@ -305,6 +306,40 @@ 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(); | ||
| 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())); | ||
| buffer.clear(); | ||
| 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())); | ||
| }; | ||
| } else if (dataType instanceof ArrayType) { | ||
| ValueWriter elementWriter = makeWriter(resolvedSchema == null ? null : resolvedSchema.getElementType(), ((ArrayType) dataType).elementType()); | ||
| if (!writeLegacyListFormat) { | ||
|
|
@@ -518,6 +553,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) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The vectors are stored as bare FIXED_LEN_BYTE_ARRAY in Parquet with no logical type annotation or key-value metadata on the Parquet column. I think it would be useful to track this. Any thoughts?
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @balaji-varadarajan-ai so you mean we want to keep track of the hudi type info around VECTOR within parquet itself? If so i think i can look into this. |
||
| .length(fixedSize).named(structField.name()); | ||
| } else if (dataType instanceof ArrayType) { | ||
| ArrayType arrayType = (ArrayType) dataType; | ||
| DataType elementType = arrayType.elementType(); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
seeing the pattern:
in HoodieSparkParquetReader, SparkFileFormatInternalRowReaderContext and HoodieFileGroupReaderBasedFileFormat. Wondering if you can bring them under one common method with specific callback.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can look into this