balaji-varadarajan-ai commented on code in PR #18328:
URL: https://github.com/apache/hudi/pull/18328#discussion_r2978541703


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java:
##########
@@ -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()));

Review Comment:
   Note: We reuse the same buffer for every row without copying. Make sure 
ColumnWriteStoreV2 doesn't hold references between writes, or consecutive rows 
will overwrite each other!
   The decimal path does this too so it's probably safe, but keep in mind  the 
vector buffer is much larger (~6KB vs ~16B).



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java:
##########
@@ -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()

Review Comment:
   Use Math.multiplyExact() here to prevent silent overflow with massive 
vectors!



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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 java.util.function.Function;
+
+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) {
+    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];

Review Comment:
   FYI: GenericArrayData(byte[]) is kinda inefficient — it actually boxes every 
byte into a Byte and stores it as an Object[]. So for a 1536-dim INT8 vector, 
that’s 1536 tiny allocations per row 😬. FLOAT/DOUBLE don’t have this issue 
since they use optimized primitive array constructors. If this becomes a 
bottleneck, consider using UnsafeArrayData.fromPrimitiveArray(int8s) to avoid 
all that boxing.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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 java.util.function.Function;
+
+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) {
+    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());

Review Comment:
   Is Metadata.empty() correct here ?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/VectorConversionUtils.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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 java.util.function.Function;
+
+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) {
+    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);
+    }
+  }
+
+  /**
+   * Returns a {@link Function} that converts a single {@link InternalRow} by 
converting binary
+   * vector columns back to typed arrays and then applying the given 
projection callback.
+   *
+   * <p>Ordinals in {@code vectorColumns} must be relative to {@code 
readSchema} — the schema
+   * that has {@code BinaryType} for vector columns (as produced by
+   * {@link #replaceVectorColumnsWithBinary}).
+   *
+   * <p><b>Thread safety:</b> The returned function is NOT thread-safe; it 
reuses a single
+   * {@link GenericInternalRow} buffer across calls. Each call to this factory 
creates its own
+   * buffer, so separate functions returned by separate calls are independent.
+   *
+   * @param readSchema         the Spark schema of incoming rows (BinaryType 
for vector columns)
+   * @param vectorColumns      map of ordinal → Vector schema for vector 
columns, keyed by
+   *                           ordinals relative to {@code readSchema}
+   * @param projectionCallback called with the converted {@link 
GenericInternalRow}; must copy
+   *                           any data it needs to retain (e.g. {@code 
UnsafeProjection::apply})
+   * @return a function that converts one row and returns the projected result
+   */
+  public static Function<InternalRow, InternalRow> buildRowMapper(
+      StructType readSchema,
+      Map<Integer, HoodieSchema.Vector> vectorColumns,
+      Function<GenericInternalRow, InternalRow> projectionCallback) {
+    GenericInternalRow converted = new 
GenericInternalRow(readSchema.fields().length);
+    return row -> {
+      convertRowVectorColumns(row, converted, readSchema, vectorColumns);
+      return projectionCallback.apply(converted);
+    };
+  }
+
+  /**
+   * 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()));

Review Comment:
    instead of copying every field through GenericInternalRow, consider a 
byte-level copy of the UnsafeRow with surgical replacement of only the vector 
column offsets. It might be faster. You can separately micro benchmark this and 
test. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to