From 08ee7d4f0946b394036509b0800f4dd4fc34b12b Mon Sep 17 00:00:00 2001 From: ColdL Date: Thu, 29 Jan 2026 10:26:08 +0800 Subject: [PATCH 1/4] add vector type definition, add APIs in DataGetters/DataTypeVisitor, add basic impls --- .../apache/paimon/types/DataTypeChecks.java | 5 + .../paimon/types/DataTypeDefaultVisitor.java | 5 + .../paimon/types/DataTypeJsonParser.java | 18 + .../org/apache/paimon/types/DataTypeRoot.java | 2 + .../apache/paimon/types/DataTypeVisitor.java | 2 + .../org/apache/paimon/types/DataTypes.java | 9 + .../apache/paimon/types/ReassignFieldId.java | 8 + .../org/apache/paimon/types/VectorType.java | 180 ++++++++ .../arrow/ArrowFieldTypeConversion.java | 7 + .../Arrow2PaimonVectorConverter.java | 6 + .../ArrowFieldWriterFactoryVisitor.java | 6 + .../apache/paimon/codegen/GenerateUtils.scala | 18 +- .../paimon/codegen/ScalarOperatorGens.scala | 11 +- .../codegen/EqualiserCodeGeneratorTest.java | 11 + .../org/apache/paimon/PartitionSettedRow.java | 8 + .../apache/paimon/casting/CastedArray.java | 6 + .../org/apache/paimon/casting/CastedRow.java | 6 + .../apache/paimon/casting/CastedVector.java | 113 +++++ .../paimon/casting/DefaultValueRow.java | 9 + .../paimon/casting/FallbackMappingRow.java | 9 + .../paimon/data/AbstractBinaryWriter.java | 36 ++ .../org/apache/paimon/data/BinaryArray.java | 5 + .../org/apache/paimon/data/BinaryRow.java | 6 + .../org/apache/paimon/data/BinaryVector.java | 393 ++++++++++++++++++ .../org/apache/paimon/data/BinaryWriter.java | 14 + .../org/apache/paimon/data/DataGetters.java | 2 + .../org/apache/paimon/data/GenericArray.java | 5 + .../org/apache/paimon/data/GenericRow.java | 5 + .../org/apache/paimon/data/InternalRow.java | 3 + .../apache/paimon/data/InternalVector.java | 90 ++++ .../org/apache/paimon/data/JoinedRow.java | 9 + .../apache/paimon/data/LazyGenericRow.java | 5 + .../org/apache/paimon/data/NestedRow.java | 5 + .../paimon/data/columnar/ColumnarArray.java | 6 + .../paimon/data/columnar/ColumnarRow.java | 6 + .../data/columnar/RowToColumnConverter.java | 6 + .../data/columnar/VectorizedColumnBatch.java | 5 + .../paimon/data/safe/SafeBinaryArray.java | 6 + .../paimon/data/safe/SafeBinaryRow.java | 6 + .../data/serializer/InternalSerializers.java | 5 + .../serializer/InternalVectorSerializer.java | 113 +++++ .../fileindex/bitmap/BitmapTypeVisitor.java | 6 + .../fileindex/bloomfilter/FastHash.java | 6 + .../paimon/memory/MemorySegmentUtils.java | 20 + .../paimon/reader/DataEvolutionArray.java | 6 + .../paimon/reader/DataEvolutionRow.java | 6 + .../paimon/sort/hilbert/HilbertIndexer.java | 6 + .../apache/paimon/sort/zorder/ZIndexer.java | 6 + .../types/InternalRowToSizeVisitor.java | 23 + .../apache/paimon/utils/InternalRowUtils.java | 2 + .../apache/paimon/utils/KeyProjectedRow.java | 6 + .../apache/paimon/utils/ProjectedArray.java | 8 +- .../org/apache/paimon/utils/ProjectedRow.java | 6 + .../apache/paimon/utils/ProjectedVector.java | 128 ++++++ .../apache/paimon/utils/TypeCheckUtils.java | 7 + .../org/apache/paimon/utils/TypeUtils.java | 19 + .../paimon/utils/VectorMappingUtils.java | 6 + .../org/apache/paimon/data/BinaryRowTest.java | 34 ++ .../apache/paimon/data/BinaryVectorTest.java | 119 ++++++ .../InternalVectorSerializerTest.java | 92 ++++ .../datagen/RandomGeneratorVisitor.java | 6 + .../paimon/types/DataTypeChecksTest.java | 7 + .../apache/paimon/utils/TypeUtilsTest.java | 9 + .../paimon/stats/SimpleStatsEvolution.java | 6 + .../org/apache/paimon/utils/OffsetRow.java | 6 + .../org/apache/paimon/utils/PartialRow.java | 6 + .../paimon/append/VectorTypeTableTest.java | 97 +++++ .../paimon/codegen/CodeGenUtilsTest.java | 19 + .../paimon/schema/DataTypeJsonParserTest.java | 10 + .../paimon/flink/DataTypeToLogicalType.java | 6 + .../apache/paimon/flink/FlinkRowWrapper.java | 11 + .../paimon/format/json/JsonFileFormat.java | 1 + .../paimon/format/json/JsonFileReader.java | 11 + .../paimon/format/json/JsonFormatWriter.java | 13 + .../format/orc/writer/FieldWriterFactory.java | 6 + .../reader/ParquetVectorUpdaterFactory.java | 6 + .../filter2/predicate/ParquetFilters.java | 6 + .../format/json/JsonFileFormatTest.java | 16 + .../hive/objectinspector/HivePaimonArray.java | 6 + .../paimon/format/lance/LanceFileFormat.java | 6 + .../paimon/spark/SparkInternalRowWrapper.java | 11 + .../org/apache/paimon/spark/SparkRow.java | 11 + 82 files changed, 1944 insertions(+), 8 deletions(-) create mode 100644 paimon-api/src/main/java/org/apache/paimon/types/VectorType.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/casting/CastedVector.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/BinaryVector.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/InternalVector.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalVectorSerializer.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/utils/ProjectedVector.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/data/BinaryVectorTest.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/data/serializer/InternalVectorSerializerTest.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/append/VectorTypeTableTest.java diff --git a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeChecks.java b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeChecks.java index 20eed4d03842..5f11c2e95d7b 100644 --- a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeChecks.java +++ b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeChecks.java @@ -240,6 +240,11 @@ public List visit(ArrayType arrayType) { return Collections.singletonList(arrayType.getElementType()); } + @Override + public List visit(VectorType vectorType) { + return Collections.singletonList(vectorType.getElementType()); + } + @Override public List visit(MultisetType multisetType) { return Collections.singletonList(multisetType.getElementType()); diff --git a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeDefaultVisitor.java b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeDefaultVisitor.java index b3dc8a3cd994..af680ede62e2 100644 --- a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeDefaultVisitor.java +++ b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeDefaultVisitor.java @@ -124,6 +124,11 @@ public R visit(ArrayType arrayType) { return defaultMethod(arrayType); } + @Override + public R visit(VectorType vectorType) { + return defaultMethod(vectorType); + } + @Override public R visit(MultisetType multisetType) { return defaultMethod(multisetType); diff --git a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeJsonParser.java b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeJsonParser.java index 13637265d45f..4079dd8c47c0 100644 --- a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeJsonParser.java +++ b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeJsonParser.java @@ -78,6 +78,10 @@ public static DataType parseDataType(JsonNode json, AtomicInteger fieldId) { if (typeString.startsWith("ARRAY")) { DataType element = parseDataType(json.get("element"), fieldId); return new ArrayType(!typeString.contains("NOT NULL"), element); + } else if (typeString.startsWith("VECTOR")) { + DataType element = parseDataType(json.get("element"), fieldId); + int length = json.get("length").asInt(); + return new VectorType(!typeString.contains("NOT NULL"), length, element); } else if (typeString.startsWith("MULTISET")) { DataType element = parseDataType(json.get("element"), fieldId); return new MultisetType(!typeString.contains("NOT NULL"), element); @@ -318,6 +322,7 @@ private enum Keyword { SECOND, TO, ARRAY, + VECTOR, MULTISET, MAP, ROW, @@ -544,6 +549,8 @@ private DataType parseTypeByKeyword() { return new VariantType(); case BLOB: return new BlobType(); + case VECTOR: + return parseVectorType(); default: throw parsingError("Unsupported type: " + token().value); } @@ -665,5 +672,16 @@ private int parseOptionalPrecision(int defaultPrecision) { } return precision; } + + private DataType parseVectorType() { + // VECTOR + nextToken(TokenType.BEGIN_SUBTYPE); + DataType elementType = parseTypeWithNullability(); + nextToken(TokenType.LIST_SEPARATOR); + nextToken(TokenType.LITERAL_INT); + int length = tokenAsInt(); + nextToken(TokenType.END_SUBTYPE); + return DataTypes.VECTOR(length, elementType); + } } } diff --git a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeRoot.java b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeRoot.java index 1b339765986c..f55da9c4706f 100644 --- a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeRoot.java +++ b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeRoot.java @@ -106,6 +106,8 @@ public enum DataTypeRoot { ARRAY(DataTypeFamily.CONSTRUCTED, DataTypeFamily.COLLECTION), + VECTOR(DataTypeFamily.CONSTRUCTED, DataTypeFamily.COLLECTION), + MULTISET(DataTypeFamily.CONSTRUCTED, DataTypeFamily.COLLECTION), MAP(DataTypeFamily.CONSTRUCTED, DataTypeFamily.EXTENSION), diff --git a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeVisitor.java b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeVisitor.java index cdeb4204b17c..6e377309f237 100644 --- a/paimon-api/src/main/java/org/apache/paimon/types/DataTypeVisitor.java +++ b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeVisitor.java @@ -68,6 +68,8 @@ public interface DataTypeVisitor { R visit(ArrayType arrayType); + R visit(VectorType vectorType); + R visit(MultisetType multisetType); R visit(MapType mapType); diff --git a/paimon-api/src/main/java/org/apache/paimon/types/DataTypes.java b/paimon-api/src/main/java/org/apache/paimon/types/DataTypes.java index 33a5c9e5e442..39b180651ef5 100644 --- a/paimon-api/src/main/java/org/apache/paimon/types/DataTypes.java +++ b/paimon-api/src/main/java/org/apache/paimon/types/DataTypes.java @@ -59,6 +59,10 @@ public static ArrayType ARRAY(DataType element) { return new ArrayType(element); } + public static VectorType VECTOR(int length, DataType element) { + return new VectorType(length, element); + } + public static CharType CHAR(int length) { return new CharType(length); } @@ -221,6 +225,11 @@ public OptionalInt visit(VarBinaryType varBinaryType) { return OptionalInt.of(varBinaryType.getLength()); } + @Override + public OptionalInt visit(VectorType vectorType) { + return OptionalInt.of(vectorType.getLength()); + } + @Override protected OptionalInt defaultMethod(DataType dataType) { return OptionalInt.empty(); diff --git a/paimon-api/src/main/java/org/apache/paimon/types/ReassignFieldId.java b/paimon-api/src/main/java/org/apache/paimon/types/ReassignFieldId.java index 2aacfeaf889a..85f104d99320 100644 --- a/paimon-api/src/main/java/org/apache/paimon/types/ReassignFieldId.java +++ b/paimon-api/src/main/java/org/apache/paimon/types/ReassignFieldId.java @@ -38,6 +38,14 @@ public DataType visit(ArrayType arrayType) { return new ArrayType(arrayType.isNullable(), arrayType.getElementType().accept(this)); } + @Override + public DataType visit(VectorType vectorType) { + return new VectorType( + vectorType.isNullable(), + vectorType.getLength(), + vectorType.getElementType().accept(this)); + } + @Override public DataType visit(MultisetType multisetType) { return new MultisetType( diff --git a/paimon-api/src/main/java/org/apache/paimon/types/VectorType.java b/paimon-api/src/main/java/org/apache/paimon/types/VectorType.java new file mode 100644 index 000000000000..a98bd6fa30d5 --- /dev/null +++ b/paimon-api/src/main/java/org/apache/paimon/types/VectorType.java @@ -0,0 +1,180 @@ +/* + * 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.paimon.types; + +import org.apache.paimon.annotation.Public; +import org.apache.paimon.utils.Preconditions; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; + +import java.io.IOException; +import java.util.Objects; +import java.util.Set; + +/** + * Data type of fixed-size vector type. The elements are densely stored. + * + * @since 2.0.0 + */ +@Public +public class VectorType extends DataType { + + private static final long serialVersionUID = 1L; + + public static final int MIN_LENGTH = 1; + + public static final int MAX_LENGTH = Integer.MAX_VALUE; + + public static final String FORMAT = "VECTOR<%s, %d>"; + + private final DataType elementType; + + private final int length; + + public VectorType(boolean isNullable, int length, DataType elementType) { + super(isNullable, DataTypeRoot.VECTOR); + // TODO: should we support nullable for vector type? + // Preconditions.checkArgument(!isNullable, "Nullable is not supported for VectorType."); + this.elementType = + Preconditions.checkNotNull(elementType, "Element type must not be null."); + Preconditions.checkArgument( + isValidElementType(elementType), "Invalid element type for vector: " + elementType); + if (length < MIN_LENGTH) { + throw new IllegalArgumentException( + String.format( + "Vector length must be between %d and %d (both inclusive).", + MIN_LENGTH, MAX_LENGTH)); + } + this.length = length; + } + + public VectorType(int length, DataType elementType) { + this(false, length, elementType); // For vector type we prefer NOT NULL + } + + public int getLength() { + return length; + } + + public DataType getElementType() { + return elementType; + } + + public static boolean isValidElementType(DataType elementType) { + switch (elementType.getTypeRoot()) { + case BOOLEAN: + case TINYINT: + case SMALLINT: + case INTEGER: + case BIGINT: + case FLOAT: + case DOUBLE: + return true; + default: + return false; + } + } + + @Override + public int defaultSize() { + return elementType.defaultSize() * length; + } + + @Override + public DataType copy(boolean isNullable) { + return new VectorType(isNullable, length, elementType.copy()); + } + + @Override + public String asSQLString() { + return withNullability(FORMAT, elementType.asSQLString(), length); + } + + @Override + public void serializeJson(JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField("type", isNullable() ? "VECTOR" : "VECTOR NOT NULL"); + generator.writeFieldName("element"); + elementType.serializeJson(generator); + generator.writeFieldName("length"); + generator.writeNumber(length); + generator.writeEndObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + VectorType vectorType = (VectorType) o; + return elementType.equals(vectorType.elementType) && length == vectorType.length; + } + + @Override + public boolean equalsIgnoreFieldId(DataType o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + VectorType vectorType = (VectorType) o; + return elementType.equalsIgnoreFieldId(vectorType.elementType) + && length == vectorType.length; + } + + @Override + public boolean isPrunedFrom(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + VectorType vectorType = (VectorType) o; + return elementType.isPrunedFrom(vectorType.elementType); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), elementType, length); + } + + @Override + public R accept(DataTypeVisitor visitor) { + return visitor.visit(this); + } + + @Override + public void collectFieldIds(Set fieldIds) { + elementType.collectFieldIds(fieldIds); + } +} diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowFieldTypeConversion.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowFieldTypeConversion.java index e6bc1281a74d..33defc8f9a01 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowFieldTypeConversion.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowFieldTypeConversion.java @@ -42,6 +42,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.arrow.vector.types.TimeUnit; import org.apache.arrow.vector.types.Types; @@ -179,6 +180,12 @@ public FieldType visit(ArrayType arrayType) { return new FieldType(arrayType.isNullable(), Types.MinorType.LIST.getType(), null); } + @Override + public FieldType visit(VectorType vectorType) { + ArrowType arrowType = new ArrowType.FixedSizeList(vectorType.getLength()); + return new FieldType(vectorType.isNullable(), arrowType, null); + } + @Override public FieldType visit(MultisetType multisetType) { throw new UnsupportedOperationException("Doesn't support MultisetType."); diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/Arrow2PaimonVectorConverter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/Arrow2PaimonVectorConverter.java index 4afd976c1747..c78aa324ae27 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/Arrow2PaimonVectorConverter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/Arrow2PaimonVectorConverter.java @@ -66,6 +66,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; @@ -482,6 +483,11 @@ public ColumnVector getColumnVector() { }; } + @Override + public Arrow2PaimonVectorConverter visit(VectorType vectorType) { + throw new UnsupportedOperationException("Doesn't support VectorType."); + } + @Override public Arrow2PaimonVectorConverter visit(MultisetType multisetType) { throw new UnsupportedOperationException("Doesn't support MultisetType."); diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriterFactoryVisitor.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriterFactoryVisitor.java index a20e6fc4814c..d07762106ebc 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriterFactoryVisitor.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriterFactoryVisitor.java @@ -41,6 +41,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.complex.ListVector; @@ -165,6 +166,11 @@ public ArrowFieldWriterFactory visit(ArrayType arrayType) { isNullable); } + @Override + public ArrowFieldWriterFactory visit(VectorType vectorType) { + throw new UnsupportedOperationException("Doesn't support VectorType."); + } + @Override public ArrowFieldWriterFactory visit(MultisetType multisetType) { throw new UnsupportedOperationException("Doesn't support MultisetType."); diff --git a/paimon-codegen/src/main/scala/org/apache/paimon/codegen/GenerateUtils.scala b/paimon-codegen/src/main/scala/org/apache/paimon/codegen/GenerateUtils.scala index 6897e5424857..967d58ad30db 100644 --- a/paimon-codegen/src/main/scala/org/apache/paimon/codegen/GenerateUtils.scala +++ b/paimon-codegen/src/main/scala/org/apache/paimon/codegen/GenerateUtils.scala @@ -127,10 +127,13 @@ object GenerateUtils { s"$sortUtil.compareBinary($leftTerm, $rightTerm)" case TINYINT | SMALLINT | INTEGER | BIGINT | FLOAT | DOUBLE | DATE | TIME_WITHOUT_TIME_ZONE => s"($leftTerm > $rightTerm ? 1 : $leftTerm < $rightTerm ? -1 : 0)" - case ARRAY => - val at = t.asInstanceOf[ArrayType] + case ARRAY | VECTOR => + val elementType = t.getTypeRoot match { + case ARRAY => t.asInstanceOf[ArrayType].getElementType + case VECTOR => t.asInstanceOf[VectorType].getElementType + } val compareFunc = newName("compareArray") - val compareCode = generateArrayCompare(ctx, nullsIsLast = false, at, "a", "b") + val compareCode = generateArrayCompare(ctx, nullsIsLast = false, elementType, "a", "b") val funcCode: String = s""" public int $compareFunc($ARRAY_DATA a, $ARRAY_DATA b) { @@ -188,11 +191,10 @@ object GenerateUtils { def generateArrayCompare( ctx: CodeGeneratorContext, nullsIsLast: Boolean, - arrayType: ArrayType, + elementType: DataType, leftTerm: String, rightTerm: String): String = { val nullIsLastRet = if (nullsIsLast) 1 else -1 - val elementType = arrayType.getElementType val fieldA = newName("fieldA") val isNullA = newName("isNullA") val lengthA = newName("lengthA") @@ -379,6 +381,7 @@ object GenerateUtils { case DOUBLE => className[JDouble] case TIMESTAMP_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE => className[Timestamp] case ARRAY => className[InternalArray] + case VECTOR => className[InternalVector] case MULTISET | MAP => className[InternalMap] case ROW => className[InternalRow] case VARIANT => className[Variant] @@ -417,6 +420,8 @@ object GenerateUtils { s"$rowTerm.getTimestamp($indexTerm, ${getPrecision(t)})" case ARRAY => s"$rowTerm.getArray($indexTerm)" + case VECTOR => + s"$rowTerm.getVector($indexTerm)" case MULTISET | MAP => s"$rowTerm.getMap($indexTerm)" case ROW => @@ -606,6 +611,9 @@ object GenerateUtils { case ARRAY => val ser = addSerializer(t) s"$writerTerm.writeArray($indexTerm, $fieldValTerm, $ser)" + case VECTOR => + val ser = addSerializer(t) + s"$writerTerm.writeVector($indexTerm, $fieldValTerm, $ser)" case MULTISET | MAP => val ser = addSerializer(t) s"$writerTerm.writeMap($indexTerm, $fieldValTerm, $ser)" diff --git a/paimon-codegen/src/main/scala/org/apache/paimon/codegen/ScalarOperatorGens.scala b/paimon-codegen/src/main/scala/org/apache/paimon/codegen/ScalarOperatorGens.scala index 5dfa4bff6835..46217b1c4cf1 100644 --- a/paimon-codegen/src/main/scala/org/apache/paimon/codegen/ScalarOperatorGens.scala +++ b/paimon-codegen/src/main/scala/org/apache/paimon/codegen/ScalarOperatorGens.scala @@ -60,7 +60,13 @@ object ScalarOperatorGens { } // array types else if (isArray(left.resultType) && canEqual) { - generateArrayComparison(ctx, left, right, resultType) + val elementType = left.resultType.asInstanceOf[ArrayType].getElementType + generateArrayComparison(ctx, left, right, elementType, resultType) + } + // vector type + else if (isVector(left.resultType) && canEqual) { + val elementType = left.resultType.asInstanceOf[VectorType].getElementType + generateArrayComparison(ctx, left, right, elementType, resultType) } // map types else if (isMap(left.resultType) && canEqual) { @@ -196,6 +202,7 @@ object ScalarOperatorGens { ctx: CodeGeneratorContext, left: GeneratedExpression, right: GeneratedExpression, + elementType: DataType, resultType: DataType): GeneratedExpression = { generateCallWithStmtIfArgsNotNull(ctx, resultType, Seq(left, right)) { args => @@ -204,7 +211,6 @@ object ScalarOperatorGens { val resultTerm = newName("compareResult") - val elementType = left.resultType.asInstanceOf[ArrayType].getElementType val elementCls = primitiveTypeTermForType(elementType) val elementDefault = primitiveDefaultValue(elementType) @@ -225,6 +231,7 @@ object ScalarOperatorGens { rightElementExpr, new BooleanType(elementType.isNullable)) + // TODO: With BinaryVector available, we can use it here. val stmt = s""" |boolean $resultTerm; diff --git a/paimon-codegen/src/test/java/org/apache/paimon/codegen/EqualiserCodeGeneratorTest.java b/paimon-codegen/src/test/java/org/apache/paimon/codegen/EqualiserCodeGeneratorTest.java index e662449858e7..7e977291e81f 100644 --- a/paimon-codegen/src/test/java/org/apache/paimon/codegen/EqualiserCodeGeneratorTest.java +++ b/paimon-codegen/src/test/java/org/apache/paimon/codegen/EqualiserCodeGeneratorTest.java @@ -30,6 +30,7 @@ import org.apache.paimon.data.serializer.InternalArraySerializer; import org.apache.paimon.data.serializer.InternalMapSerializer; import org.apache.paimon.data.serializer.InternalRowSerializer; +import org.apache.paimon.data.serializer.InternalVectorSerializer; import org.apache.paimon.data.serializer.Serializer; import org.apache.paimon.data.variant.GenericVariant; import org.apache.paimon.types.DataType; @@ -133,6 +134,16 @@ public class EqualiserCodeGeneratorTest { castFromString("[1,2,3]", DataTypes.ARRAY(new VarCharType())), castFromString("[4,5,6]", DataTypes.ARRAY(new VarCharType()))), new InternalArraySerializer(DataTypes.VARCHAR(1)))); + TEST_DATA.put( + DataTypeRoot.VECTOR, + new GeneratedData( + DataTypes.VECTOR(3, DataTypes.FLOAT()), + Pair.of( + castFromString( + "[1.1,2.2,3.3]", DataTypes.VECTOR(3, DataTypes.FLOAT())), + castFromString( + "[4.4,5.5,6.6]", DataTypes.VECTOR(3, DataTypes.FLOAT()))), + new InternalVectorSerializer(DataTypes.FLOAT(), 3))); TEST_DATA.put( DataTypeRoot.MULTISET, new GeneratedData( diff --git a/paimon-common/src/main/java/org/apache/paimon/PartitionSettedRow.java b/paimon-common/src/main/java/org/apache/paimon/PartitionSettedRow.java index 01b3ae48a2da..a62464d1981f 100644 --- a/paimon-common/src/main/java/org/apache/paimon/PartitionSettedRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/PartitionSettedRow.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; @@ -176,6 +177,13 @@ public InternalArray getArray(int pos) { : row.getArray(partitionInfo.getRealIndex(pos)); } + @Override + public InternalVector getVector(int pos) { + return partitionInfo.inPartitionRow(pos) + ? partition.getVector(partitionInfo.getRealIndex(pos)) + : row.getVector(partitionInfo.getRealIndex(pos)); + } + @Override public InternalMap getMap(int pos) { return partitionInfo.inPartitionRow(pos) diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastedArray.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastedArray.java index 821be67e2a55..4e95c9db8dcf 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/CastedArray.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastedArray.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; @@ -201,6 +202,11 @@ public InternalArray getArray(int pos) { return castElementGetter.getElementOrNull(array, pos); } + @Override + public InternalVector getVector(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + @Override public InternalMap getMap(int pos) { return castElementGetter.getElementOrNull(array, pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastedRow.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastedRow.java index a60ea635cf45..76a1366d4784 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/CastedRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastedRow.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; @@ -148,6 +149,11 @@ public InternalArray getArray(int pos) { return castMapping[pos].getFieldOrNull(row); } + @Override + public InternalVector getVector(int pos) { + return castMapping[pos].getFieldOrNull(row); + } + @Override public InternalMap getMap(int pos) { return castMapping[pos].getFieldOrNull(row); diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastedVector.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastedVector.java new file mode 100644 index 000000000000..c7bf0303467a --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastedVector.java @@ -0,0 +1,113 @@ +/* + * 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.paimon.casting; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.data.variant.Variant; + +/** + * An implementation of {@link InternalVector} which provides a casted view of the underlying {@link + * InternalVector}. + * + *

It reads data from underlying {@link InternalVector} according to source logical type and + * casts it with specific {@link CastExecutor}. + */ +public class CastedVector extends CastedArray implements InternalVector { + + protected CastedVector(CastElementGetter castElementGetter) { + super(castElementGetter); + } + + /** + * Replaces the underlying {@link InternalVector} backing this {@link CastedVector}. + * + *

This method replaces the vector in place and does not return a new object. This is done + * for performance reasons. + */ + public static CastedVector from(CastElementGetter castElementGetter) { + return new CastedVector(castElementGetter); + } + + public CastedVector replaceVector(InternalVector vector) { + super.replaceArray(vector); + return this; + } + + @Override + public CastedArray replaceArray(InternalArray array) { + throw new IllegalArgumentException("CastedVector does not support replaceArray."); + } + + @Override + public BinaryString getString(int pos) { + throw new UnsupportedOperationException("CastedVector does not support String."); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + throw new UnsupportedOperationException("CastedVector does not support Decimal."); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + throw new UnsupportedOperationException("CastedVector does not support Timestamp."); + } + + @Override + public byte[] getBinary(int pos) { + throw new UnsupportedOperationException("CastedVector does not support Binary."); + } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("CastedVector does not support Variant."); + } + + @Override + public Blob getBlob(int pos) { + throw new UnsupportedOperationException("CastedVector does not support Blob."); + } + + @Override + public InternalArray getArray(int pos) { + throw new UnsupportedOperationException("CastedVector does not support nested Array."); + } + + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("CastedVector does not support nested VectorType."); + } + + @Override + public InternalMap getMap(int pos) { + throw new UnsupportedOperationException("CastedVector does not support Map."); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + throw new UnsupportedOperationException("CastedVector does not support nested Row."); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/DefaultValueRow.java b/paimon-common/src/main/java/org/apache/paimon/casting/DefaultValueRow.java index 25b7453861dd..555f065c7031 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/DefaultValueRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/DefaultValueRow.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.DataField; @@ -176,6 +177,14 @@ public InternalArray getArray(int pos) { return defaultValueRow.getArray(pos); } + @Override + public InternalVector getVector(int pos) { + if (!row.isNullAt(pos)) { + return row.getVector(pos); + } + return defaultValueRow.getVector(pos); + } + @Override public InternalMap getMap(int pos) { if (!row.isNullAt(pos)) { diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/FallbackMappingRow.java b/paimon-common/src/main/java/org/apache/paimon/casting/FallbackMappingRow.java index b4bf853230fc..b981d552876d 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/FallbackMappingRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/FallbackMappingRow.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; @@ -174,6 +175,14 @@ public InternalArray getArray(int pos) { return main.getArray(pos); } + @Override + public InternalVector getVector(int pos) { + if (mappings[pos] != -1 && main.isNullAt(pos)) { + return fallbackRow.getVector(mappings[pos]); + } + return main.getVector(pos); + } + @Override public InternalMap getMap(int pos) { if (mappings[pos] != -1 && main.isNullAt(pos)) { diff --git a/paimon-common/src/main/java/org/apache/paimon/data/AbstractBinaryWriter.java b/paimon-common/src/main/java/org/apache/paimon/data/AbstractBinaryWriter.java index 85d044594851..bb632d5b31ed 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/AbstractBinaryWriter.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/AbstractBinaryWriter.java @@ -21,6 +21,7 @@ import org.apache.paimon.data.serializer.InternalArraySerializer; import org.apache.paimon.data.serializer.InternalMapSerializer; import org.apache.paimon.data.serializer.InternalRowSerializer; +import org.apache.paimon.data.serializer.InternalVectorSerializer; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.memory.MemorySegment; import org.apache.paimon.memory.MemorySegmentUtils; @@ -91,6 +92,12 @@ public void writeArray(int pos, InternalArray input, InternalArraySerializer ser pos, binary.getSegments(), binary.getOffset(), binary.getSizeInBytes()); } + @Override + public void writeVector(int pos, InternalVector input, InternalVectorSerializer serializer) { + BinaryVector binary = serializer.toBinaryVector(input); + writeVectorToVarLenPart(pos, binary); + } + @Override public void writeMap(int pos, InternalMap input, InternalMapSerializer serializer) { BinaryMap binary = serializer.toBinaryMap(input); @@ -270,6 +277,35 @@ private void writeBytesToVarLenPart(int pos, byte[] bytes, int offset, int len) cursor += roundedSize; } + private void writeVectorToVarLenPart(int pos, BinaryVector vector) { + // Memory layout: [numElements][segments] + final int numElementsWidth = 4; + final int size = vector.getSizeInBytes(); + + final int roundedSize = roundNumberOfBytesToNearestWord(size + numElementsWidth); + + // grow the global buffer before writing data. + ensureCapacity(roundedSize); + + zeroOutPaddingBytes(size + numElementsWidth); + + // write numElements value first + segment.putInt(cursor, vector.size()); + cursor += numElementsWidth; + + // then vector values + if (vector.getSegments().length == 1) { + vector.getSegments()[0].copyTo(vector.getOffset(), segment, cursor, size); + } else { + writeMultiSegmentsToVarLenPart(vector.getSegments(), vector.getOffset(), size); + } + + setOffsetAndSize(pos, cursor - numElementsWidth, size + numElementsWidth); + + // move the cursor forward. + cursor += (roundedSize - numElementsWidth); + } + /** Increases the capacity to ensure that it can hold at least the minimum capacity argument. */ private void grow(int minCapacity) { int oldCapacity = segment.size(); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BinaryArray.java b/paimon-common/src/main/java/org/apache/paimon/data/BinaryArray.java index d8b649d4a64c..afac3f8d3d35 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BinaryArray.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BinaryArray.java @@ -256,6 +256,11 @@ public InternalArray getArray(int pos) { return MemorySegmentUtils.readArrayData(segments, offset, getLong(pos)); } + @Override + public InternalVector getVector(int pos) { + throw new IllegalArgumentException("Unsupported type: VectorType"); + } + @Override public InternalMap getMap(int pos) { assertIndexIsValid(pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BinaryRow.java b/paimon-common/src/main/java/org/apache/paimon/data/BinaryRow.java index 70d2ec7a01a8..ff5406f7b326 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BinaryRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BinaryRow.java @@ -355,6 +355,12 @@ public InternalArray getArray(int pos) { return MemorySegmentUtils.readArrayData(segments, offset, getLong(pos)); } + @Override + public InternalVector getVector(int pos) { + assertIndexIsValid(pos); + return MemorySegmentUtils.readVectorData(segments, offset, getLong(pos)); + } + @Override public InternalMap getMap(int pos) { assertIndexIsValid(pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BinaryVector.java b/paimon-common/src/main/java/org/apache/paimon/data/BinaryVector.java new file mode 100644 index 000000000000..7ef07a85a45c --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/BinaryVector.java @@ -0,0 +1,393 @@ +/* + * 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.paimon.data; + +import org.apache.paimon.annotation.Public; +import org.apache.paimon.data.variant.Variant; +import org.apache.paimon.memory.MemorySegment; +import org.apache.paimon.memory.MemorySegmentUtils; +import org.apache.paimon.types.DataType; + +import java.util.Objects; + +import static org.apache.paimon.memory.MemorySegment.UNSAFE; + +/** + * A binary implementation of {@link InternalVector} which is backed by {@link MemorySegment}s. + * + *

Compared to {@link BinaryArray}, {@link BinaryVector} stores only primitive types, and + * nullable is not supported yet. Thus, the memory layout is very simple. + * + *

The binary layout of {@link BinaryVector}: + * + *

+ * [values]
+ * 
+ * + * @since 2.0.0 + */ +@Public +public final class BinaryVector extends BinarySection implements InternalVector, DataSetters { + + private static final long serialVersionUID = 1L; + + /** Offset for Arrays. */ + private static final int BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class); + + private static final int BOOLEAN_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(boolean[].class); + private static final int SHORT_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(short[].class); + private static final int INT_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(int[].class); + private static final int LONG_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(long[].class); + private static final int FLOAT_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(float[].class); + private static final int DOUBLE_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(double[].class); + + public static int getPrimitiveElementSize(DataType type) { + // ordered by type root definition + switch (type.getTypeRoot()) { + case BOOLEAN: + case TINYINT: + return 1; + case SMALLINT: + return 2; + case INTEGER: + case FLOAT: + return 4; + case BIGINT: + case DOUBLE: + return 8; + default: + throw new IllegalArgumentException("Unsupported type: " + type); + } + } + + // The number of elements in this vector + private final int size; + + public BinaryVector(int size) { + this.size = size; + } + + private void assertIndexIsValid(int ordinal) { + assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0"; + assert ordinal < size : "ordinal (" + ordinal + ") should < " + size; + } + + private int getElementOffset(int ordinal, int elementSize) { + return offset + ordinal * elementSize; + } + + @Override + public int size() { + return size; + } + + @Override + public boolean isNullAt(int pos) { + assertIndexIsValid(pos); + return false; + } + + @Override + public void setNullAt(int pos) { + assertIndexIsValid(pos); + throw new UnsupportedOperationException("Nullable is not supported yet for VectorType."); + } + + @Override + public long getLong(int pos) { + assertIndexIsValid(pos); + return MemorySegmentUtils.getLong(segments, getElementOffset(pos, 8)); + } + + @Override + public void setLong(int pos, long value) { + assertIndexIsValid(pos); + MemorySegmentUtils.setLong(segments, getElementOffset(pos, 8), value); + } + + @Override + public int getInt(int pos) { + assertIndexIsValid(pos); + return MemorySegmentUtils.getInt(segments, getElementOffset(pos, 4)); + } + + @Override + public void setInt(int pos, int value) { + assertIndexIsValid(pos); + MemorySegmentUtils.setInt(segments, getElementOffset(pos, 4), value); + } + + @Override + public BinaryString getString(int pos) { + throw new UnsupportedOperationException("BinaryVector does not support String."); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + throw new UnsupportedOperationException("BinaryVector does not support Decimal."); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + throw new UnsupportedOperationException("BinaryVector does not support Timestamp."); + } + + @Override + public byte[] getBinary(int pos) { + throw new UnsupportedOperationException("BinaryVector does not support Binary."); + } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("BinaryVector does not support Variant."); + } + + @Override + public Blob getBlob(int pos) { + throw new UnsupportedOperationException("BinaryVector does not support Blob."); + } + + @Override + public InternalArray getArray(int pos) { + throw new UnsupportedOperationException("BinaryVector does not support nested Array."); + } + + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("BinaryVector does not support nested Vector."); + } + + @Override + public InternalMap getMap(int pos) { + throw new UnsupportedOperationException("BinaryVector does not support Map."); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + throw new UnsupportedOperationException("BinaryVector does not support nested Row."); + } + + @Override + public boolean getBoolean(int pos) { + assertIndexIsValid(pos); + return MemorySegmentUtils.getBoolean(segments, getElementOffset(pos, 1)); + } + + @Override + public void setBoolean(int pos, boolean value) { + assertIndexIsValid(pos); + MemorySegmentUtils.setBoolean(segments, getElementOffset(pos, 1), value); + } + + @Override + public byte getByte(int pos) { + assertIndexIsValid(pos); + return MemorySegmentUtils.getByte(segments, getElementOffset(pos, 1)); + } + + @Override + public void setByte(int pos, byte value) { + assertIndexIsValid(pos); + MemorySegmentUtils.setByte(segments, getElementOffset(pos, 1), value); + } + + @Override + public short getShort(int pos) { + assertIndexIsValid(pos); + return MemorySegmentUtils.getShort(segments, getElementOffset(pos, 2)); + } + + @Override + public void setShort(int pos, short value) { + assertIndexIsValid(pos); + MemorySegmentUtils.setShort(segments, getElementOffset(pos, 2), value); + } + + @Override + public float getFloat(int pos) { + assertIndexIsValid(pos); + return MemorySegmentUtils.getFloat(segments, getElementOffset(pos, 4)); + } + + @Override + public void setFloat(int pos, float value) { + assertIndexIsValid(pos); + MemorySegmentUtils.setFloat(segments, getElementOffset(pos, 4), value); + } + + @Override + public double getDouble(int pos) { + assertIndexIsValid(pos); + return MemorySegmentUtils.getDouble(segments, getElementOffset(pos, 8)); + } + + @Override + public void setDouble(int pos, double value) { + assertIndexIsValid(pos); + MemorySegmentUtils.setDouble(segments, getElementOffset(pos, 8), value); + } + + @Override + public void setDecimal(int pos, Decimal value, int precision) { + throw new UnsupportedOperationException("BinaryVector does not support Decimal."); + } + + @Override + public void setTimestamp(int pos, Timestamp value, int precision) { + throw new UnsupportedOperationException("BinaryVector does not support Timestamp."); + } + + @Override + public boolean[] toBooleanArray() { + boolean[] values = new boolean[size]; + MemorySegmentUtils.copyToUnsafe(segments, offset, values, BOOLEAN_ARRAY_OFFSET, size); + return values; + } + + @Override + public byte[] toByteArray() { + byte[] values = new byte[size]; + MemorySegmentUtils.copyToUnsafe(segments, offset, values, BYTE_ARRAY_BASE_OFFSET, size); + return values; + } + + @Override + public short[] toShortArray() { + short[] values = new short[size]; + MemorySegmentUtils.copyToUnsafe(segments, offset, values, SHORT_ARRAY_OFFSET, size * 2); + return values; + } + + @Override + public int[] toIntArray() { + int[] values = new int[size]; + MemorySegmentUtils.copyToUnsafe(segments, offset, values, INT_ARRAY_OFFSET, size * 4); + return values; + } + + @Override + public long[] toLongArray() { + long[] values = new long[size]; + MemorySegmentUtils.copyToUnsafe(segments, offset, values, LONG_ARRAY_OFFSET, size * 8); + return values; + } + + @Override + public float[] toFloatArray() { + float[] values = new float[size]; + MemorySegmentUtils.copyToUnsafe(segments, offset, values, FLOAT_ARRAY_OFFSET, size * 4); + return values; + } + + @Override + public double[] toDoubleArray() { + double[] values = new double[size]; + MemorySegmentUtils.copyToUnsafe(segments, offset, values, DOUBLE_ARRAY_OFFSET, size * 8); + return values; + } + + public BinaryVector copy() { + return copy(new BinaryVector(size)); + } + + public BinaryVector copy(BinaryVector reuse) { + byte[] bytes = MemorySegmentUtils.copyToBytes(segments, offset, sizeInBytes); + reuse.pointTo(MemorySegment.wrap(bytes), 0, sizeInBytes); + return reuse; + } + + @Override + public int hashCode() { + return Objects.hash(MemorySegmentUtils.hashByWords(segments, offset, sizeInBytes), size); + } + + // ------------------------------------------------------------------------------------------ + // Construction Utilities + // ------------------------------------------------------------------------------------------ + + public static BinaryVector fromPrimitiveArray(boolean[] arr) { + return fromPrimitiveArray(arr, BOOLEAN_ARRAY_OFFSET, arr.length, 1); + } + + public static BinaryVector fromPrimitiveArray(byte[] arr) { + return fromPrimitiveArray(arr, BYTE_ARRAY_BASE_OFFSET, arr.length, 1); + } + + public static BinaryVector fromPrimitiveArray(short[] arr) { + return fromPrimitiveArray(arr, SHORT_ARRAY_OFFSET, arr.length, 2); + } + + public static BinaryVector fromPrimitiveArray(int[] arr) { + return fromPrimitiveArray(arr, INT_ARRAY_OFFSET, arr.length, 4); + } + + public static BinaryVector fromPrimitiveArray(long[] arr) { + return fromPrimitiveArray(arr, LONG_ARRAY_OFFSET, arr.length, 8); + } + + public static BinaryVector fromPrimitiveArray(float[] arr) { + return fromPrimitiveArray(arr, FLOAT_ARRAY_OFFSET, arr.length, 4); + } + + public static BinaryVector fromPrimitiveArray(double[] arr) { + return fromPrimitiveArray(arr, DOUBLE_ARRAY_OFFSET, arr.length, 8); + } + + private static BinaryVector fromPrimitiveArray( + Object arr, int offset, int length, int elementSize) { + // must align by 8 bytes + final long valueRegionInBytes = (long) elementSize * length; + long totalSizeInLongs = (valueRegionInBytes + 7) / 8; + if (totalSizeInLongs > Integer.MAX_VALUE / 8) { + throw new UnsupportedOperationException( + "Cannot convert this vector to unsafe format as " + "it's too big."); + } + long totalSize = totalSizeInLongs * 8; + + final byte[] data = new byte[(int) totalSize]; + + UNSAFE.copyMemory(arr, offset, data, BYTE_ARRAY_BASE_OFFSET, valueRegionInBytes); + + BinaryVector result = new BinaryVector(length); + result.pointTo(MemorySegment.wrap(data), 0, (int) totalSize); + return result; + } + + public static BinaryVector fromInternalArray(InternalArray array, DataType elementType) { + switch (elementType.getTypeRoot()) { + case BOOLEAN: + return BinaryVector.fromPrimitiveArray(array.toBooleanArray()); + case TINYINT: + return BinaryVector.fromPrimitiveArray(array.toByteArray()); + case SMALLINT: + return BinaryVector.fromPrimitiveArray(array.toShortArray()); + case INTEGER: + return BinaryVector.fromPrimitiveArray(array.toIntArray()); + case BIGINT: + return BinaryVector.fromPrimitiveArray(array.toLongArray()); + case FLOAT: + return BinaryVector.fromPrimitiveArray(array.toFloatArray()); + case DOUBLE: + return BinaryVector.fromPrimitiveArray(array.toDoubleArray()); + default: + throw new UnsupportedOperationException( + "Unsupported element type for vector " + elementType); + } + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BinaryWriter.java b/paimon-common/src/main/java/org/apache/paimon/data/BinaryWriter.java index 06b763a0eac1..2e0cd5701b71 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BinaryWriter.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BinaryWriter.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.serializer.InternalMapSerializer; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.data.serializer.InternalSerializers; +import org.apache.paimon.data.serializer.InternalVectorSerializer; import org.apache.paimon.data.serializer.Serializer; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.DataType; @@ -74,6 +75,8 @@ public interface BinaryWriter { void writeArray(int pos, InternalArray value, InternalArraySerializer serializer); + void writeVector(int pos, InternalVector value, InternalVectorSerializer serializer); + void writeMap(int pos, InternalMap value, InternalMapSerializer serializer); void writeRow(int pos, InternalRow value, InternalRowSerializer serializer); @@ -133,6 +136,9 @@ static void write( case ARRAY: writer.writeArray(pos, (InternalArray) o, (InternalArraySerializer) serializer); break; + case VECTOR: + writer.writeVector(pos, (InternalVector) o, (InternalVectorSerializer) serializer); + break; case MAP: case MULTISET: writer.writeMap(pos, (InternalMap) o, (InternalMapSerializer) serializer); @@ -210,6 +216,14 @@ static ValueSetter createValueSetter(DataType elementType, Serializer seriali pos, (InternalArray) value, (InternalArraySerializer) arraySerializer); + case VECTOR: + final Serializer vectorSerializer = + serializer == null ? InternalSerializers.create(elementType) : serializer; + return (writer, pos, value) -> + writer.writeVector( + pos, + (InternalVector) value, + (InternalVectorSerializer) vectorSerializer); case MULTISET: case MAP: final Serializer mapSerializer = diff --git a/paimon-common/src/main/java/org/apache/paimon/data/DataGetters.java b/paimon-common/src/main/java/org/apache/paimon/data/DataGetters.java index d60ca316f7f2..1043b7e3ba4f 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/DataGetters.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/DataGetters.java @@ -84,6 +84,8 @@ public interface DataGetters { /** Returns the array value at the given position. */ InternalArray getArray(int pos); + InternalVector getVector(int pos); + /** Returns the map value at the given position. */ InternalMap getMap(int pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/GenericArray.java b/paimon-common/src/main/java/org/apache/paimon/data/GenericArray.java index 0d0898ea4edc..8c1ba4e28ac9 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/GenericArray.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/GenericArray.java @@ -240,6 +240,11 @@ public InternalArray getArray(int pos) { return (InternalArray) getObject(pos); } + @Override + public InternalVector getVector(int pos) { + return (InternalVector) getObject(pos); + } + @Override public InternalMap getMap(int pos) { return (InternalMap) getObject(pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/GenericRow.java b/paimon-common/src/main/java/org/apache/paimon/data/GenericRow.java index 37be2386b44b..10aefbafdd07 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/GenericRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/GenericRow.java @@ -202,6 +202,11 @@ public InternalArray getArray(int pos) { return (InternalArray) this.fields[pos]; } + @Override + public InternalVector getVector(int pos) { + return (InternalVector) this.fields[pos]; + } + @Override public InternalMap getMap(int pos) { return (InternalMap) this.fields[pos]; diff --git a/paimon-common/src/main/java/org/apache/paimon/data/InternalRow.java b/paimon-common/src/main/java/org/apache/paimon/data/InternalRow.java index 9ec838cd273b..3bbb85f49963 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/InternalRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/InternalRow.java @@ -213,6 +213,9 @@ static FieldGetter createFieldGetter(DataType fieldType, int fieldPos) { case ARRAY: fieldGetter = row -> row.getArray(fieldPos); break; + case VECTOR: + fieldGetter = row -> row.getVector(fieldPos); + break; case MULTISET: case MAP: fieldGetter = row -> row.getMap(fieldPos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/InternalVector.java b/paimon-common/src/main/java/org/apache/paimon/data/InternalVector.java new file mode 100644 index 000000000000..4b4cedeb0279 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/InternalVector.java @@ -0,0 +1,90 @@ +/* + * 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.paimon.data; + +import org.apache.paimon.annotation.Public; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.VectorType; + +/** + * Base interface of an internal data structure representing data of {@link VectorType}. + * + *

Note: All elements of this data structure must be internal data structures and must be of the + * same type. See {@link InternalRow} for more information about internal data structures. + * + * @see BinaryVector + * @since 2.0.0 + */ +@Public +public interface InternalVector extends InternalArray { + + // ------------------------------------------------------------------------------------------ + // Access Utilities + // ------------------------------------------------------------------------------------------ + + /** + * Creates an accessor for getting elements in an internal vector data structure at the given + * position. + * + * @param elementType the element type of the vector + */ + static ElementGetter createElementGetter(DataType elementType) { + final ElementGetter elementGetter; + // ordered by type root definition + switch (elementType.getTypeRoot()) { + case BOOLEAN: + elementGetter = InternalArray::getBoolean; + break; + case TINYINT: + elementGetter = InternalArray::getByte; + break; + case SMALLINT: + elementGetter = InternalArray::getShort; + break; + case INTEGER: + elementGetter = InternalArray::getInt; + break; + case BIGINT: + elementGetter = InternalArray::getLong; + break; + case FLOAT: + elementGetter = InternalArray::getFloat; + break; + case DOUBLE: + elementGetter = InternalArray::getDouble; + break; + default: + String msg = + String.format( + "type %s not support in %s", + elementType.getTypeRoot().toString(), + InternalVector.class.getName()); + throw new IllegalArgumentException(msg); + } + if (!elementType.isNullable()) { + return elementGetter; + } + return (vector, pos) -> { + if (vector.isNullAt(pos)) { + return null; + } + return elementGetter.getElementOrNull(vector, pos); + }; + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/JoinedRow.java b/paimon-common/src/main/java/org/apache/paimon/data/JoinedRow.java index 62aa7358082b..fee5552f8c5c 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/JoinedRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/JoinedRow.java @@ -252,6 +252,15 @@ public InternalArray getArray(int pos) { } } + @Override + public InternalVector getVector(int pos) { + if (pos < row1.getFieldCount()) { + return row1.getVector(pos); + } else { + return row2.getVector(pos - row1.getFieldCount()); + } + } + @Override public InternalMap getMap(int pos) { if (pos < row1.getFieldCount()) { diff --git a/paimon-common/src/main/java/org/apache/paimon/data/LazyGenericRow.java b/paimon-common/src/main/java/org/apache/paimon/data/LazyGenericRow.java index 680e3be0df4d..6d2e8b141f57 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/LazyGenericRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/LazyGenericRow.java @@ -158,6 +158,11 @@ public InternalArray getArray(int pos) { return (InternalArray) getField(pos); } + @Override + public InternalVector getVector(int pos) { + return (InternalVector) getField(pos); + } + @Override public InternalMap getMap(int pos) { return (InternalMap) getField(pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/NestedRow.java b/paimon-common/src/main/java/org/apache/paimon/data/NestedRow.java index 708c2bc60ce6..afc4f0c47fb0 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/NestedRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/NestedRow.java @@ -306,6 +306,11 @@ public InternalArray getArray(int pos) { return MemorySegmentUtils.readArrayData(segments, offset, getLong(pos)); } + @Override + public InternalVector getVector(int pos) { + throw new IllegalArgumentException("Unsupported type: VectorType"); + } + @Override public InternalMap getMap(int pos) { assertIndexIsValid(pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarArray.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarArray.java index e91fa26b4e56..ad04f647b977 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarArray.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarArray.java @@ -26,6 +26,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.GenericVariant; import org.apache.paimon.data.variant.Variant; @@ -143,6 +144,11 @@ public InternalArray getArray(int pos) { return ((ArrayColumnVector) data).getArray(offset + pos); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("Unsupported type: VectorType"); + } + @Override public InternalMap getMap(int pos) { return ((MapColumnVector) data).getMap(offset + pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRow.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRow.java index 468c44188a69..d7423608af45 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRow.java @@ -26,6 +26,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; @@ -163,6 +164,11 @@ public InternalArray getArray(int pos) { return vectorizedColumnBatch.getArray(rowId, pos); } + @Override + public InternalVector getVector(int pos) { + return vectorizedColumnBatch.getVector(rowId, pos); + } + @Override public InternalMap getMap(int pos) { return vectorizedColumnBatch.getMap(rowId, pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/RowToColumnConverter.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/RowToColumnConverter.java index d2a378846c6c..de962ad86a39 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/RowToColumnConverter.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/RowToColumnConverter.java @@ -62,6 +62,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import java.io.Serializable; import java.util.List; @@ -281,6 +282,11 @@ public TypeConverter visit(ArrayType arrayType) { }); } + @Override + public TypeConverter visit(VectorType vectorType) { + throw new UnsupportedOperationException(); + } + @Override public TypeConverter visit(MultisetType multisetType) { throw new UnsupportedOperationException(); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/VectorizedColumnBatch.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/VectorizedColumnBatch.java index 18eecd29ed59..297d0b0f9082 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/VectorizedColumnBatch.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/VectorizedColumnBatch.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.columnar.BytesColumnVector.Bytes; import org.apache.paimon.data.variant.GenericVariant; @@ -124,6 +125,10 @@ public InternalArray getArray(int rowId, int colId) { return ((ArrayColumnVector) columns[colId]).getArray(rowId); } + public InternalVector getVector(int rowId, int colId) { + throw new UnsupportedOperationException("Unsupported type: VectorType"); + } + public InternalRow getRow(int rowId, int colId) { return ((RowColumnVector) columns[colId]).getRow(rowId); } diff --git a/paimon-common/src/main/java/org/apache/paimon/data/safe/SafeBinaryArray.java b/paimon-common/src/main/java/org/apache/paimon/data/safe/SafeBinaryArray.java index c2ac7291c211..78d717ee1b0b 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/safe/SafeBinaryArray.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/safe/SafeBinaryArray.java @@ -26,6 +26,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.memory.BytesUtils; @@ -164,6 +165,11 @@ public InternalArray getArray(int pos) { throw new UnsupportedOperationException(); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException(); + } + @Override public InternalMap getMap(int pos) { throw new UnsupportedOperationException(); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/safe/SafeBinaryRow.java b/paimon-common/src/main/java/org/apache/paimon/data/safe/SafeBinaryRow.java index bdc864dfcb4f..2c285c30a6b4 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/safe/SafeBinaryRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/safe/SafeBinaryRow.java @@ -26,6 +26,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.memory.BytesUtils; @@ -186,6 +187,11 @@ private static InternalRow readNestedRow( return new SafeBinaryRow(numFields, bytes, offset + baseOffset); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException(); + } + @Override public InternalMap getMap(int pos) { throw new UnsupportedOperationException(); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalSerializers.java b/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalSerializers.java index 38e9c4678e49..6669f347ff27 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalSerializers.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalSerializers.java @@ -24,6 +24,7 @@ import org.apache.paimon.types.MapType; import org.apache.paimon.types.MultisetType; import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VectorType; import static org.apache.paimon.types.DataTypeChecks.getFieldTypes; import static org.apache.paimon.types.DataTypeChecks.getPrecision; @@ -75,6 +76,10 @@ private static Serializer createInternal(DataType type) { return new TimestampSerializer(getPrecision(type)); case ARRAY: return new InternalArraySerializer(((ArrayType) type).getElementType()); + case VECTOR: + VectorType vectorType = (VectorType) type; + return new InternalVectorSerializer( + vectorType.getElementType(), vectorType.getLength()); case MULTISET: return new InternalMapSerializer( ((MultisetType) type).getElementType(), new IntType(false)); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalVectorSerializer.java b/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalVectorSerializer.java new file mode 100644 index 000000000000..6ae5f23a9547 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalVectorSerializer.java @@ -0,0 +1,113 @@ +/* + * 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.paimon.data.serializer; + +import org.apache.paimon.data.BinaryVector; +import org.apache.paimon.data.InternalVector; +import org.apache.paimon.io.DataInputView; +import org.apache.paimon.io.DataOutputView; +import org.apache.paimon.memory.MemorySegment; +import org.apache.paimon.memory.MemorySegmentUtils; +import org.apache.paimon.types.DataType; + +import java.io.IOException; + +/** Serializer for {@link InternalVector}. */ +public class InternalVectorSerializer implements Serializer { + private static final long serialVersionUID = 1L; + + private final DataType eleType; + private final Serializer eleSer; + private final int length; + + public InternalVectorSerializer(DataType eleType, int length) { + this(eleType, InternalSerializers.create(eleType), length); + } + + private InternalVectorSerializer(DataType eleType, Serializer eleSer, int length) { + this.eleType = eleType; + this.eleSer = eleSer; + this.length = length; + } + + @Override + public InternalVectorSerializer duplicate() { + return new InternalVectorSerializer(eleType, eleSer.duplicate(), length); + } + + @Override + public InternalVector copy(InternalVector from) { + if (from instanceof BinaryVector) { + return ((BinaryVector) from).copy(); + } else { + return toBinaryVector(from).copy(); + } + } + + @Override + public void serialize(InternalVector record, DataOutputView target) throws IOException { + if (record.size() != length) { + throw new IOException("Invalid size to serialize: " + record.size()); + } + BinaryVector binaryVector = toBinaryVector(record); + target.writeInt(binaryVector.getSizeInBytes()); + MemorySegmentUtils.copyToView( + binaryVector.getSegments(), + binaryVector.getOffset(), + binaryVector.getSizeInBytes(), + target); + } + + @Override + public InternalVector deserialize(DataInputView source) throws IOException { + int sizeInBytes = source.readInt(); + byte[] bytes = new byte[sizeInBytes]; + source.readFully(bytes); + BinaryVector vector = new BinaryVector(length); + vector.pointTo(MemorySegment.wrap(bytes), 0, bytes.length); + return vector; + } + + public BinaryVector toBinaryVector(InternalVector from) { + if (from instanceof BinaryVector) { + return (BinaryVector) from; + } else { + return BinaryVector.fromInternalArray(from, eleType); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + InternalVectorSerializer that = (InternalVectorSerializer) o; + + return eleType.equals(that.eleType) && length == that.length; + } + + @Override + public int hashCode() { + return eleType.hashCode(); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/BitmapTypeVisitor.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/BitmapTypeVisitor.java index fbcd06d8dbda..4183bfbb2bf8 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/BitmapTypeVisitor.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/BitmapTypeVisitor.java @@ -41,6 +41,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; /** Simplified visitor for bitmap index. */ public abstract class BitmapTypeVisitor implements DataTypeVisitor { @@ -146,6 +147,11 @@ public final R visit(ArrayType arrayType) { throw new UnsupportedOperationException("Does not support type array"); } + @Override + public final R visit(VectorType vectorType) { + throw new UnsupportedOperationException("Does not support type vector"); + } + @Override public final R visit(MultisetType multisetType) { throw new UnsupportedOperationException("Does not support type mutiset"); diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/FastHash.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/FastHash.java index 91330bae460d..322847f849ab 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/FastHash.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/FastHash.java @@ -44,6 +44,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import net.openhft.hashing.LongHashFunction; @@ -176,6 +177,11 @@ public FastHash visit(ArrayType arrayType) { throw new UnsupportedOperationException("Does not support type array"); } + @Override + public FastHash visit(VectorType vectorType) { + throw new UnsupportedOperationException("Does not support type vector"); + } + @Override public FastHash visit(MultisetType multisetType) { throw new UnsupportedOperationException("Does not support type mutiset"); diff --git a/paimon-common/src/main/java/org/apache/paimon/memory/MemorySegmentUtils.java b/paimon-common/src/main/java/org/apache/paimon/memory/MemorySegmentUtils.java index 691c1e60e37d..a1f181b04d7f 100644 --- a/paimon-common/src/main/java/org/apache/paimon/memory/MemorySegmentUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/memory/MemorySegmentUtils.java @@ -22,16 +22,19 @@ import org.apache.paimon.data.BinaryArray; import org.apache.paimon.data.BinaryMap; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.NestedRow; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.GenericVariant; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.io.DataOutputView; import org.apache.paimon.utils.MurmurHashUtils; +import org.apache.paimon.utils.Preconditions; import java.io.IOException; @@ -1153,6 +1156,23 @@ public static InternalArray readArrayData( return array; } + public static InternalVector readVectorData( + MemorySegment[] segments, int baseOffset, long offsetAndSize) { + final int numElementsWidth = 4; + final int size = ((int) offsetAndSize); + int offset = (int) (offsetAndSize >> 32); + Preconditions.checkArgument( + size >= numElementsWidth, + "size (" + size + ") should >= numElementsWidth (" + numElementsWidth + ")"); + + int vectorSize = MemorySegmentUtils.getInt(segments, offset + baseOffset); + Preconditions.checkArgument(vectorSize >= 0, "Invalid vector size: %s", vectorSize); + + BinaryVector vector = new BinaryVector(vectorSize); + vector.pointTo(segments, offset + baseOffset + numElementsWidth, size - numElementsWidth); + return vector; + } + /** Gets an instance of {@link InternalRow} from underlying {@link MemorySegment}. */ public static InternalRow readRowData( MemorySegment[] segments, int numFields, int baseOffset, long offsetAndSize) { diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionArray.java b/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionArray.java index 7ebccba6b2c8..8b89c1ce7f8a 100644 --- a/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionArray.java +++ b/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionArray.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; @@ -148,6 +149,11 @@ public InternalArray getArray(int pos) { return chooseArray(pos).getArray(offsetInRow(pos)); } + @Override + public InternalVector getVector(int pos) { + return chooseArray(pos).getVector(offsetInRow(pos)); + } + @Override public InternalMap getMap(int pos) { return chooseArray(pos).getMap(offsetInRow(pos)); diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionRow.java b/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionRow.java index cc5cf2b18be3..08c6d24d2b79 100644 --- a/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionRow.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; @@ -172,6 +173,11 @@ public InternalArray getArray(int pos) { return chooseRow(pos).getArray(offsetInRow(pos)); } + @Override + public InternalVector getVector(int pos) { + return chooseRow(pos).getVector(offsetInRow(pos)); + } + @Override public InternalMap getMap(int pos) { return chooseRow(pos).getMap(offsetInRow(pos)); diff --git a/paimon-common/src/main/java/org/apache/paimon/sort/hilbert/HilbertIndexer.java b/paimon-common/src/main/java/org/apache/paimon/sort/hilbert/HilbertIndexer.java index a65455c04524..241dc6100379 100644 --- a/paimon-common/src/main/java/org/apache/paimon/sort/hilbert/HilbertIndexer.java +++ b/paimon-common/src/main/java/org/apache/paimon/sort/hilbert/HilbertIndexer.java @@ -47,6 +47,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.paimon.utils.ConvertBinaryUtil; import org.davidmoten.hilbert.HilbertCurve; @@ -276,6 +277,11 @@ public HProcessFunction visit(ArrayType arrayType) { throw new RuntimeException("Unsupported type"); } + @Override + public HProcessFunction visit(VectorType vectorType) { + throw new RuntimeException("Unsupported type"); + } + @Override public HProcessFunction visit(MultisetType multisetType) { throw new RuntimeException("Unsupported type"); diff --git a/paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java b/paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java index e6ce1d3e3e55..1d40fe75e776 100644 --- a/paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java +++ b/paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java @@ -48,6 +48,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import java.io.Serializable; import java.nio.ByteBuffer; @@ -364,6 +365,11 @@ public ZProcessFunction visit(ArrayType arrayType) { throw new RuntimeException("Unsupported type"); } + @Override + public ZProcessFunction visit(VectorType vectorType) { + throw new RuntimeException("Unsupported type"); + } + @Override public ZProcessFunction visit(MultisetType multisetType) { throw new RuntimeException("Unsupported type"); diff --git a/paimon-common/src/main/java/org/apache/paimon/types/InternalRowToSizeVisitor.java b/paimon-common/src/main/java/org/apache/paimon/types/InternalRowToSizeVisitor.java index dd4548572f35..dbac55a07dde 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/InternalRowToSizeVisitor.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/InternalRowToSizeVisitor.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import java.util.List; import java.util.function.BiFunction; @@ -253,6 +254,28 @@ public BiFunction visit(ArrayType arrayType) { }; } + @Override + public BiFunction visit(VectorType vectorType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + // If it is ensured that the element type + // must be primitive type, then this can be simplified. + BiFunction function = + vectorType.getElementType().accept(this); + InternalVector internalVector = row.getVector(index); + + int size = 0; + for (int i = 0; i < internalVector.size(); i++) { + size += function.apply(internalVector, i); + } + + return size; + } + }; + } + @Override public BiFunction visit(MultisetType multisetType) { return (row, index) -> { diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowUtils.java index 14751217d4cc..4cfe35e39851 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowUtils.java @@ -318,6 +318,8 @@ public static Object get(DataGetters dataGetters, int pos, DataType fieldType) { pos, decimalType.getPrecision(), decimalType.getScale()); case ARRAY: return dataGetters.getArray(pos); + case VECTOR: + return dataGetters.getVector(pos); case MAP: case MULTISET: return dataGetters.getMap(pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/KeyProjectedRow.java b/paimon-common/src/main/java/org/apache/paimon/utils/KeyProjectedRow.java index 7eedd7b8f24c..d4999dcf57dc 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/KeyProjectedRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/KeyProjectedRow.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; @@ -140,6 +141,11 @@ public InternalArray getArray(int pos) { return row.getArray(indexMapping[pos]); } + @Override + public InternalVector getVector(int pos) { + return row.getVector(indexMapping[pos]); + } + @Override public InternalMap getMap(int pos) { return row.getMap(indexMapping[pos]); diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedArray.java b/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedArray.java index b05914271be6..015fb022edbb 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedArray.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedArray.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.DataType; @@ -42,7 +43,7 @@ public class ProjectedArray implements InternalArray { private InternalArray array; - private ProjectedArray(int[] indexMapping) { + protected ProjectedArray(int[] indexMapping) { this.indexMapping = indexMapping; } @@ -142,6 +143,11 @@ public InternalArray getArray(int pos) { return array.getArray(indexMapping[pos]); } + @Override + public InternalVector getVector(int pos) { + return array.getVector(indexMapping[pos]); + } + @Override public InternalMap getMap(int pos) { return array.getMap(indexMapping[pos]); diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedRow.java b/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedRow.java index 18c7e5db798d..a9dc3e9253d7 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedRow.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.DataType; @@ -157,6 +158,11 @@ public InternalArray getArray(int pos) { return row.getArray(indexMapping[pos]); } + @Override + public InternalVector getVector(int pos) { + return row.getVector(indexMapping[pos]); + } + @Override public InternalMap getMap(int pos) { return row.getMap(indexMapping[pos]); diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedVector.java b/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedVector.java new file mode 100644 index 000000000000..889c444ec40c --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedVector.java @@ -0,0 +1,128 @@ +/* + * 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.paimon.utils; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.data.variant.Variant; +import org.apache.paimon.types.DataType; + +/** + * An implementation of {@link InternalVector} which provides a projected view of the underlying + * {@link InternalVector}. + * + *

Projection includes both reducing the accessible fields and reordering them. + * + *

Note: This class supports only top-level projections, not nested projections. + */ +public class ProjectedVector extends ProjectedArray implements InternalVector { + + protected ProjectedVector(int[] indexMapping) { + super(indexMapping); + } + + /** + * Replaces the underlying {@link InternalVector} backing this {@link ProjectedVector}. + * + *

This method replaces the row data in place and does not return a new object. This is done + * for performance reasons. + */ + public ProjectedVector replaceVector(InternalVector vector) { + super.replaceArray(vector); + return this; + } + + @Override + public ProjectedArray replaceArray(InternalArray array) { + throw new IllegalArgumentException("ProjectedVector does not support replaceArray."); + } + + // --------------------------------------------------------------------------------------------- + + @Override + public BinaryString getString(int pos) { + throw new UnsupportedOperationException("ProjectedVector does not support String."); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + throw new UnsupportedOperationException("ProjectedVector does not support Decimal."); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + throw new UnsupportedOperationException("ProjectedVector does not support Timestamp."); + } + + @Override + public byte[] getBinary(int pos) { + throw new UnsupportedOperationException("ProjectedVector does not support Binary."); + } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("ProjectedVector does not support Variant."); + } + + @Override + public Blob getBlob(int pos) { + throw new UnsupportedOperationException("ProjectedVector does not support Blob."); + } + + @Override + public InternalArray getArray(int pos) { + throw new UnsupportedOperationException("ProjectedVector does not support nested Array."); + } + + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException( + "ProjectedVector does not support nested VectorType."); + } + + @Override + public InternalMap getMap(int pos) { + throw new UnsupportedOperationException("ProjectedVector does not support Map."); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + throw new UnsupportedOperationException("ProjectedVector does not support nested Row."); + } + + /** + * Create an empty {@link ProjectedVector} starting from a {@code projection} vector. + * + *

The vector represents the mapping of the fields of the original {@link DataType}. For + * example, {@code [0, 2, 1]} specifies to include in the following order the 1st field, the 3rd + * field and the 2nd field of the row. + * + * @see Projection + * @see ProjectedVector + */ + public static ProjectedVector from(int[] projection) { + return new ProjectedVector(projection); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/TypeCheckUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/TypeCheckUtils.java index 68450f8712b3..c1520be34107 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/TypeCheckUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/TypeCheckUtils.java @@ -33,6 +33,7 @@ import static org.apache.paimon.types.DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE; import static org.apache.paimon.types.DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE; import static org.apache.paimon.types.DataTypeRoot.VARIANT; +import static org.apache.paimon.types.DataTypeRoot.VECTOR; /** Utils for type. */ public class TypeCheckUtils { @@ -85,6 +86,10 @@ public static boolean isArray(DataType type) { return type.getTypeRoot() == ARRAY; } + public static boolean isVector(DataType type) { + return type.getTypeRoot() == VECTOR; + } + public static boolean isMap(DataType type) { return type.getTypeRoot() == MAP; } @@ -110,6 +115,7 @@ public static boolean isComparable(DataType type) { && !isMultiset(type) && !isRow(type) && !isArray(type) + && !isVector(type) && !isVariant(type) && !isBlob(type); } @@ -120,6 +126,7 @@ public static boolean isMutable(DataType type) { case CHAR: case VARCHAR: // the internal representation of String is BinaryString which is mutable case ARRAY: + case VECTOR: case MULTISET: case MAP: case ROW: diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/TypeUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/TypeUtils.java index e80b6621e9b1..bcb3f91b76df 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/TypeUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/TypeUtils.java @@ -19,21 +19,25 @@ package org.apache.paimon.utils; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.GenericArray; import org.apache.paimon.data.GenericMap; import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalArray; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeChecks; import org.apache.paimon.types.DataTypeRoot; +import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.DecimalType; import org.apache.paimon.types.LocalZonedTimestampType; import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; import org.apache.paimon.types.TimestampType; import org.apache.paimon.types.VarCharType; +import org.apache.paimon.types.VectorType; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -212,6 +216,20 @@ public static Object castFromStringInternal(String s, DataType type, boolean isC throw new RuntimeException( String.format("Failed to parse Json String %s", s), e); } + case VECTOR: + // Step 1: parse the string to an array + VectorType vectorType = (VectorType) type; + DataType vectorElementType = vectorType.getElementType(); + Object vectorArrayObject = + castFromStringInternal(s, DataTypes.ARRAY(vectorElementType), isCdcValue); + if (!(vectorArrayObject instanceof InternalArray)) { + throw new RuntimeException( + "Unexpected parsed type during building a vector: " + + vectorArrayObject.getClass()); + } + // Step 2: build a vector + return BinaryVector.fromInternalArray( + (InternalArray) vectorArrayObject, vectorElementType); case MAP: MapType mapType = (MapType) type; DataType keyType = mapType.getKeyType(); @@ -333,6 +351,7 @@ public static boolean isInteroperable(DataType t1, DataType t2) { switch (t1.getTypeRoot()) { case ARRAY: + case VECTOR: case MAP: case MULTISET: case ROW: diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java index b57eff100ac0..b7f92d06d95c 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java @@ -64,6 +64,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; /** * This is a util about how to expand the {@link ColumnVector}s with the partition row and index @@ -354,6 +355,11 @@ public ColumnVector getColumnVector() { }; } + @Override + public ColumnVector visit(VectorType vectorType) { + throw new UnsupportedOperationException("VectorType is not supported."); + } + @Override public ColumnVector visit(MultisetType multisetType) { return new MapColumnVector() { diff --git a/paimon-common/src/test/java/org/apache/paimon/data/BinaryRowTest.java b/paimon-common/src/test/java/org/apache/paimon/data/BinaryRowTest.java index 04181eb2a881..6dd3c23bdb87 100644 --- a/paimon-common/src/test/java/org/apache/paimon/data/BinaryRowTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/data/BinaryRowTest.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.serializer.InternalMapSerializer; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.data.serializer.InternalSerializers; +import org.apache.paimon.data.serializer.InternalVectorSerializer; import org.apache.paimon.data.serializer.Serializer; import org.apache.paimon.data.variant.GenericVariant; import org.apache.paimon.memory.MemorySegment; @@ -518,6 +519,39 @@ public void testGenericArray() { assertThat(array2.getInt(2)).isEqualTo(666); } + @Test + public void testBinaryVector() { + // 1. vector test + final Random rnd = new Random(System.currentTimeMillis()); + float[] vectorValues = new float[rnd.nextInt(128) + 1]; + { + byte[] bytes = new byte[vectorValues.length]; + rnd.nextBytes(bytes); + for (int i = 0; i < vectorValues.length; i++) { + vectorValues[i] = bytes[i]; + } + } + BinaryVector vector = BinaryVector.fromPrimitiveArray(vectorValues); + + assertThat(vectorValues.length).isEqualTo(vector.size()); + int[] checkIndexList = {0, rnd.nextInt(vectorValues.length), vectorValues.length - 1}; + for (int checkIndex : checkIndexList) { + assertThat(vectorValues[checkIndex]).isEqualTo(vector.getFloat(checkIndex)); + } + + // 2. test write vector to binary row + BinaryRow row = new BinaryRow(1); + BinaryRowWriter rowWriter = new BinaryRowWriter(row); + InternalVectorSerializer serializer = + new InternalVectorSerializer(DataTypes.FLOAT(), vector.size()); + rowWriter.writeVector(0, vector, serializer); + rowWriter.complete(); + + InternalVector vector2 = row.getVector(0); + assertThat(vector2.size()).isEqualTo(vector.size()); + assertThat(vector2.toFloatArray()).isEqualTo(vector.toFloatArray()); + } + @Test public void testBinaryMap() { BinaryArray array1 = new BinaryArray(); diff --git a/paimon-common/src/test/java/org/apache/paimon/data/BinaryVectorTest.java b/paimon-common/src/test/java/org/apache/paimon/data/BinaryVectorTest.java new file mode 100644 index 000000000000..ff3ea0caeb0a --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/data/BinaryVectorTest.java @@ -0,0 +1,119 @@ +/* + * 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.paimon.data; + +import org.apache.paimon.types.DataTypes; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link BinaryVector}. */ +public class BinaryVectorTest { + + @Test + public void testFromPrimitiveArray() { + int[] ints = new int[] {1, 2, 3}; + BinaryVector intVector = BinaryVector.fromPrimitiveArray(ints); + assertThat(intVector.size()).isEqualTo(ints.length); + assertThat(intVector.getInt(1)).isEqualTo(2); + assertThat(intVector.toIntArray()).isEqualTo(ints); + + float[] floats = new float[] {1.0f, 2.5f}; + BinaryVector floatVector = BinaryVector.fromPrimitiveArray(floats); + assertThat(floatVector.size()).isEqualTo(floats.length); + assertThat(floatVector.getFloat(0)).isEqualTo(1.0f); + assertThat(floatVector.toFloatArray()).isEqualTo(floats); + + boolean[] booleans = new boolean[] {true, false, true}; + BinaryVector booleanVector = BinaryVector.fromPrimitiveArray(booleans); + assertThat(booleanVector.size()).isEqualTo(booleans.length); + assertThat(booleanVector.getBoolean(2)).isTrue(); + assertThat(booleanVector.toBooleanArray()).isEqualTo(booleans); + + byte[] bytes = new byte[] {1, 2, 3}; + BinaryVector byteVector = BinaryVector.fromPrimitiveArray(bytes); + assertThat(byteVector.size()).isEqualTo(bytes.length); + assertThat(byteVector.getByte(2)).isEqualTo((byte) 3); + assertThat(byteVector.toByteArray()).isEqualTo(bytes); + + short[] shorts = new short[] {4, 5, 6}; + BinaryVector shortVector = BinaryVector.fromPrimitiveArray(shorts); + assertThat(shortVector.size()).isEqualTo(shorts.length); + assertThat(shortVector.getShort(1)).isEqualTo((short) 5); + assertThat(shortVector.toShortArray()).isEqualTo(shorts); + + long[] longs = new long[] {7L, 8L, 9L}; + BinaryVector longVector = BinaryVector.fromPrimitiveArray(longs); + assertThat(longVector.size()).isEqualTo(longs.length); + assertThat(longVector.getLong(0)).isEqualTo(7L); + assertThat(longVector.toLongArray()).isEqualTo(longs); + + double[] doubles = new double[] {1.2d, 3.4d}; + BinaryVector doubleVector = BinaryVector.fromPrimitiveArray(doubles); + assertThat(doubleVector.size()).isEqualTo(doubles.length); + assertThat(doubleVector.getDouble(1)).isEqualTo(3.4d); + assertThat(doubleVector.toDoubleArray()).isEqualTo(doubles); + } + + @Test + public void testFromGenericArray() { + float[] values = new float[] {1.0f, -2.0f, 3.5f}; + GenericArray array = new GenericArray(values); + BinaryVector vector = BinaryVector.fromInternalArray(array, DataTypes.FLOAT()); + assertThat(vector.size()).isEqualTo(values.length); + assertThat(vector.toFloatArray()).isEqualTo(values); + } + + @Test + public void testFromBinaryArray() { + float[] values = new float[] {1.0f, -2.0f, 3.5f}; + BinaryArray array = BinaryArray.fromPrimitiveArray(values); + BinaryVector vector = BinaryVector.fromInternalArray(array, DataTypes.FLOAT()); + assertThat(vector.size()).isEqualTo(values.length); + assertThat(vector.toFloatArray()).isEqualTo(values); + } + + @Test + public void testCopiedBinaryVector() { + float[] values = new float[] {1.0f, -2.0f, 3.5f}; + BinaryVector vector = BinaryVector.fromPrimitiveArray(values); + BinaryVector copied = vector.copy(); + + // Assert that the copied vector is a new object + assertThat(copied == vector).isFalse(); + assertThat(copied.getSegments() == vector.getSegments()).isFalse(); + assertThat(copied.getSegments()[0] == vector.getSegments()[0]).isFalse(); + + // Assert that the copied vector has the same values as the original vector + assertThat(copied.toFloatArray()).isEqualTo(values); + assertThat(copied.equals(vector)).isTrue(); + } + + @Test + public void testPrimitiveElementSize() { + assertThat(BinaryVector.getPrimitiveElementSize(DataTypes.BOOLEAN())).isEqualTo(1); + assertThat(BinaryVector.getPrimitiveElementSize(DataTypes.TINYINT())).isEqualTo(1); + assertThat(BinaryVector.getPrimitiveElementSize(DataTypes.SMALLINT())).isEqualTo(2); + assertThat(BinaryVector.getPrimitiveElementSize(DataTypes.INT())).isEqualTo(4); + assertThat(BinaryVector.getPrimitiveElementSize(DataTypes.BIGINT())).isEqualTo(8); + assertThat(BinaryVector.getPrimitiveElementSize(DataTypes.FLOAT())).isEqualTo(4); + assertThat(BinaryVector.getPrimitiveElementSize(DataTypes.DOUBLE())).isEqualTo(8); + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/data/serializer/InternalVectorSerializerTest.java b/paimon-common/src/test/java/org/apache/paimon/data/serializer/InternalVectorSerializerTest.java new file mode 100644 index 000000000000..972b2cf480f5 --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/data/serializer/InternalVectorSerializerTest.java @@ -0,0 +1,92 @@ +/* + * 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.paimon.data.serializer; + +import org.apache.paimon.data.BinaryVector; +import org.apache.paimon.data.InternalVector; +import org.apache.paimon.io.DataOutputSerializer; +import org.apache.paimon.types.DataTypes; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.lang.reflect.Proxy; +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link InternalVectorSerializer}. */ +class InternalVectorSerializerTest extends SerializerTestBase { + + @Override + protected InternalVectorSerializer createSerializer() { + return new InternalVectorSerializer(DataTypes.FLOAT(), 3); + } + + @Override + protected boolean deepEquals(InternalVector vector1, InternalVector vector2) { + if (vector1.size() != vector2.size()) { + return false; + } + float[] left = vector1.toFloatArray(); + float[] right = vector2.toFloatArray(); + if (left.length != right.length) { + return false; + } + for (int i = 0; i < left.length; i++) { + if (Float.compare(left[i], right[i]) != 0) { + return false; + } + } + return true; + } + + @Override + protected InternalVector[] getTestData() { + return new InternalVector[] { + BinaryVector.fromPrimitiveArray(new float[] {1.0f, 2.0f, 3.0f}), + BinaryVector.fromPrimitiveArray(new float[] {-1.0f, 0.5f, 2.0f}), + createCustomVector(new float[] {0.0f, -2.0f, 4.5f}) + }; + } + + @Override + protected InternalVector[] getSerializableTestData() { + InternalVector[] testData = getTestData(); + return Arrays.copyOfRange(testData, 0, testData.length - 1); + } + + @Test + public void testSerializeWithInvalidSize() { + InternalVectorSerializer serializer = new InternalVectorSerializer(DataTypes.FLOAT(), 2); + InternalVector vector = BinaryVector.fromPrimitiveArray(new float[] {1.0f, 2.0f, 3.0f}); + DataOutputSerializer out = new DataOutputSerializer(32); + assertThatThrownBy(() -> serializer.serialize(vector, out)).isInstanceOf(IOException.class); + } + + private static InternalVector createCustomVector(float[] values) { + BinaryVector vector = BinaryVector.fromPrimitiveArray(values); + Object proxy = + Proxy.newProxyInstance( + InternalVectorSerializerTest.class.getClassLoader(), + new Class[] {InternalVector.class}, + (obj, method, args) -> method.invoke(vector, args)); + return (InternalVector) proxy; + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/datagen/RandomGeneratorVisitor.java b/paimon-common/src/test/java/org/apache/paimon/datagen/RandomGeneratorVisitor.java index c2cc190f733c..b64e034593c3 100644 --- a/paimon-common/src/test/java/org/apache/paimon/datagen/RandomGeneratorVisitor.java +++ b/paimon-common/src/test/java/org/apache/paimon/datagen/RandomGeneratorVisitor.java @@ -45,6 +45,7 @@ import org.apache.paimon.types.TinyIntType; import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; +import org.apache.paimon.types.VectorType; import org.apache.paimon.utils.Preconditions; import java.math.BigDecimal; @@ -236,6 +237,11 @@ public DataGeneratorContainer visit(ArrayType arrayType) { container.getOptions().toArray(new ConfigOption[0])); } + @Override + public DataGeneratorContainer visit(VectorType vectorType) { + throw new RuntimeException("RandomGenerator for VectorType has not been implemented yet."); + } + @Override public DataGeneratorContainer visit(MultisetType multisetType) { ConfigOption lenOption = diff --git a/paimon-common/src/test/java/org/apache/paimon/types/DataTypeChecksTest.java b/paimon-common/src/test/java/org/apache/paimon/types/DataTypeChecksTest.java index aa2cae53885e..7fedf3a30d8f 100644 --- a/paimon-common/src/test/java/org/apache/paimon/types/DataTypeChecksTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/types/DataTypeChecksTest.java @@ -62,4 +62,11 @@ void testFieldCountExtraction() { new DataField(1, "f1", STRING_TYPE))); assertThat(DataTypeChecks.getFieldCount(dataType)).isEqualTo(2); } + + @Test + void testVectorNestedTypesAndLength() { + DataType vectorType = DataTypes.VECTOR(4, DataTypes.FLOAT()); + assertThat(DataTypeChecks.getNestedTypes(vectorType)).containsExactly(DataTypes.FLOAT()); + assertThat(DataTypeChecks.getLength(vectorType)).isEqualTo(4); + } } diff --git a/paimon-common/src/test/java/org/apache/paimon/utils/TypeUtilsTest.java b/paimon-common/src/test/java/org/apache/paimon/utils/TypeUtilsTest.java index d7dab1614a48..54c1af27d8c4 100644 --- a/paimon-common/src/test/java/org/apache/paimon/utils/TypeUtilsTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/utils/TypeUtilsTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.utils; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.GenericArray; import org.apache.paimon.data.GenericMap; @@ -116,6 +117,14 @@ public void testArrayStringCastFromString() { assertThat(result).isEqualTo(expected); } + @Test + public void testVectorCastFromString() { + String value = "[1.0, 2.5, 3.5]"; + Object result = TypeUtils.castFromString(value, DataTypes.VECTOR(3, DataTypes.FLOAT())); + BinaryVector vector = (BinaryVector) result; + assertThat(vector.toFloatArray()).isEqualTo(new float[] {1.0f, 2.5f, 3.5f}); + } + @Test public void testLongCastFromString() { String value = "12"; diff --git a/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolution.java b/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolution.java index 346cae52b3ec..906d97501f4b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolution.java +++ b/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolution.java @@ -28,6 +28,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.format.SimpleColStats; @@ -281,6 +282,11 @@ public InternalArray getArray(int pos) { throw new UnsupportedOperationException(); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException(); + } + @Override public InternalMap getMap(int pos) { throw new UnsupportedOperationException(); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/OffsetRow.java b/paimon-core/src/main/java/org/apache/paimon/utils/OffsetRow.java index 4858eebeb73c..70b7e24514db 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/OffsetRow.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/OffsetRow.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; @@ -141,6 +142,11 @@ public InternalArray getArray(int pos) { return row.getArray(offset + pos); } + @Override + public InternalVector getVector(int pos) { + return row.getVector(offset + pos); + } + @Override public InternalMap getMap(int pos) { return row.getMap(offset + pos); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/PartialRow.java b/paimon-core/src/main/java/org/apache/paimon/utils/PartialRow.java index 18b22886ce3e..2e172dadf38a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/PartialRow.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/PartialRow.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; @@ -139,6 +140,11 @@ public InternalArray getArray(int pos) { return row.getArray(pos); } + @Override + public InternalVector getVector(int pos) { + return row.getVector(pos); + } + @Override public InternalMap getMap(int pos) { return row.getMap(pos); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/VectorTypeTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/VectorTypeTableTest.java new file mode 100644 index 000000000000..afa2e1db1826 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/append/VectorTypeTableTest.java @@ -0,0 +1,97 @@ +/* + * 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.paimon.append; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.TableTestBase; +import org.apache.paimon.types.DataTypes; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicInteger; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +/** Tests for table with vector. */ +public class VectorTypeTableTest extends TableTestBase { + + private final float[] testVector = randomVector(); + + @Test + public void testBasic() throws Exception { + createTableDefault(); + + commitDefault(writeDataDefault(100, 1)); + + AtomicInteger integer = new AtomicInteger(0); + + readDefault( + row -> { + integer.incrementAndGet(); + if (integer.get() % 50 == 0) { + Assertions.assertArrayEquals( + row.getVector(2).toFloatArray(), testVector, 0); + } + }); + + assertThat(integer.get()).isEqualTo(100); + } + + @Override + protected Schema schemaDefault() { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("f0", DataTypes.INT()); + schemaBuilder.column("f1", DataTypes.STRING()); + schemaBuilder.column("f2", DataTypes.VECTOR(testVector.length, DataTypes.FLOAT())); + // schemaBuilder.option(CoreOptions.TARGET_FILE_SIZE.key(), "25 MB"); + schemaBuilder.option(CoreOptions.FILE_FORMAT.key(), "json"); + schemaBuilder.option(CoreOptions.FILE_COMPRESSION.key(), "none"); + return schemaBuilder.build(); + } + + @Override + protected InternalRow dataDefault(int time, int size) { + return GenericRow.of( + RANDOM.nextInt(), + BinaryString.fromBytes(randomBytes()), + BinaryVector.fromPrimitiveArray(testVector)); + } + + @Override + protected byte[] randomBytes() { + byte[] binary = new byte[RANDOM.nextInt(1024) + 1]; + RANDOM.nextBytes(binary); + return binary; + } + + private float[] randomVector() { + byte[] randomBytes = randomBytes(); + float[] vector = new float[randomBytes.length]; + for (int i = 0; i < vector.length; i++) { + vector[i] = randomBytes[i]; + } + return vector; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java b/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java index c0da56f54454..5872750dd788 100644 --- a/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java @@ -30,8 +30,10 @@ import static org.apache.paimon.codegen.CodeGenUtils.newRecordComparator; import static org.apache.paimon.codegen.CodeGenUtils.newRecordEqualiser; import static org.apache.paimon.types.DataTypes.DOUBLE; +import static org.apache.paimon.types.DataTypes.FLOAT; import static org.apache.paimon.types.DataTypes.INT; import static org.apache.paimon.types.DataTypes.STRING; +import static org.apache.paimon.types.DataTypes.VECTOR; import static org.assertj.core.api.Assertions.assertThat; class CodeGenUtilsTest { @@ -54,6 +56,15 @@ public void testProjectionCodegenCacheMiss() { new int[] {0, 1, 2})); } + @Test + public void testProjectionWithVector() { + assertClassEquals( + () -> + newProjection( + RowType.builder().fields(INT(), VECTOR(3, FLOAT())).build(), + new int[] {1})); + } + @Test public void testNormalizedKeyComputerCodegenCache() { assertClassEquals( @@ -74,6 +85,14 @@ public void testRecordComparatorCodegenCache() { () -> newRecordComparator(Arrays.asList(STRING(), INT()), new int[] {0, 1}, true)); } + @Test + public void testRecordComparatorCodegenCacheWithVector() { + assertClassEquals( + () -> + newRecordComparator( + Arrays.asList(STRING(), VECTOR(3, INT())), new int[] {0, 1}, true)); + } + @Test public void testRecordComparatorCodegenCacheMiss() { assertClassNotEquals( diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/DataTypeJsonParserTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/DataTypeJsonParserTest.java index 0ca5159782b6..5ec613e2f696 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/DataTypeJsonParserTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/DataTypeJsonParserTest.java @@ -44,6 +44,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.paimon.utils.JsonSerdeUtil; import org.junit.jupiter.api.Test; @@ -114,6 +115,15 @@ private static Stream testData() { TestSpec.forString("TIMESTAMP_LTZ(3)").expectType(new LocalZonedTimestampType(3)), TestSpec.forString("VARIANT").expectType(new VariantType()), TestSpec.forString("BLOB").expectType(new BlobType()), + TestSpec.forString("VECTOR NOT NULL") + .expectType(DataTypes.VECTOR(3, DataTypes.FLOAT())), + TestSpec.forString("VECTOR") + .expectType(new VectorType(true, 5, DataTypes.INT())), + TestSpec.forString( + "{\"type\":\"VECTOR NOT NULL\",\"element\":\"BOOLEAN\",\"length\":7}") + .expectType(DataTypes.VECTOR(7, DataTypes.BOOLEAN())), + TestSpec.forString("{\"type\":\"VECTOR\",\"element\":\"TINYINT\",\"length\":11}") + .expectType(new VectorType(true, 11, DataTypes.TINYINT())), TestSpec.forString( "{\"type\":\"ARRAY\",\"element\":\"TIMESTAMP(3) WITH LOCAL TIME ZONE\"}") .expectType(new ArrayType(new LocalZonedTimestampType(3))), diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataTypeToLogicalType.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataTypeToLogicalType.java index ff9c7151d25d..6fc3016d4f95 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataTypeToLogicalType.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataTypeToLogicalType.java @@ -43,6 +43,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.flink.table.types.logical.LogicalType; @@ -159,6 +160,11 @@ public LogicalType visit(ArrayType arrayType) { arrayType.isNullable(), arrayType.getElementType().accept(this)); } + @Override + public LogicalType visit(VectorType vectorType) { + throw new UnsupportedOperationException("Not support VectorType yet."); + } + @Override public LogicalType visit(MultisetType multisetType) { return new org.apache.flink.table.types.logical.MultisetType( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowWrapper.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowWrapper.java index 1f632212d506..ae8a4fc59211 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowWrapper.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowWrapper.java @@ -27,6 +27,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.GenericVariant; import org.apache.paimon.data.variant.Variant; @@ -160,6 +161,11 @@ public InternalArray getArray(int pos) { return new FlinkArrayWrapper(row.getArray(pos)); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("Not support VectorType yet."); + } + @Override public InternalMap getMap(int pos) { return new FlinkMapWrapper(row.getMap(pos)); @@ -260,6 +266,11 @@ public InternalArray getArray(int pos) { return new FlinkArrayWrapper(array.getArray(pos)); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("Not support VectorType yet."); + } + @Override public InternalMap getMap(int pos) { return new FlinkMapWrapper(array.getMap(pos)); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFileFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFileFormat.java index c1892a944eb9..3abbb6882c97 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFileFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFileFormat.java @@ -91,6 +91,7 @@ private void validateDataType(DataType dataType) { case TIMESTAMP_WITHOUT_TIME_ZONE: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: case ARRAY: + case VECTOR: case MAP: case ROW: // All types are supported in JSON diff --git a/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFileReader.java b/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFileReader.java index 7f9b7a1545f8..d5ecf3dc8fd2 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFileReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFileReader.java @@ -21,6 +21,7 @@ import org.apache.paimon.casting.CastExecutor; import org.apache.paimon.casting.CastExecutors; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.GenericArray; import org.apache.paimon.data.GenericMap; import org.apache.paimon.data.GenericRow; @@ -34,6 +35,7 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VectorType; import org.apache.paimon.utils.JsonSerdeUtil; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; @@ -102,6 +104,8 @@ private Object convertJsonValue(JsonNode node, DataType dataType, JsonOptions op } case ARRAY: return convertJsonArray(node, (ArrayType) dataType, options); + case VECTOR: + return convertJsonVector(node, (VectorType) dataType, options); case MAP: return convertJsonMap(node, (MapType) dataType, options); case ROW: @@ -136,6 +140,13 @@ private GenericArray convertJsonArray( return new GenericArray(elements.toArray()); } + private BinaryVector convertJsonVector( + JsonNode vectorNode, VectorType vectorType, JsonOptions options) { + ArrayType arrayType = DataTypes.ARRAY(vectorType.getElementType()); + GenericArray array = convertJsonArray(vectorNode, arrayType, options); + return BinaryVector.fromInternalArray(array, vectorType.getElementType()); + } + private GenericMap convertJsonMap(JsonNode objectNode, MapType mapType, JsonOptions options) { if (!objectNode.isObject()) { return handleParseError( diff --git a/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFormatWriter.java b/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFormatWriter.java index 01a55e9ae735..2aeb9a5c1bce 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFormatWriter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/json/JsonFormatWriter.java @@ -24,14 +24,17 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.format.text.AbstractTextFileWriter; import org.apache.paimon.fs.PositionOutputStream; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeRoot; +import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VectorType; import org.apache.paimon.utils.InternalRowUtils; import org.apache.paimon.utils.JsonSerdeUtil; @@ -112,6 +115,8 @@ private Object convertRowValue(Object value, DataType dataType) { return BASE64_ENCODER.encodeToString((byte[]) value); case ARRAY: return convertRowArray((InternalArray) value, (ArrayType) dataType); + case VECTOR: + return convertRowVector((InternalVector) value, (VectorType) dataType); case MAP: return convertRowMap((InternalMap) value, (MapType) dataType); case ROW: @@ -133,6 +138,14 @@ private List convertRowArray(InternalArray array, ArrayType arrayType) { return result; } + private List convertRowVector(InternalVector vector, VectorType vectorType) { + if (vector.size() != vectorType.getLength()) { + throw new IllegalArgumentException( + "Size " + vector.size() + " != " + vectorType.getLength() + " in JsonWriter"); + } + return convertRowArray(vector, DataTypes.ARRAY(vectorType.getElementType())); + } + private Map convertRowMap(InternalMap map, MapType mapType) { int size = map.size(); Map result = new LinkedHashMap<>(size); // Pre-allocate capacity diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/FieldWriterFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/FieldWriterFactory.java index 6d8e771a616e..762037deb215 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/FieldWriterFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/FieldWriterFactory.java @@ -46,6 +46,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; @@ -293,6 +294,11 @@ public FieldWriter visit(ArrayType arrayType) { }; } + @Override + public FieldWriter visit(VectorType vectorType) { + throw new UnsupportedOperationException("Unsupported type: " + vectorType); + } + @Override public FieldWriter visit(MapType mapType) { FieldWriter keyWriter = mapType.getKeyType().accept(this); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetVectorUpdaterFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetVectorUpdaterFactory.java index 79c343ed64be..880af82af181 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetVectorUpdaterFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetVectorUpdaterFactory.java @@ -57,6 +57,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; @@ -226,6 +227,11 @@ public UpdaterFactory visit(ArrayType arrayType) { throw new RuntimeException("Array type is not supported"); } + @Override + public UpdaterFactory visit(VectorType vectorType) { + throw new RuntimeException("Vector type is not supported"); + } + @Override public UpdaterFactory visit(MultisetType multisetType) { throw new RuntimeException("Multiset type is not supported"); diff --git a/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java b/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java index 1c2b2106b917..c680674d8e7a 100644 --- a/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java +++ b/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java @@ -46,6 +46,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.parquet.filter2.compat.FilterCompat; import org.apache.parquet.io.api.Binary; @@ -362,6 +363,11 @@ public Operators.Column visit(ArrayType arrayType) { throw new UnsupportedOperationException(); } + @Override + public Operators.Column visit(VectorType vectorType) { + throw new UnsupportedOperationException(); + } + @Override public Operators.Column visit(MultisetType multisetType) { throw new UnsupportedOperationException(); diff --git a/paimon-format/src/test/java/org/apache/paimon/format/json/JsonFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/json/JsonFileFormatTest.java index 92ad9c009819..40234094bb09 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/json/JsonFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/json/JsonFileFormatTest.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.GenericMap; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; @@ -378,6 +379,21 @@ public void testWithCustomLineDelimiters() throws IOException { } } + @Test + public void testVectorTypeReadWrite() throws IOException { + RowType rowType = DataTypes.ROW(DataTypes.INT(), DataTypes.VECTOR(3, DataTypes.FLOAT())); + + float[] values = new float[] {1.0f, 2.0f, 3.0f}; + List testData = + Arrays.asList(GenericRow.of(1, BinaryVector.fromPrimitiveArray(values))); + + List result = writeThenRead(new Options(), rowType, testData, "test_vector"); + + assertThat(result).hasSize(1); + assertThat(result.get(0).getInt(0)).isEqualTo(1); + assertThat(result.get(0).getVector(1).toFloatArray()).isEqualTo(values); + } + @Override public boolean supportDataFileWithoutExtension() { return true; diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/objectinspector/HivePaimonArray.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/objectinspector/HivePaimonArray.java index 291664851ad5..02d14d360625 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/objectinspector/HivePaimonArray.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/objectinspector/HivePaimonArray.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.ArrayType; @@ -133,6 +134,11 @@ public InternalArray getArray(int i) { ((HivePaimonArray) this.getAs(i)).getList()); } + @Override + public InternalVector getVector(int i) { + throw new UnsupportedOperationException("Not support VectorType yet."); + } + @Override public InternalMap getMap(int i) { return getAs(i); diff --git a/paimon-lance/src/main/java/org/apache/paimon/format/lance/LanceFileFormat.java b/paimon-lance/src/main/java/org/apache/paimon/format/lance/LanceFileFormat.java index 1592b6100cb3..64b4e2887f82 100644 --- a/paimon-lance/src/main/java/org/apache/paimon/format/lance/LanceFileFormat.java +++ b/paimon-lance/src/main/java/org/apache/paimon/format/lance/LanceFileFormat.java @@ -48,6 +48,7 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import javax.annotation.Nullable; @@ -192,6 +193,11 @@ public Void visit(ArrayType arrayType) { return null; } + @Override + public Void visit(VectorType vectorType) { + return null; + } + @Override public Void visit(MultisetType multisetType) { return null; diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRowWrapper.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRowWrapper.java index 7d0f051756a9..3ad3666e9760 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRowWrapper.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRowWrapper.java @@ -27,6 +27,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.spark.util.shim.TypeUtils$; @@ -262,6 +263,11 @@ public InternalArray getArray(int pos) { ((ArrayType) (tableSchema.fields()[pos].dataType())).elementType()); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("Not support VectorType yet."); + } + @Override public InternalMap getMap(int pos) { int actualPos = getActualFieldPosition(pos); @@ -427,6 +433,11 @@ public InternalArray getArray(int pos) { arrayData.getArray(pos), ((ArrayType) elementType).elementType()); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("Not support VectorType yet."); + } + @Override public InternalMap getMap(int pos) { MapType mapType = (MapType) elementType; diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java index 9ac1e5999422..cc947c7ea04b 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java @@ -27,6 +27,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.spark.util.shim.TypeUtils; @@ -181,6 +182,11 @@ public InternalArray getArray(int i) { return new PaimonArray(((ArrayType) type.getTypeAt(i)).getElementType(), row.getList(i)); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("Not support VectorType yet."); + } + @Override public InternalMap getMap(int i) { return toPaimonMap((MapType) type.getTypeAt(i), row.getJavaMap(i)); @@ -353,6 +359,11 @@ public InternalArray getArray(int i) { return new PaimonArray(((ArrayType) elementType).getElementType(), array); } + @Override + public InternalVector getVector(int pos) { + throw new UnsupportedOperationException("Not support VectorType yet."); + } + @Override public InternalMap getMap(int i) { Object o = getAs(i); From 1de12baf666dc3080e42b9e19b05c8e1cfe0349e Mon Sep 17 00:00:00 2001 From: ColdL Date: Thu, 29 Jan 2026 10:26:33 +0800 Subject: [PATCH 2/4] add columnar impl for InternalVector and link to arrow module --- .../org/apache/paimon/arrow/ArrowUtils.java | 15 +- .../Arrow2PaimonVectorConverter.java | 48 +++- .../ArrowFieldWriterFactoryVisitor.java | 10 +- .../arrow/writer/ArrowFieldWriters.java | 90 ++++++ .../apache/paimon/arrow/ArrowUtilsTest.java | 10 + .../ArrowVectorizedBatchConverterTest.java | 171 ++++++++++++ .../arrow/vector/ArrowFormatWriterTest.java | 27 ++ .../paimon/data/columnar/ColumnarArray.java | 2 +- .../paimon/data/columnar/ColumnarVec.java | 262 ++++++++++++++++++ .../paimon/data/columnar/VecColumnVector.java | 30 ++ .../data/columnar/VectorizedColumnBatch.java | 2 +- .../paimon/utils/VectorMappingUtils.java | 25 +- .../columnar/ColumnarRowWithVectorTest.java | 73 +++++ .../paimon/data/columnar/ColumnarVecTest.java | 61 ++++ .../paimon/utils/VectorMappingUtilsTest.java | 20 +- .../format/lance/LanceReaderWriterTest.java | 50 ++++ .../format/lance/VectorTypeWithLanceTest.java | 147 ++++++++++ 17 files changed, 1029 insertions(+), 14 deletions(-) create mode 100644 paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverterTest.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarVec.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/columnar/VecColumnVector.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/data/columnar/ColumnarRowWithVectorTest.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/data/columnar/ColumnarVecTest.java create mode 100644 paimon-lance/src/test/java/org/apache/paimon/format/lance/VectorTypeWithLanceTest.java diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java index 7b387b409e1b..041927b7fbd6 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java @@ -30,6 +30,7 @@ import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; import org.apache.paimon.types.VariantType; +import org.apache.paimon.types.VectorType; import org.apache.arrow.c.ArrowArray; import org.apache.arrow.c.ArrowSchema; @@ -137,14 +138,16 @@ public static Field toArrowField( fieldType.getDictionary(), Collections.singletonMap(PARQUET_FIELD_ID, String.valueOf(fieldId))); List children = null; - if (dataType instanceof ArrayType) { + if (dataType instanceof ArrayType || dataType instanceof VectorType) { + final DataType elementType; + if (dataType instanceof VectorType) { + elementType = ((VectorType) dataType).getElementType(); + } else { + elementType = ((ArrayType) dataType).getElementType(); + } Field field = toArrowField( - ListVector.DATA_VECTOR_NAME, - fieldId, - ((ArrayType) dataType).getElementType(), - depth + 1, - visitor); + ListVector.DATA_VECTOR_NAME, fieldId, elementType, depth + 1, visitor); FieldType typeInner = field.getFieldType(); field = new Field( diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/Arrow2PaimonVectorConverter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/Arrow2PaimonVectorConverter.java index c78aa324ae27..f4e18c66e4ab 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/Arrow2PaimonVectorConverter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/Arrow2PaimonVectorConverter.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.columnar.ArrayColumnVector; import org.apache.paimon.data.columnar.BooleanColumnVector; @@ -31,6 +32,7 @@ import org.apache.paimon.data.columnar.ColumnarArray; import org.apache.paimon.data.columnar.ColumnarMap; import org.apache.paimon.data.columnar.ColumnarRow; +import org.apache.paimon.data.columnar.ColumnarVec; import org.apache.paimon.data.columnar.DecimalColumnVector; import org.apache.paimon.data.columnar.DoubleColumnVector; import org.apache.paimon.data.columnar.FloatColumnVector; @@ -40,6 +42,7 @@ import org.apache.paimon.data.columnar.RowColumnVector; import org.apache.paimon.data.columnar.ShortColumnVector; import org.apache.paimon.data.columnar.TimestampColumnVector; +import org.apache.paimon.data.columnar.VecColumnVector; import org.apache.paimon.data.columnar.VectorizedColumnBatch; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; @@ -83,6 +86,7 @@ import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.complex.FixedSizeListVector; import org.apache.arrow.vector.complex.ListVector; import org.apache.arrow.vector.complex.StructVector; @@ -485,7 +489,49 @@ public ColumnVector getColumnVector() { @Override public Arrow2PaimonVectorConverter visit(VectorType vectorType) { - throw new UnsupportedOperationException("Doesn't support VectorType."); + final Arrow2PaimonVectorConverter arrowVectorConvertor = + vectorType.getElementType().accept(this); + + return vector -> + new VecColumnVector() { + + private boolean inited = false; + private ColumnVector columnVector; + + private void init() { + if (!inited) { + FieldVector child = ((FixedSizeListVector) vector).getDataVector(); + this.columnVector = arrowVectorConvertor.convertVector(child); + inited = true; + } + } + + @Override + public boolean isNullAt(int index) { + return vector.isNull(index); + } + + @Override + public InternalVector getVector(int index) { + init(); + FixedSizeListVector listVector = (FixedSizeListVector) vector; + int start = listVector.getElementStartIndex(index); + int end = listVector.getElementEndIndex(index); + return new ColumnarVec(columnVector, start, end - start); + } + + @Override + public ColumnVector getColumnVector() { + init(); + return columnVector; + } + + @Override + public int getVectorSize() { + FixedSizeListVector listVector = (FixedSizeListVector) vector; + return listVector.getListSize(); + } + }; } @Override diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriterFactoryVisitor.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriterFactoryVisitor.java index d07762106ebc..ccff6d6a24f6 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriterFactoryVisitor.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriterFactoryVisitor.java @@ -44,6 +44,7 @@ import org.apache.paimon.types.VectorType; import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.complex.FixedSizeListVector; import org.apache.arrow.vector.complex.ListVector; import org.apache.arrow.vector.complex.MapVector; @@ -168,7 +169,14 @@ public ArrowFieldWriterFactory visit(ArrayType arrayType) { @Override public ArrowFieldWriterFactory visit(VectorType vectorType) { - throw new UnsupportedOperationException("Doesn't support VectorType."); + ArrowFieldWriterFactory elementWriterFactory = vectorType.getElementType().accept(this); + return (fieldVector, isNullable) -> + new ArrowFieldWriters.VectorWriter( + fieldVector, + vectorType.getLength(), + elementWriterFactory.create( + ((FixedSizeListVector) fieldVector).getDataVector(), isNullable), + isNullable); } @Override diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriters.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriters.java index 409b1f9ab2ec..7cb64de7fd88 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriters.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriters.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.columnar.ArrayColumnVector; import org.apache.paimon.data.columnar.BooleanColumnVector; @@ -40,6 +41,7 @@ import org.apache.paimon.data.columnar.RowColumnVector; import org.apache.paimon.data.columnar.ShortColumnVector; import org.apache.paimon.data.columnar.TimestampColumnVector; +import org.apache.paimon.data.columnar.VecColumnVector; import org.apache.paimon.data.columnar.VectorizedColumnBatch; import org.apache.paimon.data.variant.GenericVariant; import org.apache.paimon.data.variant.PaimonShreddingUtils; @@ -65,6 +67,7 @@ import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.complex.FixedSizeListVector; import org.apache.arrow.vector.complex.ListVector; import org.apache.arrow.vector.complex.MapVector; import org.apache.arrow.vector.complex.StructVector; @@ -711,6 +714,93 @@ protected void doWrite(int rowIndex, DataGetters getters, int pos) { } } + /** Writer for VECTOR. */ + public static class VectorWriter extends ArrowFieldWriter { + + private final ArrowFieldWriter elementWriter; + + private final int length; + + public VectorWriter( + FieldVector fieldVector, + int length, + ArrowFieldWriter elementWriter, + boolean isNullable) { + super(fieldVector, isNullable); + this.length = length; + this.elementWriter = elementWriter; + } + + @Override + public void reset() { + super.reset(); + elementWriter.reset(); + } + + @Override + protected void doWrite( + ColumnVector columnVector, + @Nullable int[] pickedInColumn, + int startIndex, + int batchRows) { + VecColumnVector vecColumnVector = (VecColumnVector) columnVector; + + if (pickedInColumn == null) { + elementWriter.write( + vecColumnVector.getColumnVector(), + null, + startIndex * length, + batchRows * length); + } else { + int[] childPickedInColumn = new int[batchRows * length]; + for (int i = 0; i < batchRows; ++i) { + int pickedIndexInChild = pickedInColumn[startIndex + i] * length; + for (int j = 0; j < length; ++j) { + childPickedInColumn[i * length + j] = pickedIndexInChild + j; + } + } + elementWriter.write( + vecColumnVector.getColumnVector(), + childPickedInColumn, + 0, + batchRows * length); + } + + // set FixedSizeListVector + FixedSizeListVector listVector = (FixedSizeListVector) fieldVector; + for (int i = 0; i < batchRows; i++) { + int row = getRowNumber(startIndex, i, pickedInColumn); + if (vecColumnVector.isNullAt(row)) { + listVector.setNull(i); + } else { + listVector.startNewValue(i); + } + } + } + + @Override + protected void doWrite(int rowIndex, DataGetters getters, int pos) { + InternalVector vector = getters.getVector(pos); + if (vector.size() != length) { + throw new IllegalArgumentException( + String.format( + "The size of vector %s is not equal to the length %s", + vector.size(), length)); + } + FixedSizeListVector listVector = (FixedSizeListVector) fieldVector; + listVector.setNotNull(rowIndex); + final int rowBase = rowIndex * length; + for (int vectorIndex = 0; vectorIndex < length; ++vectorIndex) { + elementWriter.write(rowBase + vectorIndex, vector, vectorIndex); + } + // Ensure child value count is large enough. + listVector + .getDataVector() + .setValueCount( + Math.max(listVector.getDataVector().getValueCount(), rowBase + length)); + } + } + /** Writer for MAP. */ public static class MapWriter extends ArrowFieldWriter { diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/ArrowUtilsTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/ArrowUtilsTest.java index 49af9751509d..ee613a05d706 100644 --- a/paimon-arrow/src/test/java/org/apache/paimon/arrow/ArrowUtilsTest.java +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/ArrowUtilsTest.java @@ -24,6 +24,7 @@ import org.apache.paimon.types.RowType; import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.FieldType; import org.assertj.core.api.Assertions; @@ -108,4 +109,13 @@ public void testMap() { .getFieldType(); Assertions.assertThat(fieldType.isNullable()).isTrue(); } + + @Test + public void testVectorType() { + Field field = + ArrowUtils.toArrowField("embed", 0, DataTypes.VECTOR(4, DataTypes.FLOAT()), 0); + Assertions.assertThat(field.getFieldType().getType()) + .isEqualTo(new ArrowType.FixedSizeList(4)); + Assertions.assertThat(field.getChildren()).hasSize(1); + } } diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverterTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverterTest.java new file mode 100644 index 000000000000..4c7e2bb2838a --- /dev/null +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverterTest.java @@ -0,0 +1,171 @@ +/* + * 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.paimon.arrow.converter; + +import org.apache.paimon.arrow.ArrowUtils; +import org.apache.paimon.arrow.writer.ArrowFieldWriter; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; +import org.apache.paimon.data.columnar.ColumnVector; +import org.apache.paimon.data.columnar.ColumnarVec; +import org.apache.paimon.data.columnar.VecColumnVector; +import org.apache.paimon.data.columnar.VectorizedColumnBatch; +import org.apache.paimon.data.columnar.heap.HeapFloatVector; +import org.apache.paimon.reader.VectorizedRecordIterator; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.FixedSizeListVector; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link ArrowVectorizedBatchConverter}. */ +public class ArrowVectorizedBatchConverterTest { + + @Test + public void testVectorColumnWrite() { + RowType rowType = RowType.of(DataTypes.VECTOR(3, DataTypes.FLOAT())); + try (RootAllocator allocator = new RootAllocator()) { + VectorSchemaRoot vsr = ArrowUtils.createVectorSchemaRoot(rowType, allocator); + ArrowFieldWriter[] fieldWriters = ArrowUtils.createArrowFieldWriters(vsr, rowType); + + int length = 3; + int rows = 2; + HeapFloatVector elementVector = new HeapFloatVector(rows * length); + float[] values = new float[] {1.0f, 2.0f, 3.0f, 4.0f, 5.0f, 6.0f}; + for (int i = 0; i < values.length; i++) { + elementVector.setFloat(i, values[i]); + } + VecColumnVector vector = + new VecColumnVector() { + @Override + public InternalVector getVector(int i) { + return new ColumnarVec(elementVector, i * length, length); + } + + @Override + public ColumnVector getColumnVector() { + return elementVector; + } + + @Override + public int getVectorSize() { + return length; + } + + @Override + public boolean isNullAt(int i) { + return i % 2 == 1; + } + }; + + VectorizedColumnBatch batch = new VectorizedColumnBatch(new ColumnVector[] {vector}); + batch.setNumRows(rows); + + ArrowVectorizedBatchConverter converter = + new ArrowVectorizedBatchConverter(vsr, fieldWriters); + converter.reset( + new VectorizedRecordIterator() { + @Override + public VectorizedColumnBatch batch() { + return batch; + } + + @Override + public InternalRow next() { + return null; + } + + @Override + public void releaseBatch() {} + }); + converter.next(rows); + + FixedSizeListVector listVector = (FixedSizeListVector) vsr.getVector(0); + assertThat(listVector.isNull(0)).isFalse(); + assertThat(listVector.isNull(1)).isTrue(); + + @SuppressWarnings("unchecked") + List row0 = (List) listVector.getObject(0); + assertThat(row0).containsExactly(1.0f, 2.0f, 3.0f); + assertThat(listVector.getObject(1)).isNull(); + + converter.close(); + } + } + + @Test + public void testVectorColumnWriteWithPickedInColumn() { + RowType rowType = RowType.of(DataTypes.VECTOR(2, DataTypes.FLOAT())); + try (RootAllocator allocator = new RootAllocator()) { + VectorSchemaRoot vsr = ArrowUtils.createVectorSchemaRoot(rowType, allocator); + ArrowFieldWriter[] fieldWriters = ArrowUtils.createArrowFieldWriters(vsr, rowType); + + int length = 2; + int rows = 4; + HeapFloatVector elementVector = new HeapFloatVector(rows * length); + float[] values = new float[] {1.0f, 2.0f, 3.0f, 4.0f, 5.0f, 6.0f, 7.0f, 8.0f}; + for (int i = 0; i < values.length; i++) { + elementVector.setFloat(i, values[i]); + } + + VecColumnVector vector = + new VecColumnVector() { + @Override + public InternalVector getVector(int i) { + return new ColumnarVec(elementVector, i * length, length); + } + + @Override + public ColumnVector getColumnVector() { + return elementVector; + } + + @Override + public int getVectorSize() { + return length; + } + + @Override + public boolean isNullAt(int i) { + return false; + } + }; + + int[] pickedInColumn = new int[] {2, 0}; + fieldWriters[0].reset(); + fieldWriters[0].write(vector, pickedInColumn, 0, pickedInColumn.length); + + FixedSizeListVector listVector = (FixedSizeListVector) vsr.getVector(0); + @SuppressWarnings("unchecked") + List row0 = (List) listVector.getObject(0); + assertThat(row0).containsExactly(5.0f, 6.0f); + @SuppressWarnings("unchecked") + List row1 = (List) listVector.getObject(1); + assertThat(row1).containsExactly(1.0f, 2.0f); + + vsr.close(); + } + } +} diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/ArrowFormatWriterTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/ArrowFormatWriterTest.java index ec1fe10c46a4..a716264807ef 100644 --- a/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/ArrowFormatWriterTest.java +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/ArrowFormatWriterTest.java @@ -24,6 +24,7 @@ import org.apache.paimon.arrow.reader.ArrowBatchReader; import org.apache.paimon.arrow.writer.ArrowFieldWriterFactoryVisitor; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.GenericArray; import org.apache.paimon.data.GenericMap; @@ -140,6 +141,32 @@ public void testWrite() { } } + @Test + public void testWriteVector() { + RowType rowType = + new RowType( + Arrays.asList( + new DataField(0, "id", DataTypes.INT()), + new DataField(1, "embed", DataTypes.VECTOR(3, DataTypes.FLOAT())))); + float[] values = new float[] {1.0f, 2.0f, 3.0f}; + try (ArrowFormatWriter writer = new ArrowFormatWriter(rowType, 16, true)) { + writer.write(GenericRow.of(1, BinaryVector.fromPrimitiveArray(values))); + + writer.flush(); + VectorSchemaRoot vectorSchemaRoot = writer.getVectorSchemaRoot(); + + ArrowBatchReader arrowBatchReader = new ArrowBatchReader(rowType, true); + Iterable rows = arrowBatchReader.readBatch(vectorSchemaRoot); + + Iterator iterator = rows.iterator(); + InternalRow row = iterator.next(); + + assertThat(row.getInt(0)).isEqualTo(1); + assertThat(row.getVector(1).toFloatArray()).isEqualTo(values); + vectorSchemaRoot.close(); + } + } + @Test public void testWriteVariant() { RowType rowType = new RowType(Arrays.asList(new DataField(0, "v", DataTypes.VARIANT()))); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarArray.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarArray.java index ad04f647b977..28221cec0db1 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarArray.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarArray.java @@ -146,7 +146,7 @@ public InternalArray getArray(int pos) { @Override public InternalVector getVector(int pos) { - throw new UnsupportedOperationException("Unsupported type: VectorType"); + return ((VecColumnVector) data).getVector(offset + pos); } @Override diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarVec.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarVec.java new file mode 100644 index 000000000000..de07351dff76 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarVec.java @@ -0,0 +1,262 @@ +/* + * 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.paimon.data.columnar; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.DataSetters; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.data.variant.Variant; + +import java.io.Serializable; + +/** Columnar VectorType to support access to vector column data. */ +public final class ColumnarVec implements InternalVector, DataSetters, Serializable { + + private static final long serialVersionUID = 1L; + + private final ColumnVector data; + private final int offset; + private final int numElements; + + public ColumnarVec(ColumnVector data, int offset, int numElements) { + this.data = data; + this.offset = offset; + this.numElements = numElements; + } + + @Override + public int size() { + return numElements; + } + + @Override + public boolean isNullAt(int pos) { + return data.isNullAt(offset + pos); + } + + @Override + public void setNullAt(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean getBoolean(int pos) { + return ((BooleanColumnVector) data).getBoolean(offset + pos); + } + + @Override + public byte getByte(int pos) { + return ((ByteColumnVector) data).getByte(offset + pos); + } + + @Override + public short getShort(int pos) { + return ((ShortColumnVector) data).getShort(offset + pos); + } + + @Override + public int getInt(int pos) { + return ((IntColumnVector) data).getInt(offset + pos); + } + + @Override + public long getLong(int pos) { + return ((LongColumnVector) data).getLong(offset + pos); + } + + @Override + public float getFloat(int pos) { + return ((FloatColumnVector) data).getFloat(offset + pos); + } + + @Override + public double getDouble(int pos) { + return ((DoubleColumnVector) data).getDouble(offset + pos); + } + + @Override + public BinaryString getString(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public byte[] getBinary(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public Blob getBlob(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public InternalArray getArray(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public InternalVector getVector(int pos) { + return ((VecColumnVector) data).getVector(offset + pos); + } + + @Override + public InternalMap getMap(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setBoolean(int pos, boolean value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setByte(int pos, byte value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setShort(int pos, short value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setInt(int pos, int value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setLong(int pos, long value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setFloat(int pos, float value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDouble(int pos, double value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDecimal(int pos, Decimal value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setTimestamp(int pos, Timestamp value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean[] toBooleanArray() { + boolean[] res = new boolean[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getBoolean(i); + } + return res; + } + + @Override + public byte[] toByteArray() { + byte[] res = new byte[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getByte(i); + } + return res; + } + + @Override + public short[] toShortArray() { + short[] res = new short[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getShort(i); + } + return res; + } + + @Override + public int[] toIntArray() { + int[] res = new int[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getInt(i); + } + return res; + } + + @Override + public long[] toLongArray() { + long[] res = new long[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getLong(i); + } + return res; + } + + @Override + public float[] toFloatArray() { + float[] res = new float[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getFloat(i); + } + return res; + } + + @Override + public double[] toDoubleArray() { + double[] res = new double[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getDouble(i); + } + return res; + } + + @Override + public boolean equals(Object o) { + throw new UnsupportedOperationException( + "ColumnarVector do not support equals, please compare fields one by one!"); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/VecColumnVector.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/VecColumnVector.java new file mode 100644 index 000000000000..e6df065cbf68 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/VecColumnVector.java @@ -0,0 +1,30 @@ +/* + * 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.paimon.data.columnar; + +import org.apache.paimon.data.InternalVector; + +/** Column vector for VectorType. */ +public interface VecColumnVector extends ColumnVector { + InternalVector getVector(int i); + + ColumnVector getColumnVector(); + + int getVectorSize(); +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/VectorizedColumnBatch.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/VectorizedColumnBatch.java index 297d0b0f9082..01c6037ca688 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/VectorizedColumnBatch.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/VectorizedColumnBatch.java @@ -126,7 +126,7 @@ public InternalArray getArray(int rowId, int colId) { } public InternalVector getVector(int rowId, int colId) { - throw new UnsupportedOperationException("Unsupported type: VectorType"); + return ((VecColumnVector) columns[colId]).getVector(rowId); } public InternalRow getRow(int rowId, int colId) { diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java index b7f92d06d95c..99e8fd455c41 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.columnar.ArrayColumnVector; @@ -39,6 +40,7 @@ import org.apache.paimon.data.columnar.RowColumnVector; import org.apache.paimon.data.columnar.ShortColumnVector; import org.apache.paimon.data.columnar.TimestampColumnVector; +import org.apache.paimon.data.columnar.VecColumnVector; import org.apache.paimon.data.columnar.VectorizedColumnBatch; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; @@ -357,7 +359,28 @@ public ColumnVector getColumnVector() { @Override public ColumnVector visit(VectorType vectorType) { - throw new UnsupportedOperationException("VectorType is not supported."); + return new VecColumnVector() { + @Override + public InternalVector getVector(int i) { + return partition.getVector(index); + } + + @Override + public boolean isNullAt(int i) { + return partition.isNullAt(index); + } + + @Override + public int getVectorSize() { + return partition.getVector(index).size(); + } + + @Override + public ColumnVector getColumnVector() { + throw new UnsupportedOperationException( + "Doesn't support getting ColumnVector."); + } + }; } @Override diff --git a/paimon-common/src/test/java/org/apache/paimon/data/columnar/ColumnarRowWithVectorTest.java b/paimon-common/src/test/java/org/apache/paimon/data/columnar/ColumnarRowWithVectorTest.java new file mode 100644 index 000000000000..98e24888bb7e --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/data/columnar/ColumnarRowWithVectorTest.java @@ -0,0 +1,73 @@ +/* + * 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.paimon.data.columnar; + +import org.apache.paimon.data.InternalVector; +import org.apache.paimon.data.columnar.heap.HeapFloatVector; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for vector access in {@link ColumnarRow}. */ +public class ColumnarRowWithVectorTest { + + @Test + public void testVectorAccess() { + int length = 3; + float[] values = new float[] {1.0f, 2.0f, 3.0f, 4.0f, 5.0f, 6.0f}; + + HeapFloatVector elementVector = new HeapFloatVector(values.length); + for (int i = 0; i < values.length; i++) { + elementVector.setFloat(i, values[i]); + } + + VecColumnVector vector = + new VecColumnVector() { + @Override + public InternalVector getVector(int i) { + return new ColumnarVec(elementVector, i * length, length); + } + + @Override + public ColumnVector getColumnVector() { + return elementVector; + } + + @Override + public int getVectorSize() { + return length; + } + + @Override + public boolean isNullAt(int i) { + return false; + } + }; + VectorizedColumnBatch batch = new VectorizedColumnBatch(new ColumnVector[] {vector}); + batch.setNumRows(2); + + ColumnarRow row = new ColumnarRow(batch); + row.setRowId(0); + assertThat(row.getVector(0).toFloatArray()).isEqualTo(new float[] {1.0f, 2.0f, 3.0f}); + + row.setRowId(1); + assertThat(row.getVector(0).toFloatArray()).isEqualTo(new float[] {4.0f, 5.0f, 6.0f}); + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/data/columnar/ColumnarVecTest.java b/paimon-common/src/test/java/org/apache/paimon/data/columnar/ColumnarVecTest.java new file mode 100644 index 000000000000..7be1a0d40f04 --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/data/columnar/ColumnarVecTest.java @@ -0,0 +1,61 @@ +/* + * 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.paimon.data.columnar; + +import org.apache.paimon.data.columnar.heap.HeapFloatVector; +import org.apache.paimon.data.columnar.heap.HeapIntVector; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link ColumnarVec}. */ +public class ColumnarVecTest { + + @Test + public void testIntVectorAccess() { + HeapIntVector intVector = new HeapIntVector(6); + intVector.setInt(0, 1); + intVector.setInt(1, 2); + intVector.setInt(2, 3); + intVector.setInt(3, 4); + intVector.setInt(4, 5); + intVector.setInt(5, 6); + + ColumnarVec vector = new ColumnarVec(intVector, 1, 3); + + assertThat(vector.size()).isEqualTo(3); + assertThat(vector.toIntArray()).isEqualTo(new int[] {2, 3, 4}); + } + + @Test + public void testFloatVectorAccess() { + HeapFloatVector floatVector = new HeapFloatVector(5); + floatVector.setFloat(0, 1.0f); + floatVector.setFloat(1, 2.0f); + floatVector.setFloat(2, 3.0f); + floatVector.setFloat(3, 4.0f); + floatVector.setFloat(4, 5.0f); + + ColumnarVec vector = new ColumnarVec(floatVector, 2, 2); + + assertThat(vector.size()).isEqualTo(2); + assertThat(vector.toFloatArray()).isEqualTo(new float[] {3.0f, 4.0f}); + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/utils/VectorMappingUtilsTest.java b/paimon-common/src/test/java/org/apache/paimon/utils/VectorMappingUtilsTest.java index 571a0d7189d6..34488c61e116 100644 --- a/paimon-common/src/test/java/org/apache/paimon/utils/VectorMappingUtilsTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/utils/VectorMappingUtilsTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.columnar.BooleanColumnVector; @@ -33,6 +34,8 @@ import org.apache.paimon.data.columnar.LongColumnVector; import org.apache.paimon.data.columnar.ShortColumnVector; import org.apache.paimon.data.columnar.TimestampColumnVector; +import org.apache.paimon.data.columnar.VecColumnVector; +import org.apache.paimon.data.serializer.InternalVectorSerializer; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -106,10 +109,11 @@ public void testForType() { DataTypes.DATE(), DataTypes.TIME(), DataTypes.TIMESTAMP(), - DataTypes.FLOAT()) + DataTypes.FLOAT(), + DataTypes.VECTOR(3, DataTypes.FLOAT())) .build(); - BinaryRow binaryRow = new BinaryRow(13); + BinaryRow binaryRow = new BinaryRow(14); BinaryRowWriter binaryRowWriter = new BinaryRowWriter(binaryRow); binaryRowWriter.writeInt(0, 0); binaryRowWriter.writeByte(1, (byte) 1); @@ -125,9 +129,14 @@ public void testForType() { binaryRowWriter.writeTimestamp( 11, Timestamp.fromEpochMillis(System.currentTimeMillis()), 10); binaryRowWriter.writeFloat(12, (float) 12.0); + float[] vectorValues = new float[] {1.0f, 2.0f, 3.0f}; + InternalVectorSerializer vectorSerializer = + new InternalVectorSerializer(DataTypes.FLOAT(), vectorValues.length); + binaryRowWriter.writeVector( + 13, BinaryVector.fromPrimitiveArray(vectorValues), vectorSerializer); binaryRowWriter.complete(); - int[] map = {-1, -2, -3, -4, -5, -6, 1, -7, -8, -9, -10, -11, -12, -13, 0}; + int[] map = {-1, -2, -3, -4, -5, -6, 1, -7, -8, -9, -10, -11, -12, -13, -14, 0}; PartitionInfo partitionInfo = new PartitionInfo(map, rowType, binaryRow); ColumnVector[] columnVectors = new ColumnVector[1]; @@ -181,5 +190,10 @@ public void testForType() { Assertions.assertThat(newColumnVectors[13]).isInstanceOf(FloatColumnVector.class); Assertions.assertThat(((FloatColumnVector) newColumnVectors[13]).getFloat(0)) .isEqualTo((float) 12.0); + + Assertions.assertThat(newColumnVectors[14]).isInstanceOf(VecColumnVector.class); + VecColumnVector vecColumnVector = (VecColumnVector) newColumnVectors[14]; + Assertions.assertThat(vecColumnVector.getVectorSize()).isEqualTo(vectorValues.length); + Assertions.assertThat(vecColumnVector.getVector(0).toFloatArray()).isEqualTo(vectorValues); } } diff --git a/paimon-lance/src/test/java/org/apache/paimon/format/lance/LanceReaderWriterTest.java b/paimon-lance/src/test/java/org/apache/paimon/format/lance/LanceReaderWriterTest.java index c6e5937d6554..4541ca6eb635 100644 --- a/paimon-lance/src/test/java/org/apache/paimon/format/lance/LanceReaderWriterTest.java +++ b/paimon-lance/src/test/java/org/apache/paimon/format/lance/LanceReaderWriterTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.format.lance; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalRowSerializer; @@ -43,6 +44,7 @@ import java.util.List; import java.util.UUID; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -93,4 +95,52 @@ public void testWriteAndRead(@TempDir java.nio.file.Path tempDir) throws Excepti } } } + + @Test + public void testWriteAndReadVector(@TempDir java.nio.file.Path tempDir) throws Exception { + RowType rowType = RowType.of(DataTypes.INT(), DataTypes.VECTOR(3, DataTypes.FLOAT())); + Options options = new Options(); + LanceFileFormat format = + new LanceFileFormatFactory() + .create(new FileFormatFactory.FormatContext(options, 1024, 1024)); + + FileIO fileIO = new LocalFileIO(); + Path testFile = new Path(tempDir.resolve("test_vector_" + UUID.randomUUID()).toString()); + + float[] values1 = new float[] {1.0f, 2.0f, 3.0f}; + float[] values2 = new float[] {4.0f, 5.0f, 6.0f}; + + // Write data + List expectedRows = new ArrayList<>(); + try (FormatWriter writer = + ((SupportsDirectWrite) format.createWriterFactory(rowType)) + .create(fileIO, testFile, "")) { + expectedRows.add(GenericRow.of(1, BinaryVector.fromPrimitiveArray(values1))); + writer.addElement(expectedRows.get(0)); + expectedRows.add(GenericRow.of(2, BinaryVector.fromPrimitiveArray(values2))); + writer.addElement(expectedRows.get(1)); + } + + InternalRowSerializer internalRowSerializer = new InternalRowSerializer(rowType); + // Read data and check + FormatReaderFactory readerFactory = format.createReaderFactory(rowType, rowType, null); + try (RecordReader reader = + readerFactory.createReader( + new FormatReaderContext( + fileIO, testFile, fileIO.getFileSize(testFile), null)); + RecordReaderIterator iterator = new RecordReaderIterator<>(reader)) { + assertNotNull(reader); + + List actualRows = new ArrayList<>(); + while (iterator.hasNext()) { + actualRows.add(internalRowSerializer.copy(iterator.next())); + } + + assertEquals(expectedRows.size(), actualRows.size()); + assertEquals(expectedRows.get(0).getInt(0), actualRows.get(0).getInt(0)); + assertArrayEquals(values1, actualRows.get(0).getVector(1).toFloatArray(), 0.0f); + assertEquals(expectedRows.get(1).getInt(0), actualRows.get(1).getInt(0)); + assertArrayEquals(values2, actualRows.get(1).getVector(1).toFloatArray(), 0.0f); + } + } } diff --git a/paimon-lance/src/test/java/org/apache/paimon/format/lance/VectorTypeWithLanceTest.java b/paimon-lance/src/test/java/org/apache/paimon/format/lance/VectorTypeWithLanceTest.java new file mode 100644 index 000000000000..1fbe53fdc745 --- /dev/null +++ b/paimon-lance/src/test/java/org/apache/paimon/format/lance/VectorTypeWithLanceTest.java @@ -0,0 +1,147 @@ +/* + * 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.paimon.format.lance; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.FileStatus; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.RemoteIterator; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.TableTestBase; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.utils.TraceableFileIO; + +import com.lancedb.lance.file.LanceFileReader; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.paimon.options.CatalogOptions.WAREHOUSE; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +/** Tests for table with vector and lance file format. */ +public class VectorTypeWithLanceTest extends TableTestBase { + + private final float[] testVector = randomVector(); + + @BeforeEach + public void beforeEach() throws Catalog.DatabaseAlreadyExistException { + database = "default"; + warehouse = new Path(TraceableFileIO.SCHEME + "://" + tempPath.toString()); + Options options = new Options(); + options.set(WAREHOUSE, warehouse.toUri().toString()); + CatalogContext context = CatalogContext.create(options, new TraceableFileIO.Loader(), null); + catalog = CatalogFactory.createCatalog(context); + catalog.createDatabase(database, true); + } + + @Test + public void testBasic() throws Exception { + createTableDefault(); + + commitDefault(writeDataDefault(100, 1)); + + AtomicInteger integer = new AtomicInteger(0); + + readDefault( + row -> { + integer.incrementAndGet(); + if (integer.get() % 50 == 0) { + Assertions.assertArrayEquals( + row.getVector(2).toFloatArray(), testVector, 0); + } + }); + + assertThat(integer.get()).isEqualTo(100); + + FileStoreTable table = getTableDefault(); + RemoteIterator files = + table.fileIO().listFilesIterative(table.location(), true); + while (files.hasNext()) { + String file = files.next().getPath().toString(); + if (file.endsWith(".lance")) { + checkFileByLanceReader(file); + return; + } + } + Assertions.fail("Do not find any lance file."); + } + + @Override + protected Schema schemaDefault() { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("f0", DataTypes.INT()); + schemaBuilder.column("f1", DataTypes.STRING()); + schemaBuilder.column("f2", DataTypes.VECTOR(testVector.length, DataTypes.FLOAT())); + // schemaBuilder.option(CoreOptions.TARGET_FILE_SIZE.key(), "25 MB"); + schemaBuilder.option(CoreOptions.FILE_FORMAT.key(), "lance"); + schemaBuilder.option(CoreOptions.FILE_COMPRESSION.key(), "none"); + return schemaBuilder.build(); + } + + @Override + protected InternalRow dataDefault(int time, int size) { + return GenericRow.of( + RANDOM.nextInt(), + BinaryString.fromBytes(randomBytes()), + BinaryVector.fromPrimitiveArray(testVector)); + } + + @Override + protected byte[] randomBytes() { + byte[] binary = new byte[RANDOM.nextInt(1024) + 1]; + RANDOM.nextBytes(binary); + return binary; + } + + private float[] randomVector() { + byte[] randomBytes = randomBytes(); + float[] vector = new float[randomBytes.length]; + for (int i = 0; i < vector.length; i++) { + vector[i] = randomBytes[i]; + } + return vector; + } + + private void checkFileByLanceReader(String path) throws Exception { + ArrowType expected = new ArrowType.FixedSizeList(testVector.length); + RootAllocator allocator = new RootAllocator(); + Map options = new HashMap<>(); + try (LanceFileReader reader = LanceFileReader.open(path, options, allocator)) { + org.apache.arrow.vector.types.pojo.Schema schema = reader.schema(); + org.apache.arrow.vector.types.pojo.Field field = schema.findField("f2"); + Assertions.assertEquals(expected, field.getFieldType().getType()); + } + } +} From 35481366ee401c456ce9d679ecddb9883e44f625 Mon Sep 17 00:00:00 2001 From: ColdL Date: Thu, 29 Jan 2026 10:26:53 +0800 Subject: [PATCH 3/4] add flink support for VectorType --- .../paimon/flink/DataTypeToLogicalType.java | 3 +- .../org/apache/paimon/flink/FlinkCatalog.java | 25 +++- .../apache/paimon/flink/FlinkRowWrapper.java | 10 +- .../paimon/flink/LogicalTypeConversion.java | 15 +++ .../flink/FlinkRowDataWithVectorTest.java | 51 ++++++++ .../flink/FlinkRowWrapperWithVectorTest.java | 48 +++++++ .../flink/LogicalTypeConversionTest.java | 59 +++++++++ .../paimon/flink/VectorTypeTableITCase.java | 121 ++++++++++++++++++ .../utils/InternalRowSerializerTest.java | 25 ++++ 9 files changed, 351 insertions(+), 6 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkRowDataWithVectorTest.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkRowWrapperWithVectorTest.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LogicalTypeConversionTest.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/VectorTypeTableITCase.java diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataTypeToLogicalType.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataTypeToLogicalType.java index 6fc3016d4f95..16a270587d2e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataTypeToLogicalType.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataTypeToLogicalType.java @@ -162,7 +162,8 @@ public LogicalType visit(ArrayType arrayType) { @Override public LogicalType visit(VectorType vectorType) { - throw new UnsupportedOperationException("Not support VectorType yet."); + return new org.apache.flink.table.types.logical.ArrayType( + vectorType.isNullable(), vectorType.getElementType().accept(this)); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 2239114cca10..1305ec95b45a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -155,6 +155,7 @@ import static org.apache.paimon.flink.LogicalTypeConversion.toBlobType; import static org.apache.paimon.flink.LogicalTypeConversion.toDataType; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; +import static org.apache.paimon.flink.LogicalTypeConversion.toVectorType; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn; @@ -1041,14 +1042,32 @@ public static Schema fromCatalogTable(CatalogBaseTable catalogTable) { field -> schemaBuilder.column( field.getName(), - blobFields.contains(field.getName()) - ? toBlobType(field.getType()) - : toDataType(field.getType()), + resolveDataType(field.getName(), field.getType(), options), columnComments.get(field.getName()))); return schemaBuilder.build(); } + private static org.apache.paimon.types.DataType resolveDataType( + String fieldName, + org.apache.flink.table.types.logical.LogicalType logicalType, + Map options) { + List blobFields = CoreOptions.blobField(options); + if (blobFields.contains(fieldName)) { + return toBlobType(logicalType); + } + if (logicalType instanceof org.apache.flink.table.types.logical.ArrayType) { + String vectorDim = options.get(String.format("field.%s.vector-dim", fieldName)); + if (vectorDim != null) { + org.apache.flink.table.types.logical.LogicalType elementType = + ((org.apache.flink.table.types.logical.ArrayType) logicalType) + .getElementType(); + return toVectorType(elementType, vectorDim); + } + } + return toDataType(logicalType); + } + private static Map getColumnComments(CatalogBaseTable catalogTable) { return catalogTable.getUnresolvedSchema().getColumns().stream() .filter(c -> c.getComment().isPresent()) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowWrapper.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowWrapper.java index ae8a4fc59211..3201f594fe59 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowWrapper.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowWrapper.java @@ -163,7 +163,7 @@ public InternalArray getArray(int pos) { @Override public InternalVector getVector(int pos) { - throw new UnsupportedOperationException("Not support VectorType yet."); + return new FlinkVectorWrapper(row.getArray(pos)); } @Override @@ -268,7 +268,7 @@ public InternalArray getArray(int pos) { @Override public InternalVector getVector(int pos) { - throw new UnsupportedOperationException("Not support VectorType yet."); + return new FlinkVectorWrapper(array.getArray(pos)); } @Override @@ -317,6 +317,12 @@ public double[] toDoubleArray() { } } + private static class FlinkVectorWrapper extends FlinkArrayWrapper implements InternalVector { + private FlinkVectorWrapper(org.apache.flink.table.data.ArrayData array) { + super(array); + } + } + private static class FlinkMapWrapper implements InternalMap { private final org.apache.flink.table.data.MapData map; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/LogicalTypeConversion.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/LogicalTypeConversion.java index 8c7779e318e3..c83e85d6bccc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/LogicalTypeConversion.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/LogicalTypeConversion.java @@ -20,7 +20,9 @@ import org.apache.paimon.types.BlobType; import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VectorType; import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.LogicalType; @@ -49,6 +51,19 @@ public static BlobType toBlobType(LogicalType logicalType) { return new BlobType(); } + public static VectorType toVectorType(LogicalType elementType, String vectorDim) { + checkArgument( + !vectorDim.trim().isEmpty(), + "Expected an integer for vector-dim, but got empty value."); + try { + int dim = Integer.parseInt(vectorDim); + return DataTypes.VECTOR(dim, toDataType(elementType)); + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "Expected an integer for vector-dim, but got: " + vectorDim); + } + } + public static RowType toDataType(org.apache.flink.table.types.logical.RowType logicalType) { return (RowType) toDataType(logicalType, new AtomicInteger(-1)); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkRowDataWithVectorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkRowDataWithVectorTest.java new file mode 100644 index 000000000000..ba8542d39519 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkRowDataWithVectorTest.java @@ -0,0 +1,51 @@ +/* + * 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.paimon.flink; + +import org.apache.paimon.data.BinaryVector; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; + +import org.apache.flink.table.data.ArrayData; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link FlinkRowData} and vector type. */ +public class FlinkRowDataWithVectorTest { + + @Test + public void testVectorAsArrayData() { + float[] values = new float[] {1.0f, 2.0f, 3.0f}; + InternalRow row = GenericRow.of(1, BinaryVector.fromPrimitiveArray(values)); + + FlinkRowData rowData = new FlinkRowData(row); + ArrayData arrayData = rowData.getArray(1); + + assertThat(arrayData.toFloatArray()).isEqualTo(values); + } + + @Test + public void testNullVector() { + InternalRow row = GenericRow.of(1, null); + FlinkRowData rowData = new FlinkRowData(row); + + assertThat(rowData.isNullAt(1)).isTrue(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkRowWrapperWithVectorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkRowWrapperWithVectorTest.java new file mode 100644 index 000000000000..234b453a4167 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkRowWrapperWithVectorTest.java @@ -0,0 +1,48 @@ +/* + * 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.paimon.flink; + +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericRowData; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link FlinkRowWrapper} and vector type. */ +public class FlinkRowWrapperWithVectorTest { + + @Test + public void testVectorAccess() { + GenericRowData row = new GenericRowData(2); + row.setField(0, 1); + row.setField(1, new GenericArrayData(new float[] {1.0f, 2.0f, 3.0f})); + + FlinkRowWrapper wrapper = new FlinkRowWrapper(row); + assertThat(wrapper.getVector(1).toFloatArray()).isEqualTo(new float[] {1.0f, 2.0f, 3.0f}); + } + + @Test + public void testNullVector() { + GenericRowData row = new GenericRowData(1); + row.setField(0, null); + + FlinkRowWrapper wrapper = new FlinkRowWrapper(row); + assertThat(wrapper.isNullAt(0)).isTrue(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LogicalTypeConversionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LogicalTypeConversionTest.java new file mode 100644 index 000000000000..142b53c7cb22 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LogicalTypeConversionTest.java @@ -0,0 +1,59 @@ +/* + * 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.paimon.flink; + +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.VectorType; + +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.LogicalType; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link LogicalTypeConversion}. */ +public class LogicalTypeConversionTest { + + @Test + public void testToVectorType() { + VectorType vectorType = LogicalTypeConversion.toVectorType(new FloatType(), "3"); + assertThat(vectorType).isEqualTo(DataTypes.VECTOR(3, DataTypes.FLOAT())); + } + + @Test + public void testToVectorTypeInvalidDim() { + assertThatThrownBy(() -> LogicalTypeConversion.toVectorType(new FloatType(), "")) + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> LogicalTypeConversion.toVectorType(new FloatType(), "abc")) + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> LogicalTypeConversion.toVectorType(new FloatType(), "0")) + .isInstanceOf(IllegalArgumentException.class); + } + + @Test + public void testVectorTypeToLogicalType() { + LogicalType logicalType = + LogicalTypeConversion.toLogicalType(DataTypes.VECTOR(4, DataTypes.FLOAT())); + assertThat(logicalType).isInstanceOf(ArrayType.class); + ArrayType arrayType = (ArrayType) logicalType; + assertThat(arrayType.getElementType()).isInstanceOf(FloatType.class); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/VectorTypeTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/VectorTypeTableITCase.java new file mode 100644 index 000000000000..980c9be8b431 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/VectorTypeTableITCase.java @@ -0,0 +1,121 @@ +/* + * 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.paimon.flink; + +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.utils.ArrayUtils; + +import org.apache.flink.types.Row; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +/** Test write and read table with vector type. */ +public class VectorTypeTableITCase extends CatalogITCaseBase { + + private static final Random RANDOM = new Random(); + + private final String testTblName = "vector_table"; + + private final float[] testVector = randomVector(); + + @Override + protected List ddl() { + return Collections.singletonList(getCreateTableDdl()); + } + + @Test + public void testBasic() throws Exception { + batchSql("SELECT * FROM %s", testTblName); + batchSql("INSERT INTO %s VALUES %s", testTblName, makeValueStr()); + List rows = batchSql("SELECT * FROM %s", testTblName); + + Assertions.assertEquals(1, rows.size()); + Row row = rows.get(0); + Assertions.assertEquals(1, (int) row.getFieldAs("id")); + Assertions.assertEquals("paimon", row.getFieldAs("data")); + Assertions.assertArrayEquals(ArrayUtils.toObject(testVector), row.getFieldAs("embed")); + + checkTableSchema(); + checkInnerData(); + } + + private void checkTableSchema() throws Exception { + DataType vectorType = DataTypes.VECTOR(testVector.length, DataTypes.FLOAT()); + List fields = paimonTable(testTblName).schema().fields(); + Assertions.assertEquals(3, fields.size()); + Assertions.assertEquals(DataTypes.INT(), fields.get(0).type()); + Assertions.assertEquals(DataTypes.STRING(), fields.get(1).type()); + Assertions.assertEquals(vectorType, fields.get(2).type()); + } + + private void checkInnerData() throws Exception { + ReadBuilder builder = paimonTable(testTblName).newReadBuilder(); + RecordReader reader = builder.newRead().createReader(builder.newScan().plan()); + List rows = new ArrayList<>(); + reader.forEachRemaining( + row -> { + rows.add(row); + Assertions.assertTrue(rows.size() < 10); + }); + Assertions.assertEquals(1, rows.size()); + InternalRow row = rows.get(0); + Assertions.assertEquals(1, row.getInt(0)); + Assertions.assertEquals("paimon", row.getString(1).toString()); + Assertions.assertArrayEquals(testVector, row.getVector(2).toFloatArray()); + } + + private String getCreateTableDdl() { + return String.format( + "CREATE TABLE IF NOT EXISTS `%s` (" + + " `id` INT," + + " `data` STRING," + + " `embed` ARRAY NOT NULL" + + ") WITH (" + + " 'file.format' = 'json'," + + " 'file.compression' = 'none'," + + " 'field.embed.vector-dim' = '%d'" + + ")", + testTblName, testVector.length); + } + + private String makeValueStr() { + return String.format("(%d, '%s', ARRAY%s)", 1, "paimon", Arrays.toString(testVector)); + } + + private float[] randomVector() { + byte[] randomBytes = new byte[RANDOM.nextInt(1024) + 1]; + RANDOM.nextBytes(randomBytes); + float[] vector = new float[randomBytes.length]; + for (int i = 0; i < vector.length; i++) { + vector[i] = randomBytes[i]; + } + return vector; + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/utils/InternalRowSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/utils/InternalRowSerializerTest.java index c9a8c3e99777..58f0119d4578 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/utils/InternalRowSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/utils/InternalRowSerializerTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.utils; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.types.DataType; @@ -82,6 +83,30 @@ public void testCopyFromView() { Assertions.assertThat(row.getLong(2)).isEqualTo(row1.getLong(2)); } + @Test + public void testSerializeVector() throws Exception { + RowType vectorRowType = + RowType.builder() + .field("id", DataTypes.INT()) + .field("embed", DataTypes.VECTOR(3, DataTypes.FLOAT())) + .build(); + InternalRowTypeSerializer internalRowTypeSerializer = + new InternalRowTypeSerializer( + vectorRowType.getFieldTypes().toArray(new DataType[0])); + + float[] values = new float[] {1.0f, 2.0f, 3.0f}; + InternalRow row = GenericRow.of(1, BinaryVector.fromPrimitiveArray(values)); + + DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(100); + internalRowTypeSerializer.serialize(row, dataOutputSerializer); + InternalRow row1 = + internalRowTypeSerializer.deserialize( + new DataInputDeserializer(dataOutputSerializer.wrapAsByteBuffer())); + + Assertions.assertThat(row1.getInt(0)).isEqualTo(1); + Assertions.assertThat(row1.getVector(1).toFloatArray()).isEqualTo(values); + } + private BinaryString randomString() { int length = RANDOM.nextInt(100); byte[] buffer = new byte[length]; From 5559fd0c13176dab748f2f1b26015020f8d9bc02 Mon Sep 17 00:00:00 2001 From: ColdL Date: Thu, 29 Jan 2026 10:27:13 +0800 Subject: [PATCH 4/4] add vector-store with data evolution --- .../generated/core_configuration.html | 30 +- .../java/org/apache/paimon/CoreOptions.java | 43 ++ .../org/apache/paimon/format/FileFormat.java | 11 + .../paimon/append/AppendOnlyWriter.java | 39 +- ...va => DataEvolutionRollingFileWriter.java} | 181 ++++++- .../DataEvolutionCompactCoordinator.java | 71 ++- .../DataEvolutionCompactTask.java | 27 +- .../apache/paimon/io/DataFilePathFactory.java | 60 ++- .../operation/BaseAppendFileStoreWrite.java | 3 + .../operation/DataEvolutionFileStoreScan.java | 4 +- .../operation/DataEvolutionSplitRead.java | 171 ++++++- .../commit/RowTrackingCommitUtils.java | 11 + .../paimon/schema/SchemaValidation.java | 44 +- .../apache/paimon/utils/VectorStoreUtils.java | 57 +++ .../paimon/append/AppendOnlyWriterTest.java | 95 +++- .../apache/paimon/append/BlobTableTest.java | 23 +- ...> DataEvolutionRollingFileWriterTest.java} | 38 +- ...nRollingFileWriterWithVectorStoreTest.java | 445 ++++++++++++++++++ .../paimon/append/MultipleBlobTableTest.java | 4 +- .../paimon/append/VectorStoreTableTest.java | 315 +++++++++++++ .../paimon/io/KeyValueFileReadWriteTest.java | 3 + .../operation/DataEvolutionReadTest.java | 57 ++- .../operation/DataEvolutionSplitReadTest.java | 18 + .../paimon/schema/SchemaValidationTest.java | 141 ++++++ .../paimon/tests/FlinkVectorStoreE2eTest.java | 112 +++++ 25 files changed, 1886 insertions(+), 117 deletions(-) rename paimon-core/src/main/java/org/apache/paimon/append/{RollingBlobFileWriter.java => DataEvolutionRollingFileWriter.java} (62%) create mode 100644 paimon-core/src/main/java/org/apache/paimon/utils/VectorStoreUtils.java rename paimon-core/src/test/java/org/apache/paimon/append/{RollingBlobFileWriterTest.java => DataEvolutionRollingFileWriterTest.java} (95%) create mode 100644 paimon-core/src/test/java/org/apache/paimon/append/DataEvolutionRollingFileWriterWithVectorStoreTest.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/append/VectorStoreTableTest.java create mode 100644 paimon-e2e-tests/src/test/java/org/apache/paimon/tests/FlinkVectorStoreE2eTest.java diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 9e4d3f0b330a..c701409742b5 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -152,12 +152,6 @@ String Fields that are ignored for comparison while generating -U, +U changelog for the same record. This configuration is only valid for the changelog-producer.row-deduplicate is true. - -
table-read.sequence-number.enabled
- false - Boolean - Whether to include the _SEQUENCE_NUMBER field when reading the audit_log or binlog system tables. This is only valid for primary key tables. -
changelog.num-retained.max
(none) @@ -1308,6 +1302,12 @@ Duration The delay duration of stream read when scan incremental snapshots. + +
table-read.sequence-number.enabled
+ false + Boolean + Whether to include the _SEQUENCE_NUMBER field when reading the audit_log or binlog system tables. This is only valid for primary key tables. +
tag.automatic-completion
false @@ -1440,6 +1440,24 @@ String The Variant shredding schema for writing. + +
vector-store.fields
+ (none) + String + Specify the vector store fields. + + +
vector-store.format
+ (none) + String + Specify the vector store file format. + + +
vector-store.target-file-size
+ (none) + MemorySize + Target size of a vector-store file. Default is 10 * TARGET_FILE_SIZE. +
write-buffer-for-append
false diff --git a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java index 95f1387491f0..0ee34343e826 100644 --- a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java @@ -2190,6 +2190,29 @@ public InlineElement getDescription() { .withDescription( "Whether to try upgrading the data files after overwriting a primary key table."); + public static final ConfigOption VECTOR_STORE_FORMAT = + key("vector-store.format") + .stringType() + .noDefaultValue() + .withDescription("Specify the vector store file format."); + + public static final ConfigOption VECTOR_STORE_FIELDS = + key("vector-store.fields") + .stringType() + .noDefaultValue() + .withDescription("Specify the vector store fields."); + + public static final ConfigOption VECTOR_STORE_TARGET_FILE_SIZE = + key("vector-store.target-file-size") + .memoryType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Target size of a vector-store file." + + " Default is 10 * TARGET_FILE_SIZE.") + .build()); + private final Options options; public CoreOptions(Map options) { @@ -3407,6 +3430,26 @@ public boolean overwriteUpgrade() { return options.get(OVERWRITE_UPGRADE); } + public String vectorStoreFileFormatString() { + return normalizeFileFormat(options.get(VECTOR_STORE_FORMAT)); + } + + public List vectorStoreFieldNames() { + String vectorStoreFields = options.get(CoreOptions.VECTOR_STORE_FIELDS); + if (vectorStoreFields == null || vectorStoreFields.trim().isEmpty()) { + return new ArrayList<>(); + } else { + return Arrays.asList(vectorStoreFields.split(",")); + } + } + + public long vectorStoreTargetFileSize() { + // Since vectors are large, it would be better to set a larger target size for vectors. + return options.getOptional(VECTOR_STORE_TARGET_FILE_SIZE) + .map(MemorySize::getBytes) + .orElse(10 * targetFileSize(false)); + } + /** Specifies the merge engine for table with primary key. */ public enum MergeEngine implements DescribedEnum { DEDUPLICATE("deduplicate", "De-duplicate and keep the last row."), diff --git a/paimon-common/src/main/java/org/apache/paimon/format/FileFormat.java b/paimon-common/src/main/java/org/apache/paimon/format/FileFormat.java index 90f52099b4f9..c150269d04f6 100644 --- a/paimon-common/src/main/java/org/apache/paimon/format/FileFormat.java +++ b/paimon-common/src/main/java/org/apache/paimon/format/FileFormat.java @@ -107,6 +107,17 @@ public static FileFormat fileFormat(CoreOptions options) { return FileFormat.fromIdentifier(options.fileFormatString(), options.toConfiguration()); } + public static FileFormat vectorStoreFileFormat(CoreOptions options) { + if (options.vectorStoreFieldNames().isEmpty()) { + return null; + } + String vectorStoreFileFormat = options.vectorStoreFileFormatString(); + if (vectorStoreFileFormat == null) { + return fileFormat(options); + } + return FileFormat.fromIdentifier(vectorStoreFileFormat, options.toConfiguration()); + } + public static FileFormat manifestFormat(CoreOptions options) { return FileFormat.fromIdentifier(options.manifestFormatString(), options.toConfiguration()); } diff --git a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java index 7868513c9e89..b8592582dba2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java @@ -52,6 +52,7 @@ import org.apache.paimon.utils.SinkWriter.BufferedSinkWriter; import org.apache.paimon.utils.SinkWriter.DirectSinkWriter; import org.apache.paimon.utils.StatsCollectorFactories; +import org.apache.paimon.utils.VectorStoreUtils; import javax.annotation.Nullable; @@ -73,8 +74,11 @@ public class AppendOnlyWriter implements BatchRecordWriter, MemoryOwner { private final FileIO fileIO; private final long schemaId; private final FileFormat fileFormat; + private final FileFormat vectorStoreFileFormat; + private final List vectorStoreFieldNames; private final long targetFileSize; private final long blobTargetFileSize; + private final long vectorStoreTargetFileSize; private final RowType writeSchema; @Nullable private final List writeCols; private final DataFilePathFactory pathFactory; @@ -105,8 +109,11 @@ public AppendOnlyWriter( @Nullable IOManager ioManager, long schemaId, FileFormat fileFormat, + FileFormat vectorStoreFileFormat, + List vectorStoreFieldNames, long targetFileSize, long blobTargetFileSize, + long vectorStoreTargetFileSize, RowType writeSchema, @Nullable List writeCols, long maxSequenceNumber, @@ -129,8 +136,11 @@ public AppendOnlyWriter( this.fileIO = fileIO; this.schemaId = schemaId; this.fileFormat = fileFormat; + this.vectorStoreFileFormat = vectorStoreFileFormat; + this.vectorStoreFieldNames = vectorStoreFieldNames; this.targetFileSize = targetFileSize; this.blobTargetFileSize = blobTargetFileSize; + this.vectorStoreTargetFileSize = vectorStoreTargetFileSize; this.writeSchema = writeSchema; this.writeCols = writeCols; this.pathFactory = pathFactory; @@ -304,13 +314,25 @@ public void toBufferedWriter() throws Exception { } private RollingFileWriter createRollingRowWriter() { - if (writeSchema.getFieldTypes().stream().anyMatch(t -> t.is(BLOB))) { - return new RollingBlobFileWriter( + boolean hasNormal = + writeSchema.getFields().stream() + .anyMatch( + f -> + !f.type().is(BLOB) + && !vectorStoreFieldNames.contains(f.name())); + boolean hasBlob = writeSchema.getFieldTypes().stream().anyMatch(t -> t.is(BLOB)); + boolean hasSeparatedVectorStore = + VectorStoreUtils.isDifferentFormat(vectorStoreFileFormat, fileFormat); + if (hasBlob || (hasNormal && hasSeparatedVectorStore)) { + return new DataEvolutionRollingFileWriter( fileIO, schemaId, fileFormat, + vectorStoreFileFormat, + vectorStoreFieldNames, targetFileSize, blobTargetFileSize, + vectorStoreTargetFileSize, writeSchema, pathFactory, seqNumCounterProvider, @@ -322,13 +344,20 @@ private RollingFileWriter createRollingRowWriter() { statsDenseStore, blobConsumer); } + FileFormat realFileFormat = hasNormal ? fileFormat : vectorStoreFileFormat; + long realTargetFileSize = hasNormal ? targetFileSize : vectorStoreTargetFileSize; + DataFilePathFactory realPathFactory = + hasNormal + ? pathFactory + : pathFactory.vectorStorePathFactory( + vectorStoreFileFormat.getFormatIdentifier()); return new RowDataRollingFileWriter( fileIO, schemaId, - fileFormat, - targetFileSize, + realFileFormat, + realTargetFileSize, writeSchema, - pathFactory, + realPathFactory, seqNumCounterProvider, fileCompression, statsCollectorFactories.statsCollectors(writeSchema.getFieldNames()), diff --git a/paimon-core/src/main/java/org/apache/paimon/append/RollingBlobFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/append/DataEvolutionRollingFileWriter.java similarity index 62% rename from paimon-core/src/main/java/org/apache/paimon/append/RollingBlobFileWriter.java rename to paimon-core/src/main/java/org/apache/paimon/append/DataEvolutionRollingFileWriter.java index 80faa01d9238..4494794150da 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/RollingBlobFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/DataEvolutionRollingFileWriter.java @@ -28,14 +28,18 @@ import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.io.FileWriterAbortExecutor; import org.apache.paimon.io.RollingFileWriter; +import org.apache.paimon.io.RollingFileWriterImpl; import org.apache.paimon.io.RowDataFileWriter; import org.apache.paimon.io.SingleFileWriter; import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.statistics.SimpleColStatsCollector; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.LongCounter; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.StatsCollectorFactories; +import org.apache.paimon.utils.VectorStoreUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,9 +74,10 @@ * * */ -public class RollingBlobFileWriter implements RollingFileWriter { +public class DataEvolutionRollingFileWriter + implements RollingFileWriter { - private static final Logger LOG = LoggerFactory.getLogger(RollingBlobFileWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(DataEvolutionRollingFileWriter.class); /** Constant for checking rolling condition periodically. */ private static final long CHECK_ROLLING_RECORD_CNT = 1000L; @@ -82,6 +87,10 @@ public class RollingBlobFileWriter implements RollingFileWriter, DataFileMeta>> writerFactory; private final Supplier blobWriterFactory; + private final Supplier< + ProjectedFileWriter< + RollingFileWriterImpl, List>> + vectorStoreWriterFactory; private final long targetFileSize; // State management @@ -91,15 +100,21 @@ public class RollingBlobFileWriter implements RollingFileWriter, DataFileMeta> currentWriter; private MultipleBlobFileWriter blobWriter; + private ProjectedFileWriter< + RollingFileWriterImpl, List> + vectorStoreWriter; private long recordCount = 0; private boolean closed = false; - public RollingBlobFileWriter( + public DataEvolutionRollingFileWriter( FileIO fileIO, long schemaId, FileFormat fileFormat, + FileFormat vectorStoreFileFormat, + List vectorStoreFieldNames, long targetFileSize, long blobTargetFileSize, + long vectorStoreTargetFileSize, RowType writeSchema, DataFilePathFactory pathFactory, Supplier seqNumCounterSupplier, @@ -115,13 +130,25 @@ public RollingBlobFileWriter( this.results = new ArrayList<>(); this.closedWriters = new ArrayList<>(); + // Split into normal and vector-store parts + RowType normalRowType = BlobType.splitBlob(writeSchema).getLeft(); + RowType vectorStoreRowType; + if (VectorStoreUtils.isDifferentFormat(vectorStoreFileFormat, fileFormat)) { + Pair typeWithVectorStore = + VectorStoreUtils.splitVectorStore(normalRowType, vectorStoreFieldNames); + normalRowType = typeWithVectorStore.getLeft(); + vectorStoreRowType = typeWithVectorStore.getRight(); + } else { + vectorStoreRowType = new RowType(Collections.emptyList()); + } + // Initialize writer factory for normal data this.writerFactory = createNormalWriterFactory( fileIO, schemaId, fileFormat, - BlobType.splitBlob(writeSchema).getLeft(), + normalRowType, writeSchema, pathFactory, seqNumCounterSupplier, @@ -133,19 +160,45 @@ public RollingBlobFileWriter( statsDenseStore); // Initialize blob writer - this.blobWriterFactory = - () -> - new MultipleBlobFileWriter( - fileIO, - schemaId, - writeSchema, - pathFactory, - seqNumCounterSupplier, - fileSource, - asyncFileWrite, - statsDenseStore, - blobTargetFileSize, - blobConsumer); + if (!BlobType.splitBlob(writeSchema).getRight().getFields().isEmpty()) { + this.blobWriterFactory = + () -> + new MultipleBlobFileWriter( + fileIO, + schemaId, + writeSchema, + pathFactory, + seqNumCounterSupplier, + fileSource, + asyncFileWrite, + statsDenseStore, + blobTargetFileSize, + blobConsumer); + } else { + this.blobWriterFactory = null; + } + + // Initialize vector-store writer + if (!vectorStoreRowType.getFields().isEmpty()) { + this.vectorStoreWriterFactory = + () -> + createVectorStoreWriter( + fileIO, + vectorStoreFileFormat, + schemaId, + vectorStoreRowType, + writeSchema, + pathFactory, + seqNumCounterSupplier, + fileCompression, + statsCollectorFactories.statsCollectors(vectorStoreFieldNames), + fileSource, + asyncFileWrite, + statsDenseStore, + vectorStoreTargetFileSize); + } else { + this.vectorStoreWriterFactory = null; + } } /** Creates a factory for normal data writers. */ @@ -192,6 +245,53 @@ public RollingBlobFileWriter( }; } + /** Creates a vector-store writer for handling vector-store data. */ + private static ProjectedFileWriter< + RollingFileWriterImpl, List> + createVectorStoreWriter( + FileIO fileIO, + FileFormat vectorStoreFileFormat, + long schemaId, + RowType vectorStoreRowType, + RowType writeSchema, + DataFilePathFactory pathFactory, + Supplier seqNumCounterSupplier, + String fileCompression, + SimpleColStatsCollector.Factory[] statsCollectors, + FileSource fileSource, + boolean asyncFileWrite, + boolean statsDenseStore, + long targetFileSize) { + + List vectorStoreFieldNames = vectorStoreRowType.getFieldNames(); + + int[] vectorStoreProjection = writeSchema.projectIndexes(vectorStoreFieldNames); + DataFilePathFactory vectorStorePathFactory = + pathFactory.vectorStorePathFactory(vectorStoreFileFormat.getFormatIdentifier()); + return new ProjectedFileWriter<>( + new RollingFileWriterImpl<>( + () -> + new RowDataFileWriter( + fileIO, + RollingFileWriter.createFileWriterContext( + vectorStoreFileFormat, + vectorStoreRowType, + statsCollectors, + fileCompression), + vectorStorePathFactory.newPath(), + vectorStoreRowType, + schemaId, + seqNumCounterSupplier, + new FileIndexOptions(), + fileSource, + asyncFileWrite, + statsDenseStore, + pathFactory.isExternalPath(), + vectorStoreFieldNames), + targetFileSize), + vectorStoreProjection); + } + /** * Writes a single row to both normal and blob writers. Automatically handles file rolling when * target size is reached. @@ -205,11 +305,19 @@ public void write(InternalRow row) throws IOException { if (currentWriter == null) { currentWriter = writerFactory.get(); } - if (blobWriter == null) { + if ((blobWriter == null) && (blobWriterFactory != null)) { blobWriter = blobWriterFactory.get(); } + if ((vectorStoreWriter == null) && (vectorStoreWriterFactory != null)) { + vectorStoreWriter = vectorStoreWriterFactory.get(); + } currentWriter.write(row); - blobWriter.write(row); + if (blobWriter != null) { + blobWriter.write(row); + } + if (vectorStoreWriter != null) { + vectorStoreWriter.write(row); + } recordCount++; if (rollingFile()) { @@ -269,6 +377,10 @@ public void abort() { blobWriter.abort(); blobWriter = null; } + if (vectorStoreWriter != null) { + vectorStoreWriter.abort(); + vectorStoreWriter = null; + } } /** Checks if the current file should be rolled based on size and record count. */ @@ -295,12 +407,16 @@ private void closeCurrentWriter() throws IOException { // Close blob writer and process blob metadata List blobMetas = closeBlobWriter(); + // Close vector-store writer and process vector-store metadata + List vectorStoreMetas = closeVectorStoreWriter(); + // Validate consistency between main and blob files - validateFileConsistency(mainDataFileMeta, blobMetas); + validateFileConsistency(mainDataFileMeta, blobMetas, vectorStoreMetas); // Add results to the results list results.add(mainDataFileMeta); results.addAll(blobMetas); + results.addAll(vectorStoreMetas); // Reset current writer currentWriter = null; @@ -324,9 +440,22 @@ private List closeBlobWriter() throws IOException { return results; } + /** Closes the vector-store writer and processes blob metadata with appropriate tags. */ + private List closeVectorStoreWriter() throws IOException { + if (vectorStoreWriter == null) { + return Collections.emptyList(); + } + vectorStoreWriter.close(); + List results = vectorStoreWriter.result(); + vectorStoreWriter = null; + return results; + } + /** Validates that the row counts match between main and blob files. */ private void validateFileConsistency( - DataFileMeta mainDataFileMeta, List blobTaggedMetas) { + DataFileMeta mainDataFileMeta, + List blobTaggedMetas, + List vectorStoreMetas) { long mainRowCount = mainDataFileMeta.rowCount(); Map blobRowCounts = new HashMap<>(); @@ -334,6 +463,9 @@ private void validateFileConsistency( long count = file.rowCount(); blobRowCounts.compute(file.writeCols().get(0), (k, v) -> v == null ? count : v + count); } + long vectorStoreRowCount = + vectorStoreMetas.stream().mapToLong(DataFileMeta::rowCount).sum(); + for (String blobFieldName : blobRowCounts.keySet()) { long blobRowCount = blobRowCounts.get(blobFieldName); if (mainRowCount != blobRowCount) { @@ -344,6 +476,13 @@ private void validateFileConsistency( mainDataFileMeta, mainRowCount, blobFieldName, blobRowCount)); } } + if (!vectorStoreMetas.isEmpty() && (mainRowCount != vectorStoreRowCount)) { + throw new IllegalStateException( + String.format( + "This is a bug: The row count of main file and vector-store files does not match. " + + "Main file: %s (row count: %d), vector-store files: %s (total row count: %d)", + mainDataFileMeta, mainRowCount, vectorStoreMetas, vectorStoreRowCount)); + } } /** diff --git a/paimon-core/src/main/java/org/apache/paimon/append/dataevolution/DataEvolutionCompactCoordinator.java b/paimon-core/src/main/java/org/apache/paimon/append/dataevolution/DataEvolutionCompactCoordinator.java index e8081ac069f1..47eb04e32f8f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/dataevolution/DataEvolutionCompactCoordinator.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/dataevolution/DataEvolutionCompactCoordinator.java @@ -46,6 +46,7 @@ import static org.apache.paimon.format.blob.BlobFileFormat.isBlobFile; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.VectorStoreUtils.isVectorStoreFile; /** Compact coordinator to compact data evolution table. */ public class DataEvolutionCompactCoordinator { @@ -170,14 +171,19 @@ List compactPlan(List input) { for (List group : ranges) { List dataFiles = new ArrayList<>(); List blobFiles = new ArrayList<>(); + List vectorStoreFiles = new ArrayList<>(); TreeMap treeMap = new TreeMap<>(); Map> dataFileToBlobFiles = new HashMap<>(); + Map> dataFileToVectorStoreFiles = + new HashMap<>(); for (DataFileMeta f : group) { - if (!isBlobFile(f.fileName())) { + if (isBlobFile(f.fileName())) { + blobFiles.add(f); + } else if (isVectorStoreFile(f.fileName())) { + vectorStoreFiles.add(f); + } else { treeMap.put(f.nonNullFirstRowId(), f); dataFiles.add(f); - } else { - blobFiles.add(f); } } @@ -199,6 +205,25 @@ List compactPlan(List input) { } } } + if (false) { + // associate vector-store files to data files + for (DataFileMeta vectorStoreFile : vectorStoreFiles) { + Long key = treeMap.floorKey(vectorStoreFile.nonNullFirstRowId()); + if (key != null) { + DataFileMeta dataFile = treeMap.get(key); + if (vectorStoreFile.nonNullFirstRowId() + >= dataFile.nonNullFirstRowId() + && vectorStoreFile.nonNullFirstRowId() + <= dataFile.nonNullFirstRowId() + + dataFile.rowCount() + - 1) { + dataFileToVectorStoreFiles + .computeIfAbsent(dataFile, k -> new ArrayList<>()) + .add(vectorStoreFile); + } + } + } + } RangeHelper rangeHelper2 = new RangeHelper<>( @@ -221,10 +246,19 @@ List compactPlan(List input) { .sum(); if (currentGroupWeight > targetFileSize) { // compact current file group to merge field files - tasks.addAll(triggerTask(fileGroup, partition, dataFileToBlobFiles)); + tasks.addAll( + triggerTask( + fileGroup, + partition, + dataFileToBlobFiles, + dataFileToVectorStoreFiles)); // compact wait compact files tasks.addAll( - triggerTask(waitCompactFiles, partition, dataFileToBlobFiles)); + triggerTask( + waitCompactFiles, + partition, + dataFileToBlobFiles, + dataFileToVectorStoreFiles)); waitCompactFiles = new ArrayList<>(); weightSum = 0; } else { @@ -233,13 +267,21 @@ List compactPlan(List input) { if (weightSum > targetFileSize) { tasks.addAll( triggerTask( - waitCompactFiles, partition, dataFileToBlobFiles)); + waitCompactFiles, + partition, + dataFileToBlobFiles, + dataFileToVectorStoreFiles)); waitCompactFiles = new ArrayList<>(); weightSum = 0L; } } } - tasks.addAll(triggerTask(waitCompactFiles, partition, dataFileToBlobFiles)); + tasks.addAll( + triggerTask( + waitCompactFiles, + partition, + dataFileToBlobFiles, + dataFileToVectorStoreFiles)); } } return tasks; @@ -248,7 +290,8 @@ List compactPlan(List input) { private List triggerTask( List dataFiles, BinaryRow partition, - Map> dataFileToBlobFiles) { + Map> dataFileToBlobFiles, + Map> dataFileToVectorStoreFiles) { List tasks = new ArrayList<>(); if (dataFiles.size() >= compactMinFileNum) { tasks.add(new DataEvolutionCompactTask(partition, dataFiles, false)); @@ -264,6 +307,18 @@ private List triggerTask( tasks.add(new DataEvolutionCompactTask(partition, blobFiles, true)); } } + + if (false) { + List vectorStoreFiles = new ArrayList<>(); + for (DataFileMeta dataFile : dataFiles) { + vectorStoreFiles.addAll( + dataFileToVectorStoreFiles.getOrDefault( + dataFile, Collections.emptyList())); + } + if (vectorStoreFiles.size() >= compactMinFileNum) { + tasks.add(new DataEvolutionCompactTask(partition, vectorStoreFiles, false)); + } + } return tasks; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/append/dataevolution/DataEvolutionCompactTask.java b/paimon-core/src/main/java/org/apache/paimon/append/dataevolution/DataEvolutionCompactTask.java index f1f48ac56da8..29d4cf427d85 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/dataevolution/DataEvolutionCompactTask.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/dataevolution/DataEvolutionCompactTask.java @@ -22,6 +22,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.format.FileFormat; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; @@ -34,7 +35,9 @@ import org.apache.paimon.types.DataTypeRoot; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.FileStorePathFactory; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.RecordWriter; +import org.apache.paimon.utils.VectorStoreUtils; import java.util.ArrayList; import java.util.Collections; @@ -89,11 +92,33 @@ public CommitMessage doCompact(FileStoreTable table, String commitUser) throws E table = table.copy(DYNAMIC_WRITE_OPTIONS); long firstRowId = compactBefore.get(0).nonNullFirstRowId(); - RowType readWriteType = + List vectorStoreFieldNames = table.coreOptions().vectorStoreFieldNames(); + boolean isVectorStoreDifferentFormat = + VectorStoreUtils.isDifferentFormat( + FileFormat.vectorStoreFileFormat(table.coreOptions()), + FileFormat.fileFormat(table.coreOptions())); + boolean isVectorStoreFiles = + VectorStoreUtils.isVectorStoreFile(compactBefore.get(0).fileName()); + + if (isVectorStoreFiles) { + // TODO: support vector-store file compaction + throw new UnsupportedOperationException("Vector-store task is not supported"); + } + + RowType nonBlobRowType = new RowType( table.rowType().getFields().stream() .filter(f -> f.type().getTypeRoot() != DataTypeRoot.BLOB) .collect(Collectors.toList())); + RowType readWriteType; + if (isVectorStoreDifferentFormat) { + Pair pair = + VectorStoreUtils.splitVectorStore(nonBlobRowType, vectorStoreFieldNames); + readWriteType = isVectorStoreFiles ? pair.getRight() : pair.getLeft(); + } else { + readWriteType = nonBlobRowType; + } + FileStorePathFactory pathFactory = table.store().pathFactory(); AppendOnlyFileStore store = (AppendOnlyFileStore) table.store(); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java index b63a1c0b7a79..6318d70b7026 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java @@ -59,14 +59,36 @@ public DataFilePathFactory( boolean fileSuffixIncludeCompression, String fileCompression, @Nullable ExternalPathProvider externalPathProvider) { + this( + parent, + UUID.randomUUID().toString(), + new AtomicInteger(0), + formatIdentifier, + dataFilePrefix, + changelogFilePrefix, + fileSuffixIncludeCompression, + compressFileExtension(fileCompression), + externalPathProvider); + } + + private DataFilePathFactory( + Path parent, + String uuid, + AtomicInteger pathCount, + String formatIdentifier, + String dataFilePrefix, + String changelogFilePrefix, + boolean fileSuffixIncludeCompression, + @Nullable String compressExtension, + @Nullable ExternalPathProvider externalPathProvider) { this.parent = parent; - this.uuid = UUID.randomUUID().toString(); - this.pathCount = new AtomicInteger(0); + this.uuid = uuid; + this.pathCount = pathCount; this.formatIdentifier = formatIdentifier; this.dataFilePrefix = dataFilePrefix; this.changelogFilePrefix = changelogFilePrefix; this.fileSuffixIncludeCompression = fileSuffixIncludeCompression; - this.compressExtension = compressFileExtension(fileCompression); + this.compressExtension = compressExtension; this.externalPathProvider = externalPathProvider; } @@ -99,6 +121,10 @@ public Path newPath(String prefix) { } private String newFileName(String prefix) { + return newFileName(prefix, makeExtension(compressExtension, formatIdentifier)); + } + + protected String makeExtension(String compressExtension, String formatIdentifier) { String extension; if (compressExtension != null && isTextFormat(formatIdentifier)) { extension = "." + formatIdentifier + "." + compressExtension; @@ -107,7 +133,7 @@ private String newFileName(String prefix) { } else { extension = "." + formatIdentifier; } - return newFileName(prefix, extension); + return extension; } public Path newPathFromExtension(String extension) { @@ -121,7 +147,7 @@ public Path newPathFromName(String fileName) { return new Path(parent, fileName); } - private String newFileName(String prefix, String extension) { + protected String newFileName(String prefix, String extension) { return prefix + uuid + "-" + pathCount.getAndIncrement() + extension; } @@ -211,4 +237,28 @@ private static String compressFileExtension(String compression) { } return compression; } + + public DataFilePathFactory vectorStorePathFactory(String formatIdentifier) { + return new VectorStoreWrapper(this, formatIdentifier); + } + + private static class VectorStoreWrapper extends DataFilePathFactory { + private VectorStoreWrapper(DataFilePathFactory base, String formatIdentifier) { + super( + base.parent, + base.uuid, + base.pathCount, + formatIdentifier, + base.dataFilePrefix, + base.changelogFilePrefix, + base.fileSuffixIncludeCompression, + base.compressExtension, + base.externalPathProvider); + } + + @Override + protected String makeExtension(String compressExtension, String formatIdentifier) { + return ".vector-store" + super.makeExtension(compressExtension, formatIdentifier); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/BaseAppendFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/BaseAppendFileStoreWrite.java index bc39a4ae8e76..aabaa11a3e50 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/BaseAppendFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/BaseAppendFileStoreWrite.java @@ -132,8 +132,11 @@ protected RecordWriter createWriter( ioManager, schemaId, fileFormat, + FileFormat.vectorStoreFileFormat(options), + options.vectorStoreFieldNames(), options.targetFileSize(false), options.blobTargetFileSize(), + options.vectorStoreTargetFileSize(), writeType, writeCols, restoredMaxSeqNumber, diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java index ef1ea82ce94b..2a2e1b785424 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java @@ -53,6 +53,7 @@ import static org.apache.paimon.format.blob.BlobFileFormat.isBlobFile; import static org.apache.paimon.utils.Preconditions.checkNotNull; +import static org.apache.paimon.utils.VectorStoreUtils.isVectorStoreFile; /** {@link FileStoreScan} for data-evolution enabled table. */ public class DataEvolutionFileStoreScan extends AppendOnlyFileStoreScan { @@ -158,10 +159,11 @@ static EvolutionStats evolutionStats( TableSchema schema, Function scanTableSchema, List metas) { - // exclude blob files, useless for predicate eval + // exclude blob and vector-store files, useless for predicate eval metas = metas.stream() .filter(entry -> !isBlobFile(entry.file().fileName())) + .filter(entry -> !isVectorStoreFile(entry.file().fileName())) .collect(Collectors.toList()); ToLongFunction maxSeqFunc = e -> e.file().maxSequenceNumber(); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java index f501c41b4882..36491cebfa4f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java @@ -62,6 +62,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; import java.util.function.Function; import java.util.function.ToLongFunction; import java.util.stream.Collectors; @@ -72,6 +74,8 @@ import static org.apache.paimon.format.blob.BlobFileFormat.isBlobFile; import static org.apache.paimon.table.SpecialFields.rowTypeWithRowTracking; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkNotNull; +import static org.apache.paimon.utils.VectorStoreUtils.isVectorStoreFile; /** * A union {@link SplitRead} to read multiple inner files to merge columns, note that this class @@ -216,13 +220,10 @@ private DataEvolutionFileReader createUnionReader( needMergeFiles, file -> { checkArgument( - isBlobFile(file.fileName()), - "Only blob file need to call this method."); - return schemaFetcher - .apply(file.schemaId()) - .logicalRowType() - .getField(file.writeCols().get(0)) - .id(); + isBlobFile(file.fileName()) + || isVectorStoreFile(file.fileName()), + "Only blob/vector-store files need to call this method."); + return schemaFetcher.apply(file.schemaId()).logicalRowType(); }); long rowCount = fieldsFiles.get(0).rowCount(); @@ -409,25 +410,39 @@ private FileRecordReader createFileReader( @VisibleForTesting public static List splitFieldBunches( - List needMergeFiles, Function blobFileToFieldId) { - return splitFieldBunches(needMergeFiles, blobFileToFieldId, false); + List needMergeFiles, Function fileToRowType) { + return splitFieldBunches(needMergeFiles, fileToRowType, false); } @VisibleForTesting public static List splitFieldBunches( List needMergeFiles, - Function blobFileToFieldId, + Function fileToRowType, boolean rowIdPushDown) { List fieldsFiles = new ArrayList<>(); - Map blobBunchMap = new HashMap<>(); + Map blobBunchMap = new HashMap<>(); + Map vectorStoreBunchMap = new TreeMap<>(); long rowCount = -1; for (DataFileMeta file : needMergeFiles) { if (isBlobFile(file.fileName())) { - int fieldId = blobFileToFieldId.apply(file); + RowType rowType = fileToRowType.apply(file); + int fieldId = rowType.getField(file.writeCols().get(0)).id(); final long expectedRowCount = rowCount; blobBunchMap .computeIfAbsent( - fieldId, key -> new BlobBunch(expectedRowCount, rowIdPushDown)) + fieldId, key -> new SplitBunch(expectedRowCount, rowIdPushDown)) + .add(file); + } else if (isVectorStoreFile(file.fileName())) { + RowType rowType = fileToRowType.apply(file); + String fileFormat = DataFilePathFactory.formatIdentifier(file.fileName()); + VectorStoreBunchKey vectorStoreKey = + new VectorStoreBunchKey( + file.schemaId(), fileFormat, file.writeCols(), rowType); + final long expectedRowCount = rowCount; + vectorStoreBunchMap + .computeIfAbsent( + vectorStoreKey, + key -> new SplitBunch(expectedRowCount, rowIdPushDown)) .add(file); } else { // Normal file, just add it to the current merge split @@ -436,6 +451,7 @@ public static List splitFieldBunches( } } fieldsFiles.addAll(blobBunchMap.values()); + fieldsFiles.addAll(vectorStoreBunchMap.values()); return fieldsFiles; } @@ -467,7 +483,7 @@ public List files() { } @VisibleForTesting - static class BlobBunch implements FieldBunch { + static class SplitBunch implements FieldBunch { final List files; final long expectedRowCount; @@ -478,7 +494,7 @@ static class BlobBunch implements FieldBunch { long latestMaxSequenceNumber = -1; long rowCount; - BlobBunch(long expectedRowCount, boolean rowIdPushDown) { + SplitBunch(long expectedRowCount, boolean rowIdPushDown) { this.files = new ArrayList<>(); this.rowCount = 0; this.expectedRowCount = expectedRowCount; @@ -486,14 +502,15 @@ static class BlobBunch implements FieldBunch { } void add(DataFileMeta file) { - if (!isBlobFile(file.fileName())) { - throw new IllegalArgumentException("Only blob file can be added to a blob bunch."); + if (!isBlobFile(file.fileName()) && !isVectorStoreFile(file.fileName())) { + throw new IllegalArgumentException( + "Only blob/vector-store file can be added to this bunch."); } if (file.nonNullFirstRowId() == latestFistRowId) { if (file.maxSequenceNumber() >= latestMaxSequenceNumber) { throw new IllegalArgumentException( - "Blob file with same first row id should have decreasing sequence number."); + "Blob/vector-store file with same first row id should have decreasing sequence number."); } return; } @@ -512,11 +529,11 @@ void add(DataFileMeta file) { if (firstRowId < expectedNextFirstRowId) { checkArgument( file.maxSequenceNumber() < latestMaxSequenceNumber, - "Blob file with overlapping row id should have decreasing sequence number."); + "Blob/vector-store file with overlapping row id should have decreasing sequence number."); return; } else if (firstRowId > expectedNextFirstRowId) { throw new IllegalArgumentException( - "Blob file first row id should be continuous, expect " + "Blob/vector-store file first row id should be continuous, expect " + expectedNextFirstRowId + " but got " + firstRowId); @@ -525,17 +542,17 @@ void add(DataFileMeta file) { if (!files.isEmpty()) { checkArgument( file.schemaId() == files.get(0).schemaId(), - "All files in a blob bunch should have the same schema id."); + "All files in this bunch should have the same schema id."); checkArgument( file.writeCols().equals(files.get(0).writeCols()), - "All files in a blob bunch should have the same write columns."); + "All files in this bunch should have the same write columns."); } } files.add(file); rowCount += file.rowCount(); checkArgument( rowCount <= expectedRowCount, - "Blob files row count exceed the expect " + expectedRowCount); + "Blob/vector-store files row count exceed the expect " + expectedRowCount); this.latestMaxSequenceNumber = file.maxSequenceNumber(); this.latestFistRowId = file.nonNullFirstRowId(); this.expectedNextFirstRowId = latestFistRowId + file.rowCount(); @@ -560,14 +577,17 @@ public static List> mergeRangesAndSort(List fil RangeHelper rangeHelper = new RangeHelper<>(firstRowIdFunc, endRowIdF); List> result = rangeHelper.mergeOverlappingRanges(files); - // in group, sort by blob file and max_seq + // in group, sort by blob/vector-store file and max_seq for (List group : result) { - // split to data files and blob files + // split to data files, blob files, vector-store files List dataFiles = new ArrayList<>(); List blobFiles = new ArrayList<>(); + List vectorStoreFiles = new ArrayList<>(); for (DataFileMeta f : group) { if (isBlobFile(f.fileName())) { blobFiles.add(f); + } else if (isVectorStoreFile(f.fileName())) { + vectorStoreFiles.add(f); } else { dataFiles.add(f); } @@ -585,12 +605,111 @@ public static List> mergeRangesAndSort(List fil comparingLong(firstRowIdFunc) .thenComparing(reverseOrder(comparingLong(maxSeqF)))); - // concat data files and blob files + // vector-store files sort by first row id then by reversed max sequence number + vectorStoreFiles.sort( + comparingLong(firstRowIdFunc) + .thenComparing(reverseOrder(comparingLong(maxSeqF)))); + + // concat data files, blob files, vector-store files group.clear(); group.addAll(dataFiles); group.addAll(blobFiles); + group.addAll(vectorStoreFiles); } return result; } + + static final class VectorStoreBunchKey implements Comparable { + public final long schemaId; + public final String formatIdentifier; + public final List writeCols; + + public VectorStoreBunchKey( + long schemaId, + String formatIdentifier, + List writeCols, + RowType preferredColOrder) { + this.schemaId = schemaId; + this.formatIdentifier = checkNotNull(formatIdentifier, "formatIdentifier"); + this.writeCols = normalizeWriteCols(writeCols, preferredColOrder); + } + + @Override + public int compareTo(VectorStoreBunchKey o) { + int c = Long.compare(this.schemaId, o.schemaId); + if (c != 0) { + return c; + } + + c = this.formatIdentifier.compareTo(o.formatIdentifier); + if (c != 0) { + return c; + } + + int n = Math.min(this.writeCols.size(), o.writeCols.size()); + for (int i = 0; i < n; i++) { + c = this.writeCols.get(i).compareTo(o.writeCols.get(i)); + if (c != 0) { + return c; + } + } + return Integer.compare(this.writeCols.size(), o.writeCols.size()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof VectorStoreBunchKey)) { + return false; + } + VectorStoreBunchKey that = (VectorStoreBunchKey) o; + return schemaId == that.schemaId + && formatIdentifier.equals(that.formatIdentifier) + && writeCols.equals(that.writeCols); + } + + @Override + public int hashCode() { + return Objects.hash(schemaId, formatIdentifier, writeCols); + } + + @Override + public String toString() { + return "VectorStoreBunchKey{schemaId=" + + schemaId + + ", format=" + + formatIdentifier + + ", writeCols=" + + writeCols + + "}"; + } + + private static List normalizeWriteCols(List writeCols, RowType rowType) { + if (writeCols == null || writeCols.isEmpty()) { + return Collections.emptyList(); + } + + Map colPosMap = new HashMap<>(); + List namesInRowType = rowType.getFieldNames(); + for (int i = 0; i < namesInRowType.size(); i++) { + colPosMap.putIfAbsent(namesInRowType.get(i), i); + } + + ArrayList sorted = new ArrayList<>(writeCols); + sorted.sort( + (a, b) -> { + int ia = colPosMap.getOrDefault(a, Integer.MAX_VALUE); + int ib = colPosMap.getOrDefault(b, Integer.MAX_VALUE); + if (ia != ib) { + return Integer.compare(ia, ib); + } + return a.compareTo(b); + }); + + return sorted; + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/commit/RowTrackingCommitUtils.java b/paimon-core/src/main/java/org/apache/paimon/operation/commit/RowTrackingCommitUtils.java index d2f3dc8851ef..fe66f00fd1e2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/commit/RowTrackingCommitUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/commit/RowTrackingCommitUtils.java @@ -30,6 +30,7 @@ import static org.apache.paimon.format.blob.BlobFileFormat.isBlobFile; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.VectorStoreUtils.isVectorStoreFile; /** Utils for row tracking commit. */ public class RowTrackingCommitUtils { @@ -68,6 +69,7 @@ private static long assignRowTrackingMeta( long start = firstRowIdStart; long blobStartDefault = firstRowIdStart; Map blobStarts = new HashMap<>(); + long vectorStoreStart = firstRowIdStart; for (ManifestEntry entry : deltaFiles) { Optional fileSource = entry.file().fileSource(); checkArgument( @@ -91,6 +93,15 @@ private static long assignRowTrackingMeta( } rowIdAssigned.add(entry.assignFirstRowId(blobStart)); blobStarts.put(blobFieldName, blobStart + rowCount); + } else if (isVectorStoreFile(entry.file().fileName())) { + if (vectorStoreStart >= start) { + throw new IllegalStateException( + String.format( + "This is a bug, vectorStoreStart %d should be less than start %d when assigning a vector-store entry file.", + vectorStoreStart, start)); + } + rowIdAssigned.add(entry.assignFirstRowId(vectorStoreStart)); + vectorStoreStart += rowCount; } else { rowIdAssigned.add(entry.assignFirstRowId(start)); blobStartDefault = start; diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java index fd07bcef17fe..a62b45702481 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java @@ -43,10 +43,12 @@ import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.StringUtils; +import org.apache.paimon.utils.VectorStoreUtils; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -156,7 +158,17 @@ public static void validateTableSchema(TableSchema schema) { FileFormat fileFormat = FileFormat.fromIdentifier(options.formatType(), new Options(schema.options())); - fileFormat.validateDataFields(BlobType.splitBlob(new RowType(schema.fields())).getLeft()); + if (VectorStoreUtils.isDifferentFormat( + FileFormat.vectorStoreFileFormat(options), fileFormat)) { + fileFormat.validateDataFields( + VectorStoreUtils.splitVectorStore( + BlobType.splitBlob(new RowType(schema.fields())).getLeft(), + options.vectorStoreFieldNames()) + .getLeft()); + } else { + fileFormat.validateDataFields( + BlobType.splitBlob(new RowType(schema.fields())).getLeft()); + } // Check column names in schema schema.fieldNames() @@ -617,6 +629,36 @@ private static void validateRowTracking(TableSchema schema, CoreOptions options) !schema.partitionKeys().contains(blobNames.get(0)), "The BLOB type column can not be part of partition keys."); } + + FileFormat vectorStoreFileFormat = FileFormat.vectorStoreFileFormat(options); + if (VectorStoreUtils.isDifferentFormat( + vectorStoreFileFormat, FileFormat.fileFormat(options))) { + List vectorStoreNames = options.vectorStoreFieldNames(); + List nonBlobNames = + BlobType.splitBlob(schema.logicalRowType()).getLeft().getFieldNames(); + checkArgument( + blobNames.stream().noneMatch(vectorStoreNames::contains), + "The vector-store columns can not be blob type."); + checkArgument( + new HashSet<>(nonBlobNames).containsAll(vectorStoreNames), + "Some of the columns specified as vector-store are unknown."); + checkArgument( + schema.partitionKeys().stream().noneMatch(vectorStoreNames::contains), + "The vector-store columns can not be part of partition keys."); + checkArgument( + nonBlobNames.size() > vectorStoreNames.size(), + "Table with vector-store must have other normal columns."); + checkArgument( + options.dataEvolutionEnabled(), + "Data evolution config must enabled for table with vector-store file format."); + + RowType vectorStoreRowType = + VectorStoreUtils.splitVectorStore( + BlobType.splitBlob(schema.logicalRowType()).getLeft(), + vectorStoreNames) + .getRight(); + vectorStoreFileFormat.validateDataFields(vectorStoreRowType); + } } private static void validateIncrementalClustering(TableSchema schema, CoreOptions options) { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/VectorStoreUtils.java b/paimon-core/src/main/java/org/apache/paimon/utils/VectorStoreUtils.java new file mode 100644 index 000000000000..7e8c474a776e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/utils/VectorStoreUtils.java @@ -0,0 +1,57 @@ +/* + * 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.paimon.utils; + +import org.apache.paimon.format.FileFormat; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; + +import java.util.ArrayList; +import java.util.List; + +/** Utils for vector-store table. */ +public class VectorStoreUtils { + public static boolean isDifferentFormat(FileFormat vectorStoreFormat, FileFormat normalFormat) { + return (vectorStoreFormat != null) + && !vectorStoreFormat + .getFormatIdentifier() + .equals(normalFormat.getFormatIdentifier()); + } + + public static boolean isVectorStoreFile(String fileName) { + return fileName.contains(".vector-store."); + } + + public static Pair splitVectorStore( + RowType rowType, List vectorStoreFieldNames) { + List allFields = rowType.getFields(); + List normalFields = new ArrayList<>(); + List vectorStoreFields = new ArrayList<>(); + + for (DataField field : allFields) { + if (vectorStoreFieldNames.contains(field.name())) { + vectorStoreFields.add(field); + } else { + normalFields.add(field); + } + } + + return Pair.of(new RowType(normalFields), new RowType(vectorStoreFields)); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java index ff5891d37c81..d7bdf7b7812f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java @@ -73,6 +73,7 @@ import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; @@ -575,6 +576,34 @@ public void testNonSpillable() throws Exception { writer.close(); } + @Test + public void testVectorStoreSameFormatUsesRowDataWriter() throws Exception { + FileFormat format = FileFormat.fromIdentifier(AVRO, new Options()); + AppendOnlyWriter writer = + createVectorStoreWriter(1024 * 1024L, format, Collections.singletonList("name")); + writer.write(row(1, "AAA", PART)); + CommitIncrement increment = writer.prepareCommit(true); + writer.close(); + + assertThat(increment.newFilesIncrement().newFiles()).hasSize(1); + DataFileMeta meta = increment.newFilesIncrement().newFiles().get(0); + assertThat(meta.fileName()).doesNotContain(".vector-store"); + } + + @Test + public void testAllVectorStoreColumnsUseVectorStorePath() throws Exception { + FileFormat format = FileFormat.fromIdentifier(AVRO, new Options()); + AppendOnlyWriter writer = + createVectorStoreWriter(1024 * 1024L, format, Arrays.asList("id", "name", "dt")); + writer.write(row(1, "AAA", PART)); + CommitIncrement increment = writer.prepareCommit(true); + writer.close(); + + assertThat(increment.newFilesIncrement().newFiles()).hasSize(1); + DataFileMeta meta = increment.newFilesIncrement().newFiles().get(0); + assertThat(meta.fileName()).contains(".vector-store"); + } + private SimpleColStats initStats(Integer min, Integer max, long nullCount) { return new SimpleColStats(min, max, nullCount); } @@ -665,6 +694,58 @@ private Pair> createWriter( boolean hasIoManager, List scannedFiles, CountDownLatch latch) { + Map options = new HashMap<>(); + options.put("metadata.stats-mode", "truncate(16)"); + return createWriterBase( + targetFileSize, + null, + Collections.emptyList(), + forceCompact, + useWriteBuffer, + spillable, + hasIoManager, + scannedFiles, + compactBefore -> { + latch.await(); + return compactBefore.isEmpty() + ? Collections.emptyList() + : Collections.singletonList(generateCompactAfter(compactBefore)); + }, + options); + } + + private AppendOnlyWriter createVectorStoreWriter( + long targetFileSize, + FileFormat vectorStoreFileFormat, + List vectorStoreFieldNames) { + Map options = new HashMap<>(); + options.put("metadata.stats-mode", "truncate(16)"); + options.put(CoreOptions.DATA_EVOLUTION_ENABLED.key(), "true"); + return createWriterBase( + targetFileSize, + vectorStoreFileFormat, + vectorStoreFieldNames, + false, + true, + false, + true, + Collections.emptyList(), + compactBefore -> Collections.emptyList(), + options) + .getKey(); + } + + private Pair> createWriterBase( + long targetFileSize, + FileFormat vectorStoreFileFormat, + List vectorStoreFieldNames, + boolean forceCompact, + boolean useWriteBuffer, + boolean spillable, + boolean hasIoManager, + List scannedFiles, + BucketedAppendCompactManager.CompactRewriter rewriter, + Map optionsMap) { FileFormat fileFormat = FileFormat.fromIdentifier(AVRO, new Options()); LinkedList toCompact = new LinkedList<>(scannedFiles); BucketedAppendCompactManager compactManager = @@ -676,22 +757,18 @@ private Pair> createWriter( MIN_FILE_NUM, targetFileSize, false, - compactBefore -> { - latch.await(); - return compactBefore.isEmpty() - ? Collections.emptyList() - : Collections.singletonList( - generateCompactAfter(compactBefore)); - }, + rewriter, null); - CoreOptions options = - new CoreOptions(Collections.singletonMap("metadata.stats-mode", "truncate(16)")); + CoreOptions options = new CoreOptions(optionsMap); AppendOnlyWriter writer = new AppendOnlyWriter( LocalFileIO.create(), hasIoManager ? IOManager.create(tempDir.toString()) : null, SCHEMA_ID, fileFormat, + vectorStoreFileFormat, + vectorStoreFieldNames, + targetFileSize, targetFileSize, targetFileSize, AppendOnlyWriterTest.SCHEMA, diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java index 3045f33b1664..6b7d94679863 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java @@ -38,7 +38,9 @@ import org.apache.paimon.table.TableTestBase; import org.apache.paimon.table.sink.BatchTableWrite; import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Range; import org.apache.paimon.utils.UriReader; @@ -52,6 +54,7 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; import java.util.stream.Collectors; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; @@ -105,7 +108,8 @@ public void testBasic() throws Exception { .collect(Collectors.toList()); List fieldGroups = - DataEvolutionSplitRead.splitFieldBunches(filesMetas, key -> 0); + DataEvolutionSplitRead.splitFieldBunches( + filesMetas, key -> makeBlobRowType(key.writeCols(), f -> 0)); assertThat(fieldGroups.size()).isEqualTo(2); assertThat(fieldGroups.get(0).files().size()).isEqualTo(1); @@ -155,7 +159,8 @@ public void testMultiBatch() throws Exception { assertThat(batches.size()).isEqualTo(2); for (List batch : batches) { List fieldGroups = - DataEvolutionSplitRead.splitFieldBunches(batch, file -> 0); + DataEvolutionSplitRead.splitFieldBunches( + batch, file -> makeBlobRowType(file.writeCols(), f -> 0)); assertThat(fieldGroups.size()).isEqualTo(2); assertThat(fieldGroups.get(0).files().size()).isEqualTo(1); assertThat(fieldGroups.get(1).files().size()).isEqualTo(10); @@ -249,6 +254,20 @@ protected InternalRow dataDefault(int time, int size) { RANDOM.nextInt(), BinaryString.fromBytes(randomBytes()), new BlobData(blobBytes)); } + private static RowType makeBlobRowType( + List fieldNames, Function fieldIdFunc) { + List fields = new ArrayList<>(); + if (fieldNames == null) { + fieldNames = Collections.emptyList(); + } + for (String fieldName : fieldNames) { + int fieldId = fieldIdFunc.apply(fieldName); + DataField blobField = new DataField(fieldId, fieldName, DataTypes.BLOB()); + fields.add(blobField); + } + return new RowType(fields); + } + @Override protected byte[] randomBytes() { byte[] binary = new byte[2 * 1024 * 124]; diff --git a/paimon-core/src/test/java/org/apache/paimon/append/RollingBlobFileWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/append/DataEvolutionRollingFileWriterTest.java similarity index 95% rename from paimon-core/src/test/java/org/apache/paimon/append/RollingBlobFileWriterTest.java rename to paimon-core/src/test/java/org/apache/paimon/append/DataEvolutionRollingFileWriterTest.java index 0afe95eef066..113bc3f86ed9 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/RollingBlobFileWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/DataEvolutionRollingFileWriterTest.java @@ -43,13 +43,14 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Random; import static org.assertj.core.api.Assertions.assertThat; -/** Tests for {@link RollingBlobFileWriter}. */ -public class RollingBlobFileWriterTest { +/** Tests for {@link DataEvolutionRollingFileWriter}. */ +public class DataEvolutionRollingFileWriterTest { private static final RowType SCHEMA = RowType.builder() @@ -64,7 +65,7 @@ public class RollingBlobFileWriterTest { @TempDir java.nio.file.Path tempDir; - private RollingBlobFileWriter writer; + private DataEvolutionRollingFileWriter writer; private DataFilePathFactory pathFactory; private LongCounter seqNumCounter; private byte[] testBlobData; @@ -91,10 +92,13 @@ public void setUp() throws IOException { // Initialize the writer writer = - new RollingBlobFileWriter( + new DataEvolutionRollingFileWriter( fileIO, SCHEMA_ID, FileFormat.fromIdentifier("parquet", new Options()), + null, + Collections.emptyList(), + TARGET_FILE_SIZE, TARGET_FILE_SIZE, TARGET_FILE_SIZE, SCHEMA, @@ -182,13 +186,16 @@ public void testBlobTargetFileSize() throws IOException { long blobTargetFileSize = 500 * 1024 * 1024L; // 2 MB for blob files // Create a new writer with different blob target file size - RollingBlobFileWriter blobSizeTestWriter = - new RollingBlobFileWriter( + DataEvolutionRollingFileWriter blobSizeTestWriter = + new DataEvolutionRollingFileWriter( LocalFileIO.create(), SCHEMA_ID, FileFormat.fromIdentifier("parquet", new Options()), + null, + Collections.emptyList(), 128 * 1024 * 1024, blobTargetFileSize, // Different blob target size + 128 * 1024 * 1024, SCHEMA, new DataFilePathFactory( new Path(tempDir + "/blob-size-test"), @@ -267,13 +274,16 @@ public void testSchemaValidation() throws IOException { void testBlobFileNameFormatWithSharedUuid() throws IOException { long blobTargetFileSize = 2 * 1024 * 1024L; // 2 MB for blob files - RollingBlobFileWriter fileNameTestWriter = - new RollingBlobFileWriter( + DataEvolutionRollingFileWriter fileNameTestWriter = + new DataEvolutionRollingFileWriter( LocalFileIO.create(), SCHEMA_ID, FileFormat.fromIdentifier("parquet", new Options()), + null, + Collections.emptyList(), 128 * 1024 * 1024, blobTargetFileSize, + 128 * 1024 * 1024, SCHEMA, pathFactory, // Use the same pathFactory to ensure shared UUID () -> new LongCounter(), @@ -346,13 +356,16 @@ void testBlobFileNameFormatWithSharedUuid() throws IOException { void testBlobFileNameFormatWithSharedUuidNonDescriptorMode() throws IOException { long blobTargetFileSize = 2 * 1024 * 1024L; // 2 MB for blob files - RollingBlobFileWriter fileNameTestWriter = - new RollingBlobFileWriter( + DataEvolutionRollingFileWriter fileNameTestWriter = + new DataEvolutionRollingFileWriter( LocalFileIO.create(), SCHEMA_ID, FileFormat.fromIdentifier("parquet", new Options()), + null, + Collections.emptyList(), 128 * 1024 * 1024, blobTargetFileSize, + 128 * 1024 * 1024, SCHEMA, pathFactory, // Use the same pathFactory to ensure shared UUID () -> new LongCounter(), @@ -565,10 +578,13 @@ void testBlobStatsSchemaWithCustomColumnName() throws IOException { // Reinitialize writer with custom schema writer = - new RollingBlobFileWriter( + new DataEvolutionRollingFileWriter( LocalFileIO.create(), SCHEMA_ID, FileFormat.fromIdentifier("parquet", new Options()), + null, + Collections.emptyList(), + TARGET_FILE_SIZE, TARGET_FILE_SIZE, TARGET_FILE_SIZE, customSchema, // Use custom schema diff --git a/paimon-core/src/test/java/org/apache/paimon/append/DataEvolutionRollingFileWriterWithVectorStoreTest.java b/paimon-core/src/test/java/org/apache/paimon/append/DataEvolutionRollingFileWriterWithVectorStoreTest.java new file mode 100644 index 000000000000..04c7119b08f1 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/append/DataEvolutionRollingFileWriterWithVectorStoreTest.java @@ -0,0 +1,445 @@ +/* + * 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.paimon.append; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; +import org.apache.paimon.data.BlobData; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fileindex.FileIndexOptions; +import org.apache.paimon.format.FileFormat; +import org.apache.paimon.format.blob.BlobFileFormat; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.DataFilePathFactory; +import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.options.Options; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.LongCounter; +import org.apache.paimon.utils.StatsCollectorFactories; +import org.apache.paimon.utils.VectorStoreUtils; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link DataEvolutionRollingFileWriter} with vector-store. */ +public class DataEvolutionRollingFileWriterWithVectorStoreTest { + + private static final int VECTOR_DIM = 10; + private static final RowType SCHEMA = + RowType.builder() + .field("f0", DataTypes.INT()) + .field("f1", DataTypes.STRING()) + .field("f2", DataTypes.BLOB()) + .field("f3", DataTypes.VECTOR(VECTOR_DIM, DataTypes.FLOAT())) + .field("f4", DataTypes.INT()) + .build(); + + private static final long TARGET_FILE_SIZE = 2 * 1024 * 1024L; // 2 MB + private static final long VECTOR_STORE_TARGET_FILE_SIZE = 4 * 1024 * 1024L; // 4 MB + private static final long SCHEMA_ID = 1L; + private static final String COMPRESSION = "none"; + private static final Random RANDOM = new Random(System.currentTimeMillis()); + + @TempDir java.nio.file.Path tempDir; + + private DataEvolutionRollingFileWriter writer; + private DataFilePathFactory pathFactory; + private LongCounter seqNumCounter; + + @BeforeEach + public void setUp() throws IOException { + // Setup file system and path factory + LocalFileIO fileIO = LocalFileIO.create(); + pathFactory = + new DataFilePathFactory( + new Path(tempDir + "/bucket-0"), + "parquet", + "data-", // dataFilePrefix should include the hyphen to match expected + // format: data-{uuid}-{count} + "changelog", + false, + null, + null); + seqNumCounter = new LongCounter(); + + // Initialize the writer + writer = + new DataEvolutionRollingFileWriter( + fileIO, + SCHEMA_ID, + FileFormat.fromIdentifier("parquet", new Options()), + FileFormat.fromIdentifier("json", new Options()), + Arrays.asList("f3", "f4"), + TARGET_FILE_SIZE, + TARGET_FILE_SIZE, + VECTOR_STORE_TARGET_FILE_SIZE, + SCHEMA, + pathFactory, + () -> seqNumCounter, + COMPRESSION, + new StatsCollectorFactories(new CoreOptions(new Options())), + new FileIndexOptions(), + FileSource.APPEND, + false, + false, + null); + } + + @Test + public void testBasicWriting() throws IOException { + // Write a single row + writer.write(makeRows(1, 10).get(0)); + assertThat(writer.recordCount()).isEqualTo(1); + } + + @Test + public void testMultipleWrites() throws Exception { + // Write multiple rows + int rowNum = RANDOM.nextInt(64) + 1; + writer.write(makeRows(rowNum, 10).iterator()); + writer.close(); + List metasResult = writer.result(); + + assertThat(metasResult.size()).isEqualTo(3); // blob is small, normal/blob/vector 3 files + assertThat(metasResult.get(0).fileFormat()).isEqualTo("parquet"); + assertThat(metasResult.get(1).fileFormat()).isEqualTo("blob"); + assertThat(metasResult.get(2).fileFormat()).isEqualTo("json"); + assertThat(writer.recordCount()).isEqualTo(rowNum); + + assertThat(metasResult.get(0).rowCount()).isEqualTo(metasResult.get(1).rowCount()); + assertThat(metasResult.get(0).rowCount()).isEqualTo(metasResult.get(2).rowCount()); + } + + @Test + public void testVectorStoreTargetFileSize() throws Exception { + // 100k vector-store data would create 1 normal, 1 blob, and 3 vector-store files + int rowNum = 100 * 1000; + writer.write(makeRows(rowNum, 1).iterator()); + writer.close(); + List results = writer.result(); + + // Verify that we have multiple files due to rolling + assertThat(results.size()).isGreaterThan(1); + + // Check that vector-store files meet the target size requirement + List vectorStoreFiles = + results.stream() + .filter(file -> VectorStoreUtils.isVectorStoreFile(file.fileName())) + .collect(java.util.stream.Collectors.toList()); + + assertThat(vectorStoreFiles.size()).isEqualTo(3); + + // Verify that vector-store files are close to the target size (within reasonable tolerance) + for (DataFileMeta file : vectorStoreFiles.subList(0, vectorStoreFiles.size() - 1)) { + long fileSize = file.fileSize(); + assertThat(fileSize) + .as("Vector-store file size should be close to target size") + .isGreaterThanOrEqualTo(VECTOR_STORE_TARGET_FILE_SIZE) + .isLessThanOrEqualTo(VECTOR_STORE_TARGET_FILE_SIZE + 256 * 1024); + } + + // Verify total record count + assertThat(writer.recordCount()).isEqualTo(rowNum); + } + + @Test + void testVectorStoreFileNameFormatWithSharedUuid() throws Exception { + // 100k vector-store data would create 1 normal, 1 blob, and 3 vector-store files + int rowNum = 100 * 1000; + writer.write(makeRows(rowNum, 1).iterator()); + writer.close(); + List results = writer.result(); + + // Get uuid from vector-store files. The pattern is data-{uuid}-{count}.vector-store.json + DataFileMeta oneVectorStoreFile = + results.stream() + .filter(file -> VectorStoreUtils.isVectorStoreFile(file.fileName())) + .findAny() + .get(); + String uuidAndCnt = oneVectorStoreFile.fileName().split(".vector-store.")[0]; + String prefix = uuidAndCnt.substring(0, uuidAndCnt.lastIndexOf('-') + 1); // data-{uuid}- + + // Verify all files use the same UUID and have sequential counters + for (int i = 0; i < results.size(); ++i) { + String fileName = results.get(i).fileName(); + assertThat(fileName).as("All files should use the same UUID").startsWith(prefix); + int counter = Integer.parseInt(fileName.substring(prefix.length()).split("\\.")[0]); + assertThat(counter).as("File counter should be sequential").isEqualTo(i); + } + } + + @Test + void testVectorStoreStatsMainPart() throws Exception { + // Write multiple rows + int rowNum = RANDOM.nextInt(64) + 1; + writer.write(makeRows(rowNum, 10).iterator()); + writer.close(); + List metasResult = writer.result(); + + // Check row count + for (DataFileMeta file : metasResult) { + assertThat(file.rowCount()).isEqualTo(rowNum); + assertThat(file.deleteRowCount().get()).isEqualTo(0); // There is no deleted rows + } + + // Check statistics + for (DataFileMeta file : metasResult) { + if (BlobFileFormat.isBlobFile(file.fileName())) { + assertThat(file.writeCols()).isEqualTo(Collections.singletonList("f2")); + } else if (VectorStoreUtils.isVectorStoreFile(file.fileName())) { + assertThat(file.writeCols()).isEqualTo(Arrays.asList("f3", "f4")); + // Json does not implement createStatsExtractor so we skip it here. + // assertThat(file.valueStats().minValues().getInt(1)).isGreaterThan(0); + } else { + assertThat(file.writeCols()).isEqualTo(Arrays.asList("f0", "f1")); + assertThat(file.valueStats().minValues().getInt(0)).isEqualTo(0); + assertThat(file.valueStats().maxValues().getInt(0)).isEqualTo(rowNum - 1); + } + } + + // Verify total record count + assertThat(writer.recordCount()).isEqualTo(rowNum); + } + + @Test + void testVectorStoreStatsVectorStorePart() throws Exception { + // This time we set parquet as vector-store file format. + RowType schema = + RowType.builder() + .field("f0", DataTypes.VECTOR(VECTOR_DIM, DataTypes.FLOAT())) + .field("f1", DataTypes.INT()) + .field("f2", DataTypes.BLOB()) + .field("f3", DataTypes.INT()) + .field("f4", DataTypes.STRING()) + .build(); + writer = + new DataEvolutionRollingFileWriter( + LocalFileIO.create(), + SCHEMA_ID, + FileFormat.fromIdentifier("json", new Options()), + FileFormat.fromIdentifier("parquet", new Options()), + Arrays.asList("f3", "f4"), + TARGET_FILE_SIZE, + TARGET_FILE_SIZE, + VECTOR_STORE_TARGET_FILE_SIZE, + schema, + pathFactory, + () -> seqNumCounter, + COMPRESSION, + new StatsCollectorFactories(new CoreOptions(new Options())), + new FileIndexOptions(), + FileSource.APPEND, + false, + false, + null); + + // Write multiple rows + int rowNum = RANDOM.nextInt(64) + 1; + List rows = makeRows(rowNum, 10); + for (InternalRow row : rows) { + writer.write( + GenericRow.of( + row.getVector(3), + row.getInt(4), + row.getBlob(2), + row.getInt(0), + row.getString(1))); + } + writer.close(); + List metasResult = writer.result(); + + // Check row count + for (DataFileMeta file : metasResult) { + assertThat(file.rowCount()).isEqualTo(rowNum); + assertThat(file.deleteRowCount().get()).isEqualTo(0); // There is no deleted rows + } + + // Check statistics + for (DataFileMeta file : metasResult) { + if (BlobFileFormat.isBlobFile(file.fileName())) { + assertThat(file.writeCols()).isEqualTo(Collections.singletonList("f2")); + } else if (VectorStoreUtils.isVectorStoreFile(file.fileName())) { + assertThat(file.writeCols()).isEqualTo(Arrays.asList("f3", "f4")); + assertThat(file.valueStats().minValues().getInt(0)).isEqualTo(0); + assertThat(file.valueStats().maxValues().getInt(0)).isEqualTo(rowNum - 1); + } else { + assertThat(file.writeCols()).isEqualTo(Arrays.asList("f0", "f1")); + // Json does not implement createStatsExtractor so we skip it here. + // assertThat(file.valueStats().minValues().getInt(1)).isGreaterThan(0); + } + } + + // Verify total record count + assertThat(writer.recordCount()).isEqualTo(rowNum); + } + + @Test + public void testVectorStoreNoBlob() throws Exception { + RowType schema = + RowType.builder() + .field("f0", DataTypes.INT()) + .field("f1", DataTypes.STRING()) + .field("f2", DataTypes.VECTOR(VECTOR_DIM, DataTypes.FLOAT())) + .field("f3", DataTypes.INT()) + .build(); + writer = + new DataEvolutionRollingFileWriter( + LocalFileIO.create(), + SCHEMA_ID, + FileFormat.fromIdentifier("parquet", new Options()), + FileFormat.fromIdentifier("json", new Options()), + Arrays.asList("f2", "f3"), + TARGET_FILE_SIZE, + TARGET_FILE_SIZE, + VECTOR_STORE_TARGET_FILE_SIZE, + schema, + pathFactory, + () -> seqNumCounter, + COMPRESSION, + new StatsCollectorFactories(new CoreOptions(new Options())), + new FileIndexOptions(), + FileSource.APPEND, + false, + false, + null); + + // 100k vector-store data would create 1 normal and 3 vector-store files + int rowNum = 100 * 1000; + List rows = makeRows(rowNum, 1); + for (InternalRow row : rows) { + writer.write( + GenericRow.of( + row.getInt(0), row.getString(1), row.getVector(3), row.getInt(4))); + } + writer.close(); + List results = writer.result(); + + // Check normal, blob, and vector-store files + List normalFiles = new ArrayList<>(); + List blobFiles = new ArrayList<>(); + List vectorStoreFiles = new ArrayList<>(); + for (DataFileMeta file : results) { + if (BlobFileFormat.isBlobFile(file.fileName())) { + blobFiles.add(file); + } else if (VectorStoreUtils.isVectorStoreFile(file.fileName())) { + vectorStoreFiles.add(file); + } else { + normalFiles.add(file); + } + } + assertThat(normalFiles.size()).isEqualTo(1); + assertThat(blobFiles.size()).isEqualTo(0); + assertThat(vectorStoreFiles.size()).isEqualTo(3); + + // Verify total record count + assertThat(writer.recordCount()).isEqualTo(rowNum); + } + + @Test + public void testVectorStoreTheSameFormat() throws Exception { + // vector-store file format is the same as main part + writer = + new DataEvolutionRollingFileWriter( + LocalFileIO.create(), + SCHEMA_ID, + FileFormat.fromIdentifier("json", new Options()), + FileFormat.fromIdentifier("json", new Options()), + Arrays.asList("f3", "f4"), + TARGET_FILE_SIZE, + TARGET_FILE_SIZE, + VECTOR_STORE_TARGET_FILE_SIZE, + SCHEMA, + pathFactory, + () -> seqNumCounter, + COMPRESSION, + new StatsCollectorFactories(new CoreOptions(new Options())), + new FileIndexOptions(), + FileSource.APPEND, + false, + false, + null); + + // This time we use large blob files + int rowNum = 10; + writer.write(makeRows(rowNum, 512 * 1024).iterator()); + writer.close(); + List results = writer.result(); + + // Check normal, blob, and vector-store files + List normalFiles = new ArrayList<>(); + List blobFiles = new ArrayList<>(); + List vectorStoreFiles = new ArrayList<>(); + for (DataFileMeta file : results) { + if (BlobFileFormat.isBlobFile(file.fileName())) { + blobFiles.add(file); + } else if (VectorStoreUtils.isVectorStoreFile(file.fileName())) { + vectorStoreFiles.add(file); + } else { + assertThat(file.writeCols()).isEqualTo(Arrays.asList("f0", "f1", "f3", "f4")); + normalFiles.add(file); + } + } + assertThat(normalFiles.size()).isEqualTo(1); + assertThat(blobFiles.size()).isEqualTo(3); + assertThat(vectorStoreFiles.size()).isEqualTo(0); + + // Verify total record count + assertThat(writer.recordCount()).isEqualTo(rowNum); + } + + private List makeRows(int rowNum, int blobDataSize) { + List rows = new ArrayList<>(rowNum); + byte[] blobData = new byte[blobDataSize]; + RANDOM.nextBytes(blobData); + for (int i = 0; i < rowNum; ++i) { + byte[] string = new byte[1]; + RANDOM.nextBytes(string); + byte[] buf = new byte[VECTOR_DIM]; + RANDOM.nextBytes(buf); + float[] vector = new float[VECTOR_DIM]; + for (int j = 0; j < VECTOR_DIM; ++j) { + vector[j] = buf[j]; + } + int label = RANDOM.nextInt(32) + 1; + rows.add( + GenericRow.of( + i, + BinaryString.fromBytes(string), + new BlobData(blobData), + BinaryVector.fromPrimitiveArray(vector), + label)); + } + return rows; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/append/MultipleBlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/MultipleBlobTableTest.java index 97f621a72024..4b6d1c579a9c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/MultipleBlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/MultipleBlobTableTest.java @@ -62,9 +62,7 @@ public void testBasic() throws Exception { .collect(Collectors.toList()); RowType rowType = table.schema().logicalRowType(); - List fieldGroups = - splitFieldBunches( - filesMetas, file -> rowType.getField(file.writeCols().get(0)).id()); + List fieldGroups = splitFieldBunches(filesMetas, file -> rowType); assertThat(fieldGroups.size()).isEqualTo(3); assertThat(fieldGroups.get(0).files().size()).isEqualTo(1); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/VectorStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/VectorStoreTableTest.java new file mode 100644 index 000000000000..8bc7bad2d8c4 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/append/VectorStoreTableTest.java @@ -0,0 +1,315 @@ +/* + * 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.paimon.append; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BinaryVector; +import org.apache.paimon.data.BlobData; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.operation.DataEvolutionSplitRead; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.TableTestBase; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.sink.StreamTableWrite; +import org.apache.paimon.table.sink.StreamWriteBuilder; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +/** Tests for table with vector-store and data evolution. */ +public class VectorStoreTableTest extends TableTestBase { + + private static final int VECTOR_DIM = 10; + + private AtomicInteger uniqueIdGen = new AtomicInteger(0); + + private Map rowsWritten = new HashMap<>(); + + @Test + public void testBasic() throws Exception { + int rowNum = RANDOM.nextInt(64) + 1; + + createTableDefault(); + + commitDefault(writeDataDefault(rowNum, 1)); + + AtomicInteger counter = new AtomicInteger(0); + + List filesMetas = + getTableDefault().store().newScan().plan().files().stream() + .map(ManifestEntry::file) + .collect(Collectors.toList()); + + List fieldGroups = + DataEvolutionSplitRead.splitFieldBunches( + filesMetas, key -> makeBlobRowType(key.writeCols(), f -> 0)); + + assertThat(fieldGroups.size()).isEqualTo(3); + assertThat(fieldGroups.get(0).files().size()).isEqualTo(1); + assertThat(fieldGroups.get(1).files().size()).isEqualTo(1); + assertThat(fieldGroups.get(2).files().size()).isEqualTo(1); + + readDefault( + row -> { + counter.getAndIncrement(); + InternalRow expected = rowsWritten.get(row.getInt(0)); + assertThat(row.getString(1)).isEqualTo(expected.getString(1)); + assertThat(row.getBlob(2)).isEqualTo(expected.getBlob(2)); + assertThat(row.getVector(3).toFloatArray()) + .isEqualTo(expected.getVector(3).toFloatArray()); + assertThat(row.getInt(4)).isEqualTo(expected.getInt(4)); + }); + + assertThat(counter.get()).isEqualTo(rowNum); + } + + @Test + public void testMultiBatch() throws Exception { + int rowNum = (RANDOM.nextInt(64) + 1) * 2; + + createTableDefault(); + + commitDefault(writeDataDefault(rowNum / 2, 2)); + + AtomicInteger counter = new AtomicInteger(0); + + List filesMetas = + getTableDefault().store().newScan().plan().files().stream() + .map(ManifestEntry::file) + .collect(Collectors.toList()); + + List> batches = DataEvolutionSplitRead.mergeRangesAndSort(filesMetas); + assertThat(batches.size()).isEqualTo(2); + for (List batch : batches) { + List fieldGroups = + DataEvolutionSplitRead.splitFieldBunches( + batch, file -> makeBlobRowType(file.writeCols(), f -> 0)); + assertThat(fieldGroups.size()).isEqualTo(3); + assertThat(fieldGroups.get(0).files().size()).isEqualTo(1); + assertThat(fieldGroups.get(1).files().size()).isEqualTo(1); + assertThat(fieldGroups.get(2).files().size()).isEqualTo(1); + } + + readDefault( + row -> { + counter.getAndIncrement(); + InternalRow expected = rowsWritten.get(row.getInt(0)); + assertThat(row.getString(1)).isEqualTo(expected.getString(1)); + assertThat(row.getBlob(2)).isEqualTo(expected.getBlob(2)); + assertThat(row.getVector(3).toFloatArray()) + .isEqualTo(expected.getVector(3).toFloatArray()); + assertThat(row.getInt(4)).isEqualTo(expected.getInt(4)); + }); + assertThat(counter.get()).isEqualTo(rowNum); + } + + @Test + public void testRolling() throws Exception { + // 100k vector-store data would create 1 normal, 1 blob, and 3 vector-store files + int rowNum = 100 * 1000 * 3; + + createTableDefault(); + + commitDefault(writeDataDefault(rowNum / 3, 3)); + + AtomicInteger counter = new AtomicInteger(0); + + List filesMetas = + getTableDefault().store().newScan().plan().files().stream() + .map(ManifestEntry::file) + .collect(Collectors.toList()); + + List> batches = DataEvolutionSplitRead.mergeRangesAndSort(filesMetas); + assertThat(batches.size()).isEqualTo(3); + for (List batch : batches) { + List fieldGroups = + DataEvolutionSplitRead.splitFieldBunches( + batch, file -> makeBlobRowType(file.writeCols(), f -> 0)); + assertThat(fieldGroups.size()).isEqualTo(3); + assertThat(fieldGroups.get(0).files().size()).isEqualTo(1); + assertThat(fieldGroups.get(1).files().size()).isEqualTo(1); + assertThat(fieldGroups.get(2).files().size()).isEqualTo(3); + } + + readDefault( + row -> { + counter.getAndIncrement(); + InternalRow expected = rowsWritten.get(row.getInt(0)); + assertThat(row.getString(1)).isEqualTo(expected.getString(1)); + assertThat(row.getBlob(2)).isEqualTo(expected.getBlob(2)); + assertThat(row.getVector(3).toFloatArray()) + .isEqualTo(expected.getVector(3).toFloatArray()); + assertThat(row.getInt(4)).isEqualTo(expected.getInt(4)); + }); + + assertThat(counter.get()).isEqualTo(rowNum); + } + + @Test + public void testWithoutBlob() throws Exception { + // 100k vector-store data would create 1 normal, 1 blob, and 3 vector-store files + int rowNum = 100 * 1000 * 3; + + catalog.createTable(identifier(), schemaWithoutBlob(), true); + + commitDefault(writeDataWithoutBlob(rowNum / 3, 3)); + + AtomicInteger counter = new AtomicInteger(0); + + List filesMetas = + getTableDefault().store().newScan().plan().files().stream() + .map(ManifestEntry::file) + .collect(Collectors.toList()); + + List> batches = DataEvolutionSplitRead.mergeRangesAndSort(filesMetas); + assertThat(batches.size()).isEqualTo(3); + for (List batch : batches) { + List fieldGroups = + DataEvolutionSplitRead.splitFieldBunches( + batch, file -> makeBlobRowType(file.writeCols(), f -> 0)); + assertThat(fieldGroups.size()).isEqualTo(2); + assertThat(fieldGroups.get(0).files().size()).isEqualTo(1); + assertThat(fieldGroups.get(1).files().size()).isEqualTo(3); + } + + readDefault( + row -> { + counter.getAndIncrement(); + InternalRow expected = rowsWritten.get(row.getInt(0)); + assertThat(row.getString(1)).isEqualTo(expected.getString(1)); + assertThat(row.getVector(2).toFloatArray()) + .isEqualTo(expected.getVector(3).toFloatArray()); + assertThat(row.getInt(3)).isEqualTo(expected.getInt(4)); + }); + + assertThat(counter.get()).isEqualTo(rowNum); + } + + @Override + protected Schema schemaDefault() { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("f0", DataTypes.INT()); + schemaBuilder.column("f1", DataTypes.STRING()); + schemaBuilder.column("f2", DataTypes.BLOB()); + schemaBuilder.column("f3", DataTypes.VECTOR(VECTOR_DIM, DataTypes.FLOAT())); + schemaBuilder.column("f4", DataTypes.INT()); + schemaBuilder.option(CoreOptions.TARGET_FILE_SIZE.key(), "2 MB"); + schemaBuilder.option(CoreOptions.VECTOR_STORE_TARGET_FILE_SIZE.key(), "4 MB"); + schemaBuilder.option(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + schemaBuilder.option(CoreOptions.DATA_EVOLUTION_ENABLED.key(), "true"); + schemaBuilder.option(CoreOptions.VECTOR_STORE_FIELDS.key(), "f3,f4"); + schemaBuilder.option(CoreOptions.VECTOR_STORE_FORMAT.key(), "json"); + schemaBuilder.option(CoreOptions.FILE_COMPRESSION.key(), "none"); + return schemaBuilder.build(); + } + + private Schema schemaWithoutBlob() { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("f0", DataTypes.INT()); + schemaBuilder.column("f1", DataTypes.STRING()); + schemaBuilder.column("f2", DataTypes.VECTOR(VECTOR_DIM, DataTypes.FLOAT())); + schemaBuilder.column("f3", DataTypes.INT()); + schemaBuilder.option(CoreOptions.TARGET_FILE_SIZE.key(), "2 MB"); + schemaBuilder.option(CoreOptions.VECTOR_STORE_TARGET_FILE_SIZE.key(), "4 MB"); + schemaBuilder.option(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + schemaBuilder.option(CoreOptions.DATA_EVOLUTION_ENABLED.key(), "true"); + schemaBuilder.option(CoreOptions.VECTOR_STORE_FIELDS.key(), "f2,f3"); + schemaBuilder.option(CoreOptions.VECTOR_STORE_FORMAT.key(), "json"); + schemaBuilder.option(CoreOptions.FILE_COMPRESSION.key(), "none"); + return schemaBuilder.build(); + } + + protected List writeDataWithoutBlob(int size, int times) throws Exception { + Table table = getTableDefault(); + List messages = new ArrayList<>(); + for (int time = 0; time < times; time++) { + StreamWriteBuilder builder = table.newStreamWriteBuilder(); + builder.withCommitUser(commitUser); + try (StreamTableWrite streamTableWrite = builder.newWrite()) { + for (int j = 0; j < size; j++) { + InternalRow row = dataDefault(time, j); + streamTableWrite.write( + GenericRow.of( + row.getInt(0), + row.getString(1), + row.getVector(3), + row.getInt(4))); + } + messages.addAll(streamTableWrite.prepareCommit(false, Long.MAX_VALUE)); + } + } + return messages; + } + + @Override + protected InternalRow dataDefault(int time, int size) { + byte[] stringBytes = new byte[1]; + RANDOM.nextBytes(stringBytes); + byte[] blobBytes = new byte[1]; + RANDOM.nextBytes(blobBytes); + byte[] vectorBytes = new byte[VECTOR_DIM]; + RANDOM.nextBytes(vectorBytes); + float[] vector = new float[VECTOR_DIM]; + for (int i = 0; i < VECTOR_DIM; i++) { + vector[i] = vectorBytes[i]; + } + int id = uniqueIdGen.getAndIncrement(); + InternalRow row = + GenericRow.of( + id, + BinaryString.fromBytes(stringBytes), + new BlobData(blobBytes), + BinaryVector.fromPrimitiveArray(vector), + RANDOM.nextInt(32) + 1); + rowsWritten.put(id, row); + return row; + } + + private static RowType makeBlobRowType( + List fieldNames, Function fieldIdFunc) { + List fields = new ArrayList<>(); + if (fieldNames == null) { + fieldNames = Collections.emptyList(); + } + for (String fieldName : fieldNames) { + int fieldId = fieldIdFunc.apply(fieldName); + DataField blobField = new DataField(fieldId, fieldName, DataTypes.BLOB()); + fields.add(blobField); + } + return new RowType(fields); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index 3ffe8606ab08..f1baede275bb 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -271,6 +271,9 @@ public void testFileSuffix(@TempDir java.nio.file.Path tempDir) throws Exception IOManager.create(tempDir.toString()), 0, fileFormat, + null, + Collections.emptyList(), + 10, 10, 10, schema, diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/DataEvolutionReadTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/DataEvolutionReadTest.java index 452813d9d93c..19d53f45309b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/DataEvolutionReadTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/DataEvolutionReadTest.java @@ -21,9 +21,12 @@ import org.apache.paimon.data.Timestamp; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.manifest.FileSource; -import org.apache.paimon.operation.DataEvolutionSplitRead.BlobBunch; import org.apache.paimon.operation.DataEvolutionSplitRead.FieldBunch; +import org.apache.paimon.operation.DataEvolutionSplitRead.SplitBunch; import org.apache.paimon.stats.SimpleStats; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -32,20 +35,21 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.function.Function; import static org.apache.paimon.operation.DataEvolutionSplitRead.splitFieldBunches; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests for {@link BlobBunch}. */ +/** Tests for {@link SplitBunch}. */ public class DataEvolutionReadTest { - private BlobBunch blobBunch; + private SplitBunch blobBunch; @BeforeEach public void setUp() { - blobBunch = new BlobBunch(Long.MAX_VALUE, false); + blobBunch = new SplitBunch(Long.MAX_VALUE, false); } @Test @@ -84,7 +88,7 @@ public void testAddNonBlobFileThrowsException() { assertThatThrownBy(() -> blobBunch.add(normalFile)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Only blob file can be added to a blob bunch."); + .hasMessage("Only blob/vector-store file can be added to this bunch."); } @Test @@ -97,7 +101,7 @@ public void testAddBlobFileWithSameFirstRowId() { assertThatThrownBy(() -> blobBunch.add(blobEntry2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Blob file with same first row id should have decreasing sequence number."); + "Blob/vector-store file with same first row id should have decreasing sequence number."); } @Test @@ -136,7 +140,7 @@ public void testAddBlobFileWithOverlappingRowIdAndHigherSequenceNumber() { assertThatThrownBy(() -> blobBunch.add(blobEntry2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Blob file with overlapping row id should have decreasing sequence number."); + "Blob/vector-store file with overlapping row id should have decreasing sequence number."); } @Test @@ -148,7 +152,8 @@ public void testAddBlobFileWithNonContinuousRowId() { // Adding file with non-continuous row id should throw exception assertThatThrownBy(() -> blobBunch.add(blobEntry2)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Blob file first row id should be continuous, expect 100 but got 200"); + .hasMessage( + "Blob/vector-store file first row id should be continuous, expect 100 but got 200"); } @Test @@ -161,7 +166,7 @@ public void testAddBlobFileWithDifferentWriteCols() { // Adding file with different write columns should throw exception assertThatThrownBy(() -> blobBunch.add(blobEntry2)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("All files in a blob bunch should have the same write columns."); + .hasMessage("All files in this bunch should have the same write columns."); } @Test @@ -214,10 +219,11 @@ public void testComplexBlobBunchScenario2() { assertThat(batch.get(8).fileName()).contains("blob4"); // skip assertThat(batch.get(9).fileName()).contains("blob8"); // pick - List fieldBunches = splitFieldBunches(batch, file -> 0); + List fieldBunches = + splitFieldBunches(batch, file -> makeBlobRowType(file.writeCols(), f -> 0)); assertThat(fieldBunches.size()).isEqualTo(2); - BlobBunch blobBunch = (BlobBunch) fieldBunches.get(1); + SplitBunch blobBunch = (SplitBunch) fieldBunches.get(1); assertThat(blobBunch.files).hasSize(4); assertThat(blobBunch.files.get(0).fileName()).contains("blob5"); assertThat(blobBunch.files.get(1).fileName()).contains("blob9"); @@ -265,17 +271,18 @@ public void testComplexBlobBunchScenario3() { List batch = batches.get(0); List fieldBunches = - splitFieldBunches(batch, file -> file.writeCols().get(0).hashCode()); + splitFieldBunches( + batch, file -> makeBlobRowType(file.writeCols(), String::hashCode)); assertThat(fieldBunches.size()).isEqualTo(3); - BlobBunch blobBunch = (BlobBunch) fieldBunches.get(1); + SplitBunch blobBunch = (SplitBunch) fieldBunches.get(1); assertThat(blobBunch.files).hasSize(4); assertThat(blobBunch.files.get(0).fileName()).contains("blob5"); assertThat(blobBunch.files.get(1).fileName()).contains("blob9"); assertThat(blobBunch.files.get(2).fileName()).contains("blob7"); assertThat(blobBunch.files.get(3).fileName()).contains("blob8"); - blobBunch = (BlobBunch) fieldBunches.get(2); + blobBunch = (SplitBunch) fieldBunches.get(2); assertThat(blobBunch.files).hasSize(4); assertThat(blobBunch.files.get(0).fileName()).contains("blob15"); assertThat(blobBunch.files.get(1).fileName()).contains("blob19"); @@ -322,22 +329,22 @@ private DataFileMeta createBlobFileWithCols( @Test public void testRowIdPushDown() { - BlobBunch blobBunch = new BlobBunch(Long.MAX_VALUE, true); + SplitBunch blobBunch = new SplitBunch(Long.MAX_VALUE, true); DataFileMeta blobEntry1 = createBlobFile("blob1", 0, 100, 1); DataFileMeta blobEntry2 = createBlobFile("blob2", 200, 300, 1); blobBunch.add(blobEntry1); - BlobBunch finalBlobBunch = blobBunch; + SplitBunch finalBlobBunch = blobBunch; DataFileMeta finalBlobEntry = blobEntry2; assertThatCode(() -> finalBlobBunch.add(finalBlobEntry)).doesNotThrowAnyException(); - blobBunch = new BlobBunch(Long.MAX_VALUE, true); + blobBunch = new SplitBunch(Long.MAX_VALUE, true); blobEntry1 = createBlobFile("blob1", 0, 100, 1); blobEntry2 = createBlobFile("blob2", 50, 200, 2); blobBunch.add(blobEntry1); blobBunch.add(blobEntry2); assertThat(blobBunch.files).containsExactlyInAnyOrder(blobEntry2); - BlobBunch finalBlobBunch2 = blobBunch; + SplitBunch finalBlobBunch2 = blobBunch; DataFileMeta blobEntry3 = createBlobFile("blob2", 250, 100, 2); assertThatCode(() -> finalBlobBunch2.add(blobEntry3)).doesNotThrowAnyException(); } @@ -371,4 +378,18 @@ private DataFileMeta createNormalFile( firstRowId, null); } + + private static RowType makeBlobRowType( + List fieldNames, Function fieldIdFunc) { + List fields = new ArrayList<>(); + if (fieldNames == null) { + fieldNames = Collections.emptyList(); + } + for (String fieldName : fieldNames) { + int fieldId = fieldIdFunc.apply(fieldName); + DataField blobField = new DataField(fieldId, fieldName, DataTypes.BLOB()); + fields.add(blobField); + } + return new RowType(fields); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/DataEvolutionSplitReadTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/DataEvolutionSplitReadTest.java index 2f0b1b2f1b4d..305c5c2b78f9 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/DataEvolutionSplitReadTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/DataEvolutionSplitReadTest.java @@ -115,6 +115,24 @@ public void testSplitWithMultipleBlobFilesPerGroup() { assertEquals(Arrays.asList(file4, file5, file6), result.get(1)); } + @Test + public void testSplitWithMultipleVectorStoreFilesPerGroup() { + DataFileMeta file1 = createFile("file1.parquet", 1L, 10, 1); + DataFileMeta file2 = createFile("file2.vector-store.json", 1L, 1, 1); + DataFileMeta file3 = createFile("file3.vector-store.json", 2L, 9, 1); + DataFileMeta file4 = createFile("file4.parquet", 20L, 10, 2); + DataFileMeta file5 = createFile("file5.vector-store.json", 20L, 5, 2); + DataFileMeta file6 = createFile("file6.vector-store.json", 25L, 5, 2); + DataFileMeta file7 = createFile("file7.parquet", 1L, 10, 3); + + List files = Arrays.asList(file1, file2, file3, file4, file5, file6, file7); + List> result = DataEvolutionSplitRead.mergeRangesAndSort(files); + + assertEquals(2, result.size()); + assertEquals(Arrays.asList(file7, file1, file2, file3), result.get(0)); + assertEquals(Arrays.asList(file4, file5, file6), result.get(1)); + } + private static DataFileMeta createFile( String name, long firstRowId, long rowCount, long maxSequence) { return DataFileMeta.create( diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java index 57613fe67f38..3edad55173d0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java @@ -35,7 +35,10 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static org.apache.paimon.CoreOptions.BUCKET; +import static org.apache.paimon.CoreOptions.DATA_EVOLUTION_ENABLED; import static org.apache.paimon.CoreOptions.SCAN_SNAPSHOT_ID; +import static org.apache.paimon.CoreOptions.VECTOR_STORE_FIELDS; +import static org.apache.paimon.CoreOptions.VECTOR_STORE_FORMAT; import static org.apache.paimon.schema.SchemaValidation.validateTableSchema; import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatNoException; @@ -176,4 +179,142 @@ public void testPartialUpdateTableAggregateFunctionWithoutSequenceGroup() { options.put("fields.f2.sequence-group", "f3"); assertThatCode(() -> validateTableSchemaExec(options)).doesNotThrowAnyException(); } + + @Test + public void testVectorStoreUnknownColumn() { + Map options = new HashMap<>(); + options.put(BUCKET.key(), String.valueOf(-1)); + options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + options.put(DATA_EVOLUTION_ENABLED.key(), "true"); + options.put(CoreOptions.FILE_FORMAT.key(), "avro"); + options.put(VECTOR_STORE_FORMAT.key(), "json"); + options.put(VECTOR_STORE_FIELDS.key(), "f99"); + + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField(1, "f1", DataTypes.STRING())); + assertThatThrownBy( + () -> + validateTableSchema( + new TableSchema( + 1, + fields, + 10, + emptyList(), + emptyList(), + options, + ""))) + .hasMessage("Some of the columns specified as vector-store are unknown."); + } + + @Test + public void testVectorStoreContainsBlobColumn() { + Map options = new HashMap<>(); + options.put(BUCKET.key(), String.valueOf(-1)); + options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + options.put(DATA_EVOLUTION_ENABLED.key(), "true"); + options.put(CoreOptions.FILE_FORMAT.key(), "avro"); + options.put(VECTOR_STORE_FORMAT.key(), "json"); + options.put(VECTOR_STORE_FIELDS.key(), "blob"); + + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField(1, "blob", DataTypes.BLOB())); + assertThatThrownBy( + () -> + validateTableSchema( + new TableSchema( + 1, + fields, + 10, + emptyList(), + emptyList(), + options, + ""))) + .hasMessage("The vector-store columns can not be blob type."); + } + + @Test + public void testVectorStoreContainsPartitionColumn() { + Map options = new HashMap<>(); + options.put(BUCKET.key(), String.valueOf(-1)); + options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + options.put(DATA_EVOLUTION_ENABLED.key(), "true"); + options.put(CoreOptions.FILE_FORMAT.key(), "avro"); + options.put(VECTOR_STORE_FORMAT.key(), "json"); + options.put(VECTOR_STORE_FIELDS.key(), "f1"); + + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField(1, "f1", DataTypes.STRING())); + assertThatThrownBy( + () -> + validateTableSchema( + new TableSchema( + 1, + fields, + 10, + singletonList("f1"), + emptyList(), + options, + ""))) + .hasMessage("The vector-store columns can not be part of partition keys."); + } + + @Test + public void testVectorStoreRequireNormalColumns() { + Map options = new HashMap<>(); + options.put(BUCKET.key(), String.valueOf(-1)); + options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + options.put(DATA_EVOLUTION_ENABLED.key(), "true"); + options.put(CoreOptions.FILE_FORMAT.key(), "avro"); + options.put(VECTOR_STORE_FORMAT.key(), "json"); + options.put(VECTOR_STORE_FIELDS.key(), "f0,f1"); + + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField(1, "f1", DataTypes.STRING())); + assertThatThrownBy( + () -> + validateTableSchema( + new TableSchema( + 1, + fields, + 10, + emptyList(), + emptyList(), + options, + ""))) + .hasMessage("Table with vector-store must have other normal columns."); + } + + @Test + public void testVectorStoreRequiresDataEvolutionEnabled() { + Map options = new HashMap<>(); + options.put(CoreOptions.FILE_FORMAT.key(), "avro"); + options.put(VECTOR_STORE_FORMAT.key(), "json"); + options.put(VECTOR_STORE_FIELDS.key(), "f1"); + + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField(1, "f1", DataTypes.STRING())); + assertThatThrownBy( + () -> + validateTableSchema( + new TableSchema( + 1, + fields, + 10, + emptyList(), + emptyList(), + options, + ""))) + .hasMessage( + "Data evolution config must enabled for table with vector-store file format."); + } } diff --git a/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/FlinkVectorStoreE2eTest.java b/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/FlinkVectorStoreE2eTest.java new file mode 100644 index 000000000000..c429cc241278 --- /dev/null +++ b/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/FlinkVectorStoreE2eTest.java @@ -0,0 +1,112 @@ +/* + * 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.paimon.tests; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +/** E2E test for vector-store with data evolution. */ +public class FlinkVectorStoreE2eTest extends E2eTestBase { + + @Test + public void testVectorStoreTable() throws Exception { + Random rnd = new Random(System.currentTimeMillis()); + int vectorDim = rnd.nextInt(1024) + 1; + final int itemNum = rnd.nextInt(64) + 1; + + String catalogDdl = + String.format( + "CREATE CATALOG ts_catalog WITH (\n" + + " 'type' = 'paimon',\n" + + " 'warehouse' = '%s'\n" + + ");", + TEST_DATA_DIR + "/" + UUID.randomUUID() + ".store"); + + String useCatalogCmd = "USE CATALOG ts_catalog;"; + + String createTableDdl = + String.format( + "CREATE TABLE IF NOT EXISTS ts_table (\n" + + " id BIGINT,\n" + + " embed ARRAY\n" + + ") WITH (\n" + + " 'file.format' = 'parquet',\n" + + " 'file.compression' = 'none',\n" + + " 'row-tracking.enabled' = 'true',\n" + + " 'data-evolution.enabled' = 'true',\n" + + " 'vector-store.format' = 'json',\n" + + " 'vector-store.fields' = 'embed',\n" + + " 'field.embed.vector-dim' = '%d'\n" + + ");", + vectorDim); + + float[][] vectors = new float[itemNum][vectorDim]; + byte[] vectorDataBuf = new byte[vectorDim]; + for (int i = 0; i < itemNum; ++i) { + vectors[i] = new float[vectorDim]; + rnd.nextBytes(vectorDataBuf); + for (int j = 0; j < vectorDim; ++j) { + vectors[i][j] = vectorDataBuf[j]; + } + } + + List values = new ArrayList<>(); + String[] expected = new String[itemNum]; + for (int id = 0; id < itemNum; ++id) { + values.add(String.format("(%d, %s)", id, arrayLiteral(vectors[id]))); + expected[id] = String.format("%d, %s", id, Arrays.toString(vectors[id])); + } + + runBatchSql( + "INSERT INTO ts_table VALUES " + String.join(", ", values) + ";", + catalogDdl, + useCatalogCmd, + createTableDdl); + + runBatchSql( + "INSERT INTO result1 SELECT * FROM ts_table;", + catalogDdl, + useCatalogCmd, + createTableDdl, + createResultSink("result1", "id BIGINT, embed ARRAY")); + checkResult(expected); + clearCurrentResults(); + + runBatchSql( + "INSERT INTO result2 SELECT " + + "COUNT(*) AS total, " + + "SUM(CASE WHEN file_path LIKE '%.vector-store%.json' THEN 1 ELSE 0 END) " + + "AS vector_files " + + "FROM `ts_table$files`;", + catalogDdl, + useCatalogCmd, + createTableDdl, + createResultSink("result2", "total BIGINT, vector_files BIGINT")); + checkResult("2, 1"); + } + + private String arrayLiteral(float[] vector) { + return "ARRAY" + Arrays.toString(vector); + } +}