From 60651f947913ed0c07026dafd45639d3b9950eac Mon Sep 17 00:00:00 2001 From: ColdL Date: Mon, 12 Jan 2026 17:55:18 +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 | 4 + .../apache/paimon/types/ReassignFieldId.java | 6 + .../java/org/apache/paimon/types/VecType.java | 177 +++++++++++++++ .../arrow/ArrowFieldTypeConversion.java | 7 + .../Arrow2PaimonVectorConverter.java | 6 + .../ArrowFieldWriterFactoryVisitor.java | 6 + .../org/apache/paimon/PartitionSettedRow.java | 8 + .../apache/paimon/casting/CastedArray.java | 6 + .../org/apache/paimon/casting/CastedRow.java | 6 + .../org/apache/paimon/casting/CastedVec.java | 113 ++++++++++ .../paimon/casting/DefaultValueRow.java | 9 + .../paimon/casting/FallbackMappingRow.java | 9 + .../paimon/data/AbstractBinaryWriter.java | 8 + .../org/apache/paimon/data/ArrayBasedVec.java | 203 ++++++++++++++++++ .../org/apache/paimon/data/BinaryArray.java | 5 + .../org/apache/paimon/data/BinaryRow.java | 5 + .../org/apache/paimon/data/BinaryWriter.java | 9 + .../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 + .../org/apache/paimon/data/InternalVec.java | 89 ++++++++ .../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 | 4 + .../serializer/InternalVecSerializer.java | 120 +++++++++++ .../fileindex/bitmap/BitmapTypeVisitor.java | 6 + .../fileindex/bloomfilter/FastHash.java | 6 + .../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 + .../org/apache/paimon/utils/ProjectedVec.java | 127 +++++++++++ .../paimon/utils/VectorMappingUtils.java | 6 + .../datagen/RandomGeneratorVisitor.java | 6 + .../paimon/stats/SimpleStatsEvolution.java | 6 + .../org/apache/paimon/utils/OffsetRow.java | 6 + .../org/apache/paimon/utils/PartialRow.java | 6 + .../paimon/append/VecTypeTableTest.java | 97 +++++++++ .../paimon/flink/DataTypeToLogicalType.java | 6 + .../apache/paimon/flink/FlinkRowWrapper.java | 11 + .../paimon/format/json/JsonFileFormat.java | 1 + .../paimon/format/json/JsonFileReader.java | 10 + .../paimon/format/json/JsonFormatWriter.java | 13 ++ .../format/orc/writer/FieldWriterFactory.java | 6 + .../reader/ParquetVectorUpdaterFactory.java | 6 + .../filter2/predicate/ParquetFilters.java | 6 + .../hive/objectinspector/HivePaimonArray.java | 6 + .../paimon/format/lance/LanceFileFormat.java | 6 + .../paimon/spark/SparkInternalRowWrapper.java | 11 + .../org/apache/paimon/spark/SparkRow.java | 11 + 68 files changed, 1328 insertions(+), 1 deletion(-) create mode 100644 paimon-api/src/main/java/org/apache/paimon/types/VecType.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/casting/CastedVec.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/ArrayBasedVec.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/InternalVec.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalVecSerializer.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/utils/ProjectedVec.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/append/VecTypeTableTest.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..df7323f25b8a 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(VecType vecType) { + return Collections.singletonList(vecType.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..773d4dfa674b 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(VecType vecType) { + return defaultMethod(vecType); + } + @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..64731d9b48d4 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 VecType(!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 parseVecType(); default: throw parsingError("Unsupported type: " + token().value); } @@ -665,5 +672,16 @@ private int parseOptionalPrecision(int defaultPrecision) { } return precision; } + + private DataType parseVecType() { + // 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..5c1062d89c64 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(VecType vecType); + 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..47052105394c 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 VecType VECTOR(int length, DataType element) { + return new VecType(length, element); + } + public static CharType CHAR(int length) { return new CharType(length); } 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..122ae28fe708 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,12 @@ public DataType visit(ArrayType arrayType) { return new ArrayType(arrayType.isNullable(), arrayType.getElementType().accept(this)); } + @Override + public DataType visit(VecType vecType) { + return new VecType( + vecType.isNullable(), vecType.getLength(), vecType.getElementType().accept(this)); + } + @Override public DataType visit(MultisetType multisetType) { return new MultisetType( diff --git a/paimon-api/src/main/java/org/apache/paimon/types/VecType.java b/paimon-api/src/main/java/org/apache/paimon/types/VecType.java new file mode 100644 index 000000000000..46298e1fb8b0 --- /dev/null +++ b/paimon-api/src/main/java/org/apache/paimon/types/VecType.java @@ -0,0 +1,177 @@ +/* + * 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 VecType 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 VecType(boolean isNullable, int length, DataType elementType) { + super(isNullable, DataTypeRoot.VECTOR); + 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 VecType(int length, DataType elementType) { + this(true, length, elementType); + } + + 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 VecType(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; + } + VecType vecType = (VecType) o; + return elementType.equals(vecType.elementType) && length == vecType.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; + } + VecType vecType = (VecType) o; + return elementType.equalsIgnoreFieldId(vecType.elementType) && length == vecType.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; + } + VecType vecType = (VecType) o; + return elementType.isPrunedFrom(vecType.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 d97df368bdf2..cea79e5847e4 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.VecType; 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(VecType vecType) { + ArrowType arrowType = new ArrowType.FixedSizeList(vecType.getLength()); + return new FieldType(vecType.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..a3289c526067 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.VecType; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; @@ -482,6 +483,11 @@ public ColumnVector getColumnVector() { }; } + @Override + public Arrow2PaimonVectorConverter visit(VecType vecType) { + throw new UnsupportedOperationException("Doesn't support VecType."); + } + @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 3c22ef8bb551..fce2c58d09e9 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.VecType; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.complex.ListVector; @@ -164,6 +165,11 @@ public ArrowFieldWriterFactory visit(ArrayType arrayType) { isNullable); } + @Override + public ArrowFieldWriterFactory visit(VecType vecType) { + throw new UnsupportedOperationException("Doesn't support VecType."); + } + @Override public ArrowFieldWriterFactory visit(MultisetType multisetType) { throw new UnsupportedOperationException("Doesn't support MultisetType."); 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..4a3e592b3ad6 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.InternalVec; 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 InternalVec getVec(int pos) { + return partitionInfo.inPartitionRow(pos) + ? partition.getVec(partitionInfo.getRealIndex(pos)) + : row.getVec(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..4b402449d45a 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.InternalVec; 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 InternalVec getVec(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..c9cceaea16a4 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.InternalVec; 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 InternalVec getVec(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/CastedVec.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastedVec.java new file mode 100644 index 000000000000..1ec4fed05232 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastedVec.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.InternalVec; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.data.variant.Variant; + +/** + * An implementation of {@link InternalVec} which provides a casted view of the underlying {@link + * InternalVec}. + * + *

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

This method replaces the vector in place and does not return a new object. This is done + * for performance reasons. + */ + public static CastedVec from(CastElementGetter castElementGetter) { + return new CastedVec(castElementGetter); + } + + public CastedVec replaceVec(InternalVec vec) { + super.replaceArray(vec); + return this; + } + + @Override + public CastedArray replaceArray(InternalArray array) { + throw new IllegalArgumentException("CastedVec does not support replaceArray."); + } + + @Override + public BinaryString getString(int pos) { + throw new UnsupportedOperationException("CastedVec does not support String."); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + throw new UnsupportedOperationException("CastedVec does not support Decimal."); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + throw new UnsupportedOperationException("CastedVec does not support Timestamp."); + } + + @Override + public byte[] getBinary(int pos) { + throw new UnsupportedOperationException("CastedVec does not support Binary."); + } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("CastedVec does not support Variant."); + } + + @Override + public Blob getBlob(int pos) { + throw new UnsupportedOperationException("CastedVec does not support Blob."); + } + + @Override + public InternalArray getArray(int pos) { + throw new UnsupportedOperationException("CastedVec does not support nested Array."); + } + + @Override + public InternalVec getVec(int pos) { + throw new UnsupportedOperationException("CastedVec does not support nested VecType."); + } + + @Override + public InternalMap getMap(int pos) { + throw new UnsupportedOperationException("CastedVec does not support Map."); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + throw new UnsupportedOperationException("CastedVec 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..5802942f5273 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.InternalVec; 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 InternalVec getVec(int pos) { + if (!row.isNullAt(pos)) { + return row.getVec(pos); + } + return defaultValueRow.getVec(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..3833685f02ec 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.InternalVec; 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 InternalVec getVec(int pos) { + if (mappings[pos] != -1 && main.isNullAt(pos)) { + return fallbackRow.getVec(mappings[pos]); + } + return main.getVec(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..3f56015a6257 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.InternalVecSerializer; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.memory.MemorySegment; import org.apache.paimon.memory.MemorySegmentUtils; @@ -91,6 +92,13 @@ public void writeArray(int pos, InternalArray input, InternalArraySerializer ser pos, binary.getSegments(), binary.getOffset(), binary.getSizeInBytes()); } + @Override + public void writeVec(int pos, InternalVec input, InternalVecSerializer serializer) { + BinaryArray binary = serializer.toBinaryArray(input); + writeSegmentsToVarLenPart( + pos, binary.getSegments(), binary.getOffset(), binary.getSizeInBytes()); + } + @Override public void writeMap(int pos, InternalMap input, InternalMapSerializer serializer) { BinaryMap binary = serializer.toBinaryMap(input); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/ArrayBasedVec.java b/paimon-common/src/main/java/org/apache/paimon/data/ArrayBasedVec.java new file mode 100644 index 000000000000..7e4550a3d820 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/ArrayBasedVec.java @@ -0,0 +1,203 @@ +/* + * 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.types.VecType; + +import java.io.Serializable; +import java.util.Objects; + +/** + * An internal data structure representing data of {@link VecType}. + * + *

Note: All elements of this data structure must be numeric. + * + *

{@link ArrayBasedVec} is an implementation of {@link InternalVec} which wraps {@link + * InternalArray}. + * + * @since 2.0.0 + */ +@Public +public final class ArrayBasedVec implements InternalVec, Serializable { + + private static final long serialVersionUID = 1L; + private final InternalArray array; + + private ArrayBasedVec(InternalArray array) { + this.array = array; + } + + /** Creates an instance of {@link ArrayBasedVec} by {@link InternalArray}. */ + public static ArrayBasedVec from(InternalArray array) { + return new ArrayBasedVec(array); + } + + public InternalArray getInnerArray() { + return array; + } + + @Override + public int size() { + return array.size(); + } + + @Override + public boolean isNullAt(int pos) { + return array.isNullAt(pos); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ArrayBasedVec that = (ArrayBasedVec) o; + return array.equals(that.array); + } + + @Override + public int hashCode() { + return Objects.hash(array.size(), array); + } + + @Override + public boolean getBoolean(int pos) { + return array.getBoolean(pos); + } + + @Override + public byte getByte(int pos) { + return array.getByte(pos); + } + + @Override + public short getShort(int pos) { + return array.getShort(pos); + } + + @Override + public int getInt(int pos) { + return array.getInt(pos); + } + + @Override + public long getLong(int pos) { + return array.getLong(pos); + } + + @Override + public float getFloat(int pos) { + return array.getFloat(pos); + } + + @Override + public double getDouble(int pos) { + return array.getDouble(pos); + } + + @Override + public byte[] getBinary(int pos) { + throw new UnsupportedOperationException("ArrayBasedVec does not support Binary."); + } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("ArrayBasedVec does not support Variant."); + } + + @Override + public Blob getBlob(int pos) { + throw new UnsupportedOperationException("ArrayBasedVec does not support Blob."); + } + + @Override + public BinaryString getString(int pos) { + throw new UnsupportedOperationException("ArrayBasedVec does not support BinaryString."); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + throw new UnsupportedOperationException("ArrayBasedVec does not support Decimal."); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + throw new UnsupportedOperationException("ArrayBasedVec does not support Timestamp."); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + throw new UnsupportedOperationException("ArrayBasedVec does not support Row."); + } + + @Override + public InternalArray getArray(int pos) { + throw new UnsupportedOperationException("ArrayBasedVec does not support nested Array."); + } + + @Override + public InternalVec getVec(int pos) { + throw new UnsupportedOperationException("ArrayBasedVec does not support nested VecType."); + } + + @Override + public InternalMap getMap(int pos) { + throw new UnsupportedOperationException("ArrayBasedVec does not support Map."); + } + + @Override + public boolean[] toBooleanArray() { + return array.toBooleanArray(); + } + + @Override + public byte[] toByteArray() { + return array.toByteArray(); + } + + @Override + public short[] toShortArray() { + return array.toShortArray(); + } + + @Override + public int[] toIntArray() { + return array.toIntArray(); + } + + @Override + public long[] toLongArray() { + return array.toLongArray(); + } + + @Override + public float[] toFloatArray() { + return array.toFloatArray(); + } + + @Override + public double[] toDoubleArray() { + return array.toDoubleArray(); + } +} 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..b3854f49c6b8 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 InternalVec getVec(int pos) { + throw new IllegalArgumentException("Unsupported type: vecType"); + } + @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..3d819c22c19e 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,11 @@ public InternalArray getArray(int pos) { return MemorySegmentUtils.readArrayData(segments, offset, getLong(pos)); } + @Override + public InternalVec getVec(int pos) { + throw new IllegalArgumentException("Unsupported type: vecType"); + } + @Override public InternalMap getMap(int pos) { assertIndexIsValid(pos); 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..fa926a702698 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.InternalVecSerializer; 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 writeVec(int pos, InternalVec value, InternalVecSerializer serializer); + void writeMap(int pos, InternalMap value, InternalMapSerializer serializer); void writeRow(int pos, InternalRow value, InternalRowSerializer serializer); @@ -210,6 +213,12 @@ static ValueSetter createValueSetter(DataType elementType, Serializer seriali pos, (InternalArray) value, (InternalArraySerializer) arraySerializer); + case VECTOR: + final Serializer vecSerializer = + serializer == null ? InternalSerializers.create(elementType) : serializer; + return (writer, pos, value) -> + writer.writeVec( + pos, (InternalVec) value, (InternalVecSerializer) vecSerializer); 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..dfbcabc317ce 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); + InternalVec getVec(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..1fc17f1c1751 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 InternalVec getVec(int pos) { + return (InternalVec) 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..9a674a184106 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 InternalVec getVec(int pos) { + return (InternalVec) 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..55d79d069930 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.getVec(fieldPos); + break; case MULTISET: case MAP: fieldGetter = row -> row.getMap(fieldPos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/InternalVec.java b/paimon-common/src/main/java/org/apache/paimon/data/InternalVec.java new file mode 100644 index 000000000000..fbf9be6fff45 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/InternalVec.java @@ -0,0 +1,89 @@ +/* + * 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.VecType; + +/** + * Base interface of an internal data structure representing data of {@link VecType}. + * + *

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 ArrayBasedVec + * @since 2.0.0 + */ +@Public +public interface InternalVec 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(), InternalVec.class.getName()); + throw new IllegalArgumentException(msg); + } + if (!elementType.isNullable()) { + return elementGetter; + } + return (vec, pos) -> { + if (vec.isNullAt(pos)) { + return null; + } + return elementGetter.getElementOrNull(vec, 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..e5e54a93548c 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 InternalVec getVec(int pos) { + if (pos < row1.getFieldCount()) { + return row1.getVec(pos); + } else { + return row2.getVec(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..8b747311fad5 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 InternalVec getVec(int pos) { + return (InternalVec) 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..1fb46491081f 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 InternalVec getVec(int pos) { + throw new IllegalArgumentException("Unsupported type: vecType"); + } + @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..5b34a3e4c79b 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.InternalVec; 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 InternalVec getVec(int pos) { + throw new UnsupportedOperationException("Unsupported type: vecType"); + } + @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..79f550ff615a 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.InternalVec; 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 InternalVec getVec(int pos) { + return vectorizedColumnBatch.getVec(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..58122f09744d 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.VecType; import java.io.Serializable; import java.util.List; @@ -281,6 +282,11 @@ public TypeConverter visit(ArrayType arrayType) { }); } + @Override + public TypeConverter visit(VecType vecType) { + 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..4d51bfa3d13b 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.InternalVec; 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 InternalVec getVec(int rowId, int colId) { + throw new UnsupportedOperationException("Unsupported type: vecType"); + } + 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..b6123617e15a 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.InternalVec; 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 InternalVec getVec(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..e47fb3b10752 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.InternalVec; 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 InternalVec getVec(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..7a620cf4daf4 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.VecType; import static org.apache.paimon.types.DataTypeChecks.getFieldTypes; import static org.apache.paimon.types.DataTypeChecks.getPrecision; @@ -75,6 +76,9 @@ private static Serializer createInternal(DataType type) { return new TimestampSerializer(getPrecision(type)); case ARRAY: return new InternalArraySerializer(((ArrayType) type).getElementType()); + case VECTOR: + VecType vecType = (VecType) type; + return new InternalVecSerializer(vecType.getElementType(), vecType.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/InternalVecSerializer.java b/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalVecSerializer.java new file mode 100644 index 000000000000..ae5ccec62fa3 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalVecSerializer.java @@ -0,0 +1,120 @@ +/* + * 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.ArrayBasedVec; +import org.apache.paimon.data.BinaryArray; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalVec; +import org.apache.paimon.io.DataInputView; +import org.apache.paimon.io.DataOutputView; +import org.apache.paimon.types.DataType; + +import java.io.IOException; + +/** Serializer for {@link InternalVec}. */ +public class InternalVecSerializer implements Serializer { + private static final long serialVersionUID = 1L; + + private final DataType eleType; + private final int length; + private final InternalArraySerializer arraySer; + private final Serializer eleSer; + + public InternalVecSerializer(DataType eleType, int length) { + this( + eleType, + length, + new InternalArraySerializer(eleType), + InternalSerializers.create(eleType)); + } + + private InternalVecSerializer( + DataType eleType, + int len, + InternalArraySerializer arraySer, + Serializer eleSer) { + this.eleType = eleType; + this.length = len; + this.arraySer = arraySer; + this.eleSer = eleSer; + } + + @Override + public InternalVecSerializer duplicate() { + return new InternalVecSerializer(eleType, length, arraySer.duplicate(), eleSer.duplicate()); + } + + @Override + public InternalVec copy(InternalVec from) { + if (from instanceof ArrayBasedVec) { + return ArrayBasedVec.from(arraySer.copy(((ArrayBasedVec) from).getInnerArray())); + } else { + return ArrayBasedVec.from(arraySer.copy(from)); + } + } + + @Override + public void serialize(InternalVec record, DataOutputView target) throws IOException { + if (record.size() != length) { + throw new IOException("Invalid size to serialize: " + record.size()); + } + if (record instanceof ArrayBasedVec) { + arraySer.serialize(((ArrayBasedVec) record).getInnerArray(), target); + } else { + arraySer.serialize(record, target); + } + } + + @Override + public InternalVec deserialize(DataInputView source) throws IOException { + InternalArray array = arraySer.deserialize(source); + if (array.size() != length) { + throw new IOException("Invalid size to deserialize: " + array.size()); + } + return ArrayBasedVec.from(array); + } + + public BinaryArray toBinaryArray(InternalVec from) { + if (from instanceof ArrayBasedVec) { + return arraySer.toBinaryArray(((ArrayBasedVec) from).getInnerArray()); + } else { + return arraySer.toBinaryArray(from); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + InternalVecSerializer that = (InternalVecSerializer) 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..8d987e7e3ef3 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.VecType; /** 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(VecType vecType) { + 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..ea26bdd4b8b6 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.VecType; 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(VecType vecType) { + 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/reader/DataEvolutionArray.java b/paimon-common/src/main/java/org/apache/paimon/reader/DataEvolutionArray.java index 7ebccba6b2c8..39a11d6ed58d 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.InternalVec; 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 InternalVec getVec(int pos) { + return chooseArray(pos).getVec(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..fb4b61b40bb7 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.InternalVec; 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 InternalVec getVec(int pos) { + return chooseRow(pos).getVec(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..4ddfce17db48 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.VecType; 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(VecType vecType) { + 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..eebf52f04c65 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.VecType; 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(VecType vecType) { + 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..3553ab22876a 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.InternalVec; import java.util.List; import java.util.function.BiFunction; @@ -253,6 +254,28 @@ public BiFunction visit(ArrayType arrayType) { }; } + @Override + public BiFunction visit(VecType vecType) { + 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 = + vecType.getElementType().accept(this); + InternalVec internalVec = row.getVec(index); + + int size = 0; + for (int i = 0; i < internalVec.size(); i++) { + size += function.apply(internalVec, 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..1b6a63409b98 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.getVec(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..2aee51debb77 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.InternalVec; 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 InternalVec getVec(int pos) { + return row.getVec(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..d15cd1669fbb 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.InternalVec; 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 InternalVec getVec(int pos) { + return array.getVec(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 f9b78a8bc676..e4f585910d59 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.InternalVec; 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 InternalVec getVec(int pos) { + return row.getVec(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/ProjectedVec.java b/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedVec.java new file mode 100644 index 000000000000..ae844a14afc0 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/utils/ProjectedVec.java @@ -0,0 +1,127 @@ +/* + * 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.InternalVec; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.data.variant.Variant; +import org.apache.paimon.types.DataType; + +/** + * An implementation of {@link InternalVec} which provides a projected view of the underlying {@link + * InternalVec}. + * + *

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

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

This method replaces the row data in place and does not return a new object. This is done + * for performance reasons. + */ + public ProjectedVec replaceVec(InternalVec vec) { + super.replaceArray(vec); + return this; + } + + @Override + public ProjectedArray replaceArray(InternalArray array) { + throw new IllegalArgumentException("ProjectedVec does not support replaceArray."); + } + + // --------------------------------------------------------------------------------------------- + + @Override + public BinaryString getString(int pos) { + throw new UnsupportedOperationException("ProjectedVec does not support String."); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + throw new UnsupportedOperationException("ProjectedVec does not support Decimal."); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + throw new UnsupportedOperationException("ProjectedVec does not support Timestamp."); + } + + @Override + public byte[] getBinary(int pos) { + throw new UnsupportedOperationException("ProjectedVec does not support Binary."); + } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("ProjectedVec does not support Variant."); + } + + @Override + public Blob getBlob(int pos) { + throw new UnsupportedOperationException("ProjectedVec does not support Blob."); + } + + @Override + public InternalArray getArray(int pos) { + throw new UnsupportedOperationException("ProjectedVec does not support nested Array."); + } + + @Override + public InternalVec getVec(int pos) { + throw new UnsupportedOperationException("ProjectedVec does not support nested VecType."); + } + + @Override + public InternalMap getMap(int pos) { + throw new UnsupportedOperationException("ProjectedVec does not support Map."); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + throw new UnsupportedOperationException("ProjectedVec does not support nested Row."); + } + + /** + * Create an empty {@link ProjectedVec} 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 ProjectedVec + */ + public static ProjectedVec from(int[] projection) { + return new ProjectedVec(projection); + } +} 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..c5533e6396d9 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.VecType; /** * 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(VecType vecType) { + throw new UnsupportedOperationException("VecType is not supported."); + } + @Override public ColumnVector visit(MultisetType multisetType) { return new MapColumnVector() { 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..c886cd9a8d85 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.VecType; 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(VecType vecType) { + throw new RuntimeException("RandomGenerator for VecType has not been implemented yet."); + } + @Override public DataGeneratorContainer visit(MultisetType multisetType) { ConfigOption lenOption = 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..c984cdf21980 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.InternalVec; 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 InternalVec getVec(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..1a20a7ae40bb 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.InternalVec; 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 InternalVec getVec(int pos) { + return row.getVec(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..2ce6fe101877 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.InternalVec; 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 InternalVec getVec(int pos) { + return row.getVec(pos); + } + @Override public InternalMap getMap(int pos) { return row.getMap(pos); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/VecTypeTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/VecTypeTableTest.java new file mode 100644 index 000000000000..28ba98855f51 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/append/VecTypeTableTest.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.ArrayBasedVec; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericArray; +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 VecTypeTableTest extends TableTestBase { + + private final float[] testVec = randomVec(); + + @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.getVec(2).toFloatArray(), testVec, 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(testVec.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()), + ArrayBasedVec.from(new GenericArray(testVec))); + } + + @Override + protected byte[] randomBytes() { + byte[] binary = new byte[RANDOM.nextInt(1024) + 1]; + RANDOM.nextBytes(binary); + return binary; + } + + private float[] randomVec() { + byte[] randomBytes = randomBytes(); + float[] vec = new float[randomBytes.length]; + for (int i = 0; i < vec.length; i++) { + vec[i] = randomBytes[i]; + } + return vec; + } +} 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..b9b12e532aab 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.VecType; 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(VecType vecType) { + throw new UnsupportedOperationException("Not support vecType 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..121ec9ab9721 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.InternalVec; 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 InternalVec getVec(int pos) { + throw new UnsupportedOperationException("Not support vecType 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 InternalVec getVec(int pos) { + throw new UnsupportedOperationException("Not support vecType 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..7722802f757b 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 @@ -20,6 +20,7 @@ import org.apache.paimon.casting.CastExecutor; import org.apache.paimon.casting.CastExecutors; +import org.apache.paimon.data.ArrayBasedVec; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericArray; import org.apache.paimon.data.GenericMap; @@ -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.VecType; 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 convertJsonVec(node, (VecType) dataType, options); case MAP: return convertJsonMap(node, (MapType) dataType, options); case ROW: @@ -136,6 +140,12 @@ private GenericArray convertJsonArray( return new GenericArray(elements.toArray()); } + private ArrayBasedVec convertJsonVec(JsonNode vecNode, VecType vecType, JsonOptions options) { + ArrayType arrayType = DataTypes.ARRAY(vecType.getElementType()); + GenericArray array = convertJsonArray(vecNode, arrayType, options); + return ArrayBasedVec.from(array); + } + 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..bdd7999faf20 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.InternalVec; 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.VecType; 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 convertRowVec((InternalVec) value, (VecType) 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 convertRowVec(InternalVec vec, VecType vecType) { + if (vec.size() != vecType.getLength()) { + throw new IllegalArgumentException( + "Size " + vec.size() + " != " + vecType.getLength() + " in JsonWriter"); + } + return convertRowArray(vec, DataTypes.ARRAY(vecType.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..1f3b1faf84e9 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.VecType; 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(VecType vecType) { + throw new UnsupportedOperationException("Unsupported type: " + vecType); + } + @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..b7abf2492c5e 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.VecType; 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(VecType vecType) { + 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..b02ff06b784d 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.VecType; 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(VecType vecType) { + throw new UnsupportedOperationException(); + } + @Override public Operators.Column visit(MultisetType multisetType) { throw new UnsupportedOperationException(); 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..5f8aca13bf8d 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.InternalVec; 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 InternalVec getVec(int i) { + throw new UnsupportedOperationException("Not support vecType 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..4e5aaa30470c 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.VecType; import javax.annotation.Nullable; @@ -192,6 +193,11 @@ public Void visit(ArrayType arrayType) { return null; } + @Override + public Void visit(VecType vecType) { + 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 7de1695af0be..72d317a4ff99 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 @@ -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.InternalVec; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.spark.util.shim.TypeUtils$; @@ -258,6 +259,11 @@ public InternalArray getArray(int pos) { ((ArrayType) (tableSchema.fields()[pos].dataType())).elementType()); } + @Override + public InternalVec getVec(int pos) { + throw new UnsupportedOperationException("Not support vecType yet."); + } + @Override public InternalMap getMap(int pos) { int actualPos = getActualFieldPosition(pos); @@ -425,6 +431,11 @@ public InternalArray getArray(int pos) { arrayData.getArray(pos), ((ArrayType) elementType).elementType()); } + @Override + public InternalVec getVec(int pos) { + throw new UnsupportedOperationException("Not support vecType 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 d06574376883..1389a2ef1abe 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.InternalVec; 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 InternalVec getVec(int pos) { + throw new UnsupportedOperationException("Not support vecType 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 InternalVec getVec(int pos) { + throw new UnsupportedOperationException("Not support vecType yet."); + } + @Override public InternalMap getMap(int i) { Object o = getAs(i); From 3afa4a77d734b4ba8d094faaafc9bb47ed4bf4bb Mon Sep 17 00:00:00 2001 From: ColdL Date: Mon, 12 Jan 2026 17:55:39 +0800 Subject: [PATCH 2/4] add columnar impl for InternalVec and link to arrow module --- .../org/apache/paimon/arrow/ArrowUtils.java | 15 +- .../Arrow2PaimonVectorConverter.java | 48 +++- .../ArrowFieldWriterFactoryVisitor.java | 10 +- .../arrow/writer/ArrowFieldWriters.java | 90 ++++++ .../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 +- .../format/lance/VecTypeWithLanceTest.java | 147 ++++++++++ 10 files changed, 620 insertions(+), 11 deletions(-) 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-lance/src/test/java/org/apache/paimon/format/lance/VecTypeWithLanceTest.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 952223b6c910..d4c59d47b126 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 @@ -28,6 +28,7 @@ import org.apache.paimon.types.DataType; import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VecType; import org.apache.arrow.c.ArrowArray; import org.apache.arrow.c.ArrowSchema; @@ -135,14 +136,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 VecType) { + final DataType elementType; + if (dataType instanceof VecType) { + elementType = ((VecType) 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 a3289c526067..b77218b83536 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.InternalVec; 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(VecType vecType) { - throw new UnsupportedOperationException("Doesn't support VecType."); + final Arrow2PaimonVectorConverter arrowVectorConvertor = + vecType.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 InternalVec getVec(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 getVecSize() { + 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 fce2c58d09e9..5cfd095606d3 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.VecType; 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; @@ -167,7 +168,14 @@ public ArrowFieldWriterFactory visit(ArrayType arrayType) { @Override public ArrowFieldWriterFactory visit(VecType vecType) { - throw new UnsupportedOperationException("Doesn't support VecType."); + ArrowFieldWriterFactory elementWriterFactory = vecType.getElementType().accept(this); + return (fieldVector, isNullable) -> + new ArrowFieldWriters.VecWriter( + fieldVector, + vecType.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 9e4f371a793d..1c907919445f 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 @@ -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.InternalVec; import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.columnar.ArrayColumnVector; import org.apache.paimon.data.columnar.BooleanColumnVector; @@ -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.memory.MemorySegment; import org.apache.paimon.utils.IntArrayList; @@ -57,6 +59,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; @@ -594,6 +597,93 @@ protected void doWrite(int rowIndex, DataGetters getters, int pos) { } } + /** Writer for VECTOR. */ + public static class VecWriter extends ArrowFieldWriter { + + private final ArrowFieldWriter elementWriter; + + private final int length; + + public VecWriter( + 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) { + InternalVec vec = getters.getVec(pos); + if (vec.size() != length) { + throw new IllegalArgumentException( + String.format( + "The size of vec %s is not equal to the length %s", + vec.size(), length)); + } + FixedSizeListVector listVector = (FixedSizeListVector) fieldVector; + listVector.setNotNull(rowIndex); + final int rowBase = rowIndex * length; + for (int vecIndex = 0; vecIndex < length; ++vecIndex) { + elementWriter.write(rowBase + vecIndex, vec, vecIndex); + } + // 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-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 5b34a3e4c79b..abc91b55ddd8 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 InternalVec getVec(int pos) { - throw new UnsupportedOperationException("Unsupported type: vecType"); + return ((VecColumnVector) data).getVec(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..e48c1810e1af --- /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.InternalVec; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.data.variant.Variant; + +import java.io.Serializable; + +/** Columnar vecType to support access to vector column data. */ +public final class ColumnarVec implements InternalVec, 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 InternalVec getVec(int pos) { + return ((VecColumnVector) data).getVec(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..ccbd5e3e425f --- /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.InternalVec; + +/** Column vector for VecType. */ +public interface VecColumnVector extends ColumnVector { + InternalVec getVec(int i); + + ColumnVector getColumnVector(); + + int getVecSize(); +} 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 4d51bfa3d13b..456b95bba938 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 InternalVec getVec(int rowId, int colId) { - throw new UnsupportedOperationException("Unsupported type: vecType"); + return ((VecColumnVector) columns[colId]).getVec(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 c5533e6396d9..73cefc2612cd 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.InternalVec; 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(VecType vecType) { - throw new UnsupportedOperationException("VecType is not supported."); + return new VecColumnVector() { + @Override + public InternalVec getVec(int i) { + return partition.getVec(index); + } + + @Override + public boolean isNullAt(int i) { + return partition.isNullAt(index); + } + + @Override + public int getVecSize() { + return partition.getVec(index).size(); + } + + @Override + public ColumnVector getColumnVector() { + throw new UnsupportedOperationException( + "Doesn't support getting ColumnVector."); + } + }; } @Override diff --git a/paimon-lance/src/test/java/org/apache/paimon/format/lance/VecTypeWithLanceTest.java b/paimon-lance/src/test/java/org/apache/paimon/format/lance/VecTypeWithLanceTest.java new file mode 100644 index 000000000000..091cd891be02 --- /dev/null +++ b/paimon-lance/src/test/java/org/apache/paimon/format/lance/VecTypeWithLanceTest.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.ArrayBasedVec; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericArray; +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 VecTypeWithLanceTest extends TableTestBase { + + private final float[] testVec = randomVec(); + + @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.getVec(2).toFloatArray(), testVec, 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(testVec.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()), + ArrayBasedVec.from(new GenericArray(testVec))); + } + + @Override + protected byte[] randomBytes() { + byte[] binary = new byte[RANDOM.nextInt(1024) + 1]; + RANDOM.nextBytes(binary); + return binary; + } + + private float[] randomVec() { + byte[] randomBytes = randomBytes(); + float[] vec = new float[randomBytes.length]; + for (int i = 0; i < vec.length; i++) { + vec[i] = randomBytes[i]; + } + return vec; + } + + private void checkFileByLanceReader(String path) throws Exception { + ArrowType expected = new ArrowType.FixedSizeList(testVec.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 c4c39ee4d185a8707dcdefd682cfc457fcba46d1 Mon Sep 17 00:00:00 2001 From: ColdL Date: Mon, 12 Jan 2026 17:56:05 +0800 Subject: [PATCH 3/4] add flink support for VecType --- .../paimon/flink/DataTypeToLogicalType.java | 3 +- .../org/apache/paimon/flink/FlinkCatalog.java | 25 +++- .../apache/paimon/flink/FlinkRowWrapper.java | 10 +- .../paimon/flink/LogicalTypeConversion.java | 12 ++ .../paimon/flink/VecTypeTableITCase.java | 122 ++++++++++++++++++ 5 files changed, 166 insertions(+), 6 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/VecTypeTableITCase.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 b9b12e532aab..174ed8ce4fd6 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(VecType vecType) { - throw new UnsupportedOperationException("Not support vecType yet."); + return new org.apache.flink.table.types.logical.ArrayType( + vecType.isNullable(), vecType.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 8bba96cf78d8..5031a263760f 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.toVecType; 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(), - field.getName().equals(blobName) - ? 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) { + String blobName = options.get(CoreOptions.BLOB_FIELD.key()); + if (fieldName.equals(blobName)) { + return toBlobType(logicalType); + } + if (logicalType instanceof org.apache.flink.table.types.logical.ArrayType) { + String vecType = options.get(String.format("field.%s.vector-store.enabled", fieldName)); + if (Boolean.parseBoolean(vecType)) { + org.apache.flink.table.types.logical.LogicalType elementType = + ((org.apache.flink.table.types.logical.ArrayType) logicalType) + .getElementType(); + return toVecType(elementType, fieldName, options); + } + } + 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 121ec9ab9721..2149999389ff 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 InternalVec getVec(int pos) { - throw new UnsupportedOperationException("Not support vecType yet."); + return new FlinkVecWrapper(row.getArray(pos)); } @Override @@ -268,7 +268,7 @@ public InternalArray getArray(int pos) { @Override public InternalVec getVec(int pos) { - throw new UnsupportedOperationException("Not support vecType yet."); + return new FlinkVecWrapper(array.getArray(pos)); } @Override @@ -317,6 +317,12 @@ public double[] toDoubleArray() { } } + private static class FlinkVecWrapper extends FlinkArrayWrapper implements InternalVec { + private FlinkVecWrapper(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..e321529ceb02 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,15 +20,19 @@ 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.VecType; import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.VarBinaryType; +import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkNotNull; /** Conversion between {@link LogicalType} and {@link DataType}. */ public class LogicalTypeConversion { @@ -49,6 +53,14 @@ public static BlobType toBlobType(LogicalType logicalType) { return new BlobType(); } + public static VecType toVecType( + LogicalType elementType, String fieldName, Map options) { + String vecDim = options.get(String.format("field.%s.vector-dim", fieldName)); + checkNotNull(vecDim, "Absent 'field." + fieldName + ".vector-dim' for vector type"); + int dim = Integer.parseInt(vecDim); + return DataTypes.VECTOR(dim, toDataType(elementType)); + } + 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/VecTypeTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/VecTypeTableITCase.java new file mode 100644 index 000000000000..1ad127872e50 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/VecTypeTableITCase.java @@ -0,0 +1,122 @@ +/* + * 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 VecTypeTableITCase extends CatalogITCaseBase { + + private static final Random RANDOM = new Random(); + + private final String testTblName = "vec_table"; + + private final float[] testVec = randomVec(); + + @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(testVec), row.getFieldAs("embed")); + + checkTableSchema(); + checkInnerData(); + } + + private void checkTableSchema() throws Exception { + DataType vecType = DataTypes.VECTOR(testVec.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(vecType, 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(testVec, row.getVec(2).toFloatArray()); + } + + private String getCreateTableDdl() { + return String.format( + "CREATE TABLE IF NOT EXISTS `%s` (" + + " `id` INT," + + " `data` STRING," + + " `embed` ARRAY" + + ") WITH (" + + " 'file.format' = 'json'," + + " 'file.compression' = 'none'," + + " 'field.embed.vector-store.enabled' = 'true'," + + " 'field.embed.vector-dim' = '%d'" + + ")", + testTblName, testVec.length); + } + + private String makeValueStr() { + return String.format("(%d, '%s', ARRAY%s)", 1, "paimon", Arrays.toString(testVec)); + } + + private float[] randomVec() { + byte[] randomBytes = new byte[RANDOM.nextInt(1024) + 1]; + RANDOM.nextBytes(randomBytes); + float[] vec = new float[randomBytes.length]; + for (int i = 0; i < vec.length; i++) { + vec[i] = randomBytes[i]; + } + return vec; + } +} From c0ba527f4c15c309027237525d8d7a08f1a14f11 Mon Sep 17 00:00:00 2001 From: ColdL Date: Tue, 13 Jan 2026 16:20:59 +0800 Subject: [PATCH 4/4] fix: add VecType to codegen --- .../org/apache/paimon/codegen/GenerateUtils.scala | 15 ++++++++++----- .../paimon/codegen/ScalarOperatorGens.scala | 11 +++++++++-- .../codegen/EqualiserCodeGeneratorTest.java | 11 +++++++++++ .../java/org/apache/paimon/data/BinaryRow.java | 4 +++- .../java/org/apache/paimon/data/BinaryWriter.java | 3 +++ .../org/apache/paimon/utils/TypeCheckUtils.java | 7 +++++++ .../java/org/apache/paimon/utils/TypeUtils.java | 15 +++++++++++++++ .../apache/paimon/codegen/CodeGenUtilsTest.java | 9 +++++++++ 8 files changed, 67 insertions(+), 8 deletions(-) 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..4d8419182379 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[VecType].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[InternalVec] 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.getVec($indexTerm)" case MULTISET | MAP => s"$rowTerm.getMap($indexTerm)" case ROW => 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..8e28c08f502b 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 (isVec(left.resultType) && canEqual) { + val elementType = left.resultType.asInstanceOf[VecType].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 BinaryVec 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..fbf4efb3b4e6 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.InternalVecSerializer; 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 InternalVecSerializer(DataTypes.FLOAT(), 3))); TEST_DATA.put( DataTypeRoot.MULTISET, new GeneratedData( 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 3d819c22c19e..a0ebfaab8828 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 @@ -357,7 +357,9 @@ public InternalArray getArray(int pos) { @Override public InternalVec getVec(int pos) { - throw new IllegalArgumentException("Unsupported type: vecType"); + assertIndexIsValid(pos); + InternalArray array = MemorySegmentUtils.readArrayData(segments, offset, getLong(pos)); + return ArrayBasedVec.from(array); } @Override 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 fa926a702698..a0099a68e217 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 @@ -136,6 +136,9 @@ static void write( case ARRAY: writer.writeArray(pos, (InternalArray) o, (InternalArraySerializer) serializer); break; + case VECTOR: + writer.writeVec(pos, (InternalVec) o, (InternalVecSerializer) serializer); + break; case MAP: case MULTISET: writer.writeMap(pos, (InternalMap) o, (InternalMapSerializer) serializer); 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..81617cffa278 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 isVec(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) + && !isVec(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..ed956777040f 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 @@ -18,22 +18,26 @@ package org.apache.paimon.utils; +import org.apache.paimon.data.ArrayBasedVec; import org.apache.paimon.data.BinaryString; 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.VecType; 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,16 @@ 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: + VecType vecType = (VecType) type; + DataType vecElementType = vecType.getElementType(); + Object vecBaseArr = + castFromStringInternal(s, DataTypes.ARRAY(vecElementType), isCdcValue); + if (vecBaseArr instanceof InternalArray) { + return ArrayBasedVec.from((InternalArray) vecBaseArr); + } else { + throw new RuntimeException("Failed to make array during building a vector"); + } case MAP: MapType mapType = (MapType) type; DataType keyType = mapType.getKeyType(); @@ -333,6 +347,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-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java b/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java index c0da56f54454..27c26a72995e 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 @@ -32,6 +32,7 @@ import static org.apache.paimon.types.DataTypes.DOUBLE; 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 { @@ -74,6 +75,14 @@ public void testRecordComparatorCodegenCache() { () -> newRecordComparator(Arrays.asList(STRING(), INT()), new int[] {0, 1}, true)); } + @Test + public void testRecordComparatorCodegenCacheWithVec() { + assertClassEquals( + () -> + newRecordComparator( + Arrays.asList(STRING(), VECTOR(3, INT())), new int[] {0, 1}, true)); + } + @Test public void testRecordComparatorCodegenCacheMiss() { assertClassNotEquals(