From a6f37fe65871a5ad9ef325099582ff83db6a1af4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Thu, 16 Apr 2026 15:43:44 +0800 Subject: [PATCH 01/19] Introduce Blob Ref --- .../java/org/apache/paimon/CoreOptions.java | 25 +- .../org/apache/paimon/types/BlobRefType.java | 65 +++++ .../paimon/types/DataTypeDefaultVisitor.java | 5 + .../paimon/types/DataTypeJsonParser.java | 3 + .../org/apache/paimon/types/DataTypeRoot.java | 2 + .../apache/paimon/types/DataTypeVisitor.java | 2 + .../org/apache/paimon/types/DataTypes.java | 4 + .../arrow/ArrowFieldTypeConversion.java | 6 + .../Arrow2PaimonVectorConverter.java | 6 + .../ArrowFieldWriterFactoryVisitor.java | 6 + .../org/apache/paimon/data/BinaryWriter.java | 9 + .../java/org/apache/paimon/data/Blob.java | 4 + .../org/apache/paimon/data/BlobReference.java | 145 ++++++++++ .../paimon/data/BlobReferenceResolver.java | 28 ++ .../org/apache/paimon/data/BlobUtils.java | 65 +++++ .../org/apache/paimon/data/InternalRow.java | 4 + .../apache/paimon/data/UnresolvedBlob.java | 75 +++++ .../paimon/data/columnar/ColumnarRow.java | 12 +- .../data/columnar/RowToColumnConverter.java | 6 + .../data/serializer/BlobRefSerializer.java | 52 ++++ .../data/serializer/InternalSerializers.java | 2 + .../fileindex/bitmap/BitmapTypeVisitor.java | 6 + .../fileindex/bloomfilter/FastHash.java | 6 + .../paimon/sort/hilbert/HilbertIndexer.java | 6 + .../apache/paimon/sort/zorder/ZIndexer.java | 6 + .../types/InternalRowToSizeVisitor.java | 17 +- .../paimon/utils/VectorMappingUtils.java | 6 + .../apache/paimon/data/BlobReferenceTest.java | 71 +++++ .../types/InternalRowToSizeVisitorTest.java | 19 ++ .../apache/paimon/io/RowDataFileWriter.java | 10 +- .../paimon/schema/SchemaValidation.java | 55 +++- .../table/AppendOnlyFileStoreTable.java | 3 +- .../table/PrimaryKeyFileStoreTable.java | 5 +- .../table/source/AbstractDataTableRead.java | 72 ++++- .../paimon/table/source/AppendTableRead.java | 6 +- .../table/source/BlobRefResolvingRow.java | 161 +++++++++++ .../table/source/KeyValueTableRead.java | 6 +- .../paimon/utils/BlobReferenceLookup.java | 254 +++++++++++++++++ .../apache/paimon/append/BlobTableTest.java | 109 ++++++++ .../paimon/io/RowDataFileWriterTest.java | 93 +++++++ .../schema/BlobRefSchemaValidationTest.java | 69 +++++ .../paimon/utils/BlobReferenceLookupTest.java | 262 ++++++++++++++++++ .../paimon/flink/DataTypeToLogicalType.java | 6 + .../org/apache/paimon/flink/FlinkCatalog.java | 15 + .../apache/paimon/flink/FlinkRowWrapper.java | 17 +- .../paimon/flink/LogicalTypeConversion.java | 8 + .../flink/lookup/LookupCompactDiffRead.java | 6 +- .../flink/lookup/LookupFileStoreTable.java | 4 +- .../source/FileStoreSourceSplitReader.java | 3 +- .../source/TestChangelogDataReadWrite.java | 2 +- .../format/avro/AvroSchemaConverter.java | 1 + .../format/avro/FieldReaderFactory.java | 11 +- .../format/avro/FieldWriterFactory.java | 21 +- .../apache/paimon/format/orc/OrcTypeUtil.java | 1 + .../format/orc/writer/FieldWriterFactory.java | 13 + .../parquet/ParquetSchemaConverter.java | 1 + .../parquet/reader/ParquetReaderUtil.java | 4 +- .../reader/ParquetVectorUpdaterFactory.java | 6 + .../parquet/writer/ParquetRowDataWriter.java | 22 ++ .../filter2/predicate/ParquetFilters.java | 6 + .../format/avro/AvroFileFormatTest.java | 28 ++ .../paimon/format/orc/OrcTypeUtilTest.java | 1 + .../parquet/ParquetSchemaConverterTest.java | 12 + .../org/apache/paimon/hive/HiveTypeUtils.java | 6 + .../paimon/format/lance/LanceFileFormat.java | 6 + .../org/apache/paimon/spark/SparkCatalog.java | 7 + .../paimon/spark/SparkInternalRowWrapper.java | 14 +- .../org/apache/paimon/spark/SparkRow.java | 16 +- .../apache/paimon/spark/SparkTypeUtils.java | 6 + .../paimon/spark/data/SparkInternalRow.scala | 5 +- .../format/vortex/VortexFileFormat.java | 7 + 71 files changed, 1945 insertions(+), 78 deletions(-) create mode 100644 paimon-api/src/main/java/org/apache/paimon/types/BlobRefType.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/BlobReference.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceResolver.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/serializer/BlobRefSerializer.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/data/BlobReferenceTest.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/schema/BlobRefSchemaValidationTest.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java diff --git a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java index 897bebcf686c..8b38b2d8ee0f 100644 --- a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java @@ -2254,6 +2254,14 @@ public InlineElement getDescription() { "Specifies column names that should be stored as blob type. " + "This is used when you want to treat a BYTES column as a BLOB."); + public static final ConfigOption BLOB_REF_FIELD = + key("blob-ref-field") + .stringType() + .noDefaultValue() + .withDescription( + "Specifies column names that should be stored as blob reference type. " + + "This is used when you want to treat a BYTES column as a BLOB_REF."); + @Immutable public static final ConfigOption BLOB_DESCRIPTOR_FIELD = key("blob-descriptor-field") @@ -2935,7 +2943,13 @@ public Set blobExternalStorageField() { * subset of descriptor fields and therefore are also updatable. */ public Set updatableBlobFields() { - return blobDescriptorField(); + Set fields = new HashSet<>(blobDescriptorField()); + fields.addAll(blobRefField()); + return fields; + } + + public Set blobRefField() { + return parseCommaSeparatedSet(BLOB_REF_FIELD); } /** @@ -3274,6 +3288,15 @@ public static List blobField(Map options) { return Arrays.stream(string.split(",")).map(String::trim).collect(Collectors.toList()); } + public static List blobRefField(Map options) { + String string = options.get(BLOB_REF_FIELD.key()); + if (string == null) { + return Collections.emptyList(); + } + + return Arrays.stream(string.split(",")).map(String::trim).collect(Collectors.toList()); + } + public boolean sequenceFieldSortOrderIsAscending() { return options.get(SEQUENCE_FIELD_SORT_ORDER) == SortOrder.ASCENDING; } diff --git a/paimon-api/src/main/java/org/apache/paimon/types/BlobRefType.java b/paimon-api/src/main/java/org/apache/paimon/types/BlobRefType.java new file mode 100644 index 000000000000..e5d53ead4e4a --- /dev/null +++ b/paimon-api/src/main/java/org/apache/paimon/types/BlobRefType.java @@ -0,0 +1,65 @@ +/* + * 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; + +/** + * Data type of blob reference. + * + *

{@link BlobRefType} stores reference bytes inline in data files instead of writing payloads to + * Paimon-managed {@code .blob} files. + * + * @since 1.5.0 + */ +@Public +public final class BlobRefType extends DataType { + + private static final long serialVersionUID = 1L; + + private static final String FORMAT = "BLOB_REF"; + + public BlobRefType(boolean isNullable) { + super(isNullable, DataTypeRoot.BLOB_REF); + } + + public BlobRefType() { + this(true); + } + + @Override + public int defaultSize() { + return BlobType.DEFAULT_SIZE; + } + + @Override + public DataType copy(boolean isNullable) { + return new BlobRefType(isNullable); + } + + @Override + public String asSQLString() { + return withNullability(FORMAT); + } + + @Override + public R accept(DataTypeVisitor visitor) { + return visitor.visit(this); + } +} 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 af680ede62e2..4a819d42ae2c 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 @@ -119,6 +119,11 @@ public R visit(BlobType blobType) { return defaultMethod(blobType); } + @Override + public R visit(BlobRefType blobRefType) { + return defaultMethod(blobRefType); + } + @Override public R visit(ArrayType arrayType) { return defaultMethod(arrayType); 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 4079dd8c47c0..5e2a39a29fcd 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 @@ -331,6 +331,7 @@ private enum Keyword { LEGACY, VARIANT, BLOB, + BLOB_REF, NOT } @@ -549,6 +550,8 @@ private DataType parseTypeByKeyword() { return new VariantType(); case BLOB: return new BlobType(); + case BLOB_REF: + return new BlobRefType(); case VECTOR: return parseVectorType(); default: 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 f55da9c4706f..27f8d65a40bf 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 @@ -104,6 +104,8 @@ public enum DataTypeRoot { BLOB(DataTypeFamily.PREDEFINED), + BLOB_REF(DataTypeFamily.PREDEFINED), + ARRAY(DataTypeFamily.CONSTRUCTED, DataTypeFamily.COLLECTION), VECTOR(DataTypeFamily.CONSTRUCTED, DataTypeFamily.COLLECTION), 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 6e377309f237..074a1d82ec70 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 @@ -66,6 +66,8 @@ public interface DataTypeVisitor { R visit(BlobType blobType); + R visit(BlobRefType blobRefType); + R visit(ArrayType arrayType); R visit(VectorType vectorType); 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 39b180651ef5..0033984bc6cc 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 @@ -163,6 +163,10 @@ public static BlobType BLOB() { return new BlobType(); } + public static BlobRefType BLOB_REF() { + return new BlobRefType(); + } + public static OptionalInt getPrecision(DataType dataType) { return dataType.accept(PRECISION_EXTRACTOR); } 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 33defc8f9a01..37b36a24d154 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 @@ -21,6 +21,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -163,6 +164,11 @@ public FieldType visit(BlobType blobType) { throw new UnsupportedOperationException(); } + @Override + public FieldType visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException(); + } + private TimeUnit getTimeUnit(int precision) { if (precision == 0) { return TimeUnit.SECOND; 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 e1fe66883a84..d8672dfdc23b 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 @@ -47,6 +47,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -447,6 +448,11 @@ public Arrow2PaimonVectorConverter visit(BlobType blobType) { throw new UnsupportedOperationException(); } + @Override + public Arrow2PaimonVectorConverter visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException(); + } + @Override public Arrow2PaimonVectorConverter visit(ArrayType arrayType) { final Arrow2PaimonVectorConverter arrowVectorConvertor = 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 ccff6d6a24f6..b4d38e2dae61 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 @@ -21,6 +21,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -156,6 +157,11 @@ public ArrowFieldWriterFactory visit(BlobType blobType) { throw new UnsupportedOperationException("Doesn't support BlobType."); } + @Override + public ArrowFieldWriterFactory visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException("Doesn't support BlobRefType."); + } + @Override public ArrowFieldWriterFactory visit(ArrayType arrayType) { ArrowFieldWriterFactory elementWriterFactory = arrayType.getElementType().accept(this); 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 2e0cd5701b71..b22336e31b8d 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 @@ -157,6 +157,10 @@ static void write( case BLOB: writer.writeBlob(pos, (Blob) o); break; + case BLOB_REF: + byte[] refBytes = BlobUtils.serializeBlobReference((Blob) o); + writer.writeBinary(pos, refBytes, 0, refBytes.length); + break; default: throw new UnsupportedOperationException("Not support type: " + type); } @@ -241,6 +245,11 @@ static ValueSetter createValueSetter(DataType elementType, Serializer seriali return (writer, pos, value) -> writer.writeVariant(pos, (Variant) value); case BLOB: return (writer, pos, value) -> writer.writeBlob(pos, (Blob) value); + case BLOB_REF: + return (writer, pos, value) -> { + byte[] bytes = BlobUtils.serializeBlobReference((Blob) value); + writer.writeBinary(pos, bytes, 0, bytes.length); + }; default: String msg = String.format( diff --git a/paimon-common/src/main/java/org/apache/paimon/data/Blob.java b/paimon-common/src/main/java/org/apache/paimon/data/Blob.java index 6586124e466b..2ab095d23a9e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/Blob.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/Blob.java @@ -65,6 +65,10 @@ static Blob fromDescriptor(UriReader reader, BlobDescriptor descriptor) { return new BlobRef(reader, descriptor); } + static Blob fromReference(BlobReference reference) { + return new UnresolvedBlob(reference); + } + static Blob fromInputStream(Supplier supplier) { return new BlobStream(supplier); } diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobReference.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobReference.java new file mode 100644 index 000000000000..d4a7926bd514 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobReference.java @@ -0,0 +1,145 @@ +/* + * 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 java.io.Serializable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Objects; + +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Serialized metadata for a {@code BLOB_REF} field. + * + *

A blob reference only stores the coordinates needed to locate the original blob value in the + * upstream table: {@code tableName}, {@code fieldId} and {@code rowId}. The actual blob data is + * resolved at read time by scanning the upstream table. + * + *

Serialization layout (Little Endian): + * + *

+ * | Offset       | Field         | Type    | Size |
+ * |--------------|---------------|---------|------|
+ * | 0            | version       | byte    | 1    |
+ * | 1            | magicNumber   | long    | 8    |
+ * | 9            | tableNameLen  | int     | 4    |
+ * | 13           | tableNameBytes| byte[N] | N    |
+ * | 13 + N       | fieldId       | int     | 4    |
+ * | 17 + N       | rowId         | long    | 8    |
+ * 
+ */ +public class BlobReference implements Serializable { + + private static final long serialVersionUID = 1L; + + private static final long MAGIC = 0x424C4F4252454631L; // "BLOBREF1" + private static final byte CURRENT_VERSION = 1; + + private final String tableName; + private final int fieldId; + private final long rowId; + + public BlobReference(String tableName, int fieldId, long rowId) { + this.tableName = tableName; + this.fieldId = fieldId; + this.rowId = rowId; + } + + public String tableName() { + return tableName; + } + + public int fieldId() { + return fieldId; + } + + public long rowId() { + return rowId; + } + + public byte[] serialize() { + byte[] tableBytes = tableName.getBytes(UTF_8); + + int totalSize = 1 + 8 + 4 + tableBytes.length + 4 + 8; + ByteBuffer buffer = ByteBuffer.allocate(totalSize).order(ByteOrder.LITTLE_ENDIAN); + buffer.put(CURRENT_VERSION); + buffer.putLong(MAGIC); + buffer.putInt(tableBytes.length); + buffer.put(tableBytes); + buffer.putInt(fieldId); + buffer.putLong(rowId); + return buffer.array(); + } + + public static BlobReference deserialize(byte[] bytes) { + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + byte version = buffer.get(); + + if (version != CURRENT_VERSION) { + throw new UnsupportedOperationException( + "Expecting BlobReference version to be " + + CURRENT_VERSION + + ", but found " + + version + + "."); + } + + long magic = buffer.getLong(); + if (magic != MAGIC) { + throw new IllegalArgumentException( + "Invalid BlobReference: missing magic header. Expected magic: " + + MAGIC + + ", but found: " + + magic); + } + + byte[] tableBytes = new byte[buffer.getInt()]; + buffer.get(tableBytes); + + int fieldId = buffer.getInt(); + long rowId = buffer.getLong(); + return new BlobReference(new String(tableBytes, UTF_8), fieldId, rowId); + } + + public static boolean isBlobReference(byte[] bytes) { + if (bytes == null || bytes.length < 9) { + return false; + } + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + byte version = buffer.get(); + return version == CURRENT_VERSION && MAGIC == buffer.getLong(); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + BlobReference that = (BlobReference) o; + return fieldId == that.fieldId + && rowId == that.rowId + && Objects.equals(tableName, that.tableName); + } + + @Override + public int hashCode() { + return Objects.hash(tableName, fieldId, rowId); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceResolver.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceResolver.java new file mode 100644 index 000000000000..9e0d78485542 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceResolver.java @@ -0,0 +1,28 @@ +/* + * 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 java.io.Serializable; + +/** Resolves a {@link BlobReference} through fallback metadata. */ +@FunctionalInterface +public interface BlobReferenceResolver extends Serializable { + + Blob resolve(BlobReference reference); +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java new file mode 100644 index 000000000000..210c13e0f1d5 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java @@ -0,0 +1,65 @@ +/* + * 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.fs.FileIO; +import org.apache.paimon.utils.UriReader; +import org.apache.paimon.utils.UriReaderFactory; + +import javax.annotation.Nullable; + +/** Utilities for decoding and encoding blob-related bytes. */ +public class BlobUtils { + + public static Blob fromBytes( + byte[] bytes, + @Nullable UriReaderFactory uriReaderFactory, + @Nullable FileIO fileIO) { + if (bytes == null) { + return null; + } + + if (BlobReference.isBlobReference(bytes)) { + return new UnresolvedBlob(BlobReference.deserialize(bytes)); + } + + if (BlobDescriptor.isBlobDescriptor(bytes)) { + BlobDescriptor descriptor = BlobDescriptor.deserialize(bytes); + UriReader reader = + uriReaderFactory != null + ? uriReaderFactory.create(descriptor.uri()) + : UriReader.fromFile(fileIO); + return Blob.fromDescriptor(reader, descriptor); + } + + return new BlobData(bytes); + } + + public static byte[] serializeBlobReference(Blob blob) { + if (blob instanceof UnresolvedBlob) { + return ((UnresolvedBlob) blob).reference().serialize(); + } + throw new IllegalArgumentException( + "BLOB_REF fields only accept UnresolvedBlob values, but found " + + blob.getClass().getSimpleName() + + "."); + } + + private BlobUtils() {} +} 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 3bbb85f49963..f4e9e6960b6b 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 @@ -146,6 +146,9 @@ static Class getDataClass(DataType type) { case TIMESTAMP_WITHOUT_TIME_ZONE: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: return Timestamp.class; + case BLOB: + case BLOB_REF: + return Blob.class; case ARRAY: return InternalArray.class; case MULTISET: @@ -228,6 +231,7 @@ static FieldGetter createFieldGetter(DataType fieldType, int fieldPos) { fieldGetter = row -> row.getVariant(fieldPos); break; case BLOB: + case BLOB_REF: fieldGetter = row -> row.getBlob(fieldPos); break; default: diff --git a/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java b/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java new file mode 100644 index 000000000000..6ba949d92576 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java @@ -0,0 +1,75 @@ +/* + * 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.fs.SeekableInputStream; + +import java.io.Serializable; +import java.util.Objects; + +/** + * An unresolved {@link Blob} that only holds a {@link BlobReference}. It cannot provide data + * directly — callers must resolve it through a {@link BlobReferenceResolver} first. + */ +public class UnresolvedBlob implements Blob, Serializable { + + private static final long serialVersionUID = 1L; + + private final BlobReference reference; + + public UnresolvedBlob(BlobReference reference) { + this.reference = reference; + } + + public BlobReference reference() { + return reference; + } + + @Override + public byte[] toData() { + throw new IllegalStateException( + "UnresolvedBlob cannot provide data. Resolve it first via BlobReferenceResolver."); + } + + @Override + public BlobDescriptor toDescriptor() { + throw new IllegalStateException( + "UnresolvedBlob cannot provide descriptor. Resolve it first via BlobReferenceResolver."); + } + + @Override + public SeekableInputStream newInputStream() { + throw new IllegalStateException( + "UnresolvedBlob cannot provide stream. Resolve it first via BlobReferenceResolver."); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + UnresolvedBlob that = (UnresolvedBlob) o; + return Objects.equals(reference, that.reference); + } + + @Override + public int hashCode() { + return Objects.hash(reference); + } +} 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 13d345b1f03f..79a97915efe7 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 @@ -20,7 +20,7 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; -import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataSetters; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; @@ -31,7 +31,6 @@ import org.apache.paimon.data.variant.Variant; import org.apache.paimon.fs.FileIO; import org.apache.paimon.types.RowKind; -import org.apache.paimon.utils.UriReader; import java.io.Serializable; @@ -162,14 +161,7 @@ public Blob getBlob(int pos) { if (bytes == null) { return null; } - if (fileIO == null) { - throw new IllegalStateException("FileIO is null, cannot read blob data from uri!"); - } - - // Only blob descriptor could be able to stored in columnar format. - BlobDescriptor blobDescriptor = BlobDescriptor.deserialize(bytes); - UriReader uriReader = UriReader.fromFile(fileIO); - return Blob.fromDescriptor(uriReader, blobDescriptor); + return BlobUtils.fromBytes(bytes, null, fileIO); } @Override 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 de962ad86a39..12b7a567ec65 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 @@ -41,6 +41,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -262,6 +263,11 @@ public TypeConverter visit(BlobType blobType) { throw new UnsupportedOperationException(); } + @Override + public TypeConverter visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException(); + } + @Override public TypeConverter visit(ArrayType arrayType) { return createConverter( diff --git a/paimon-common/src/main/java/org/apache/paimon/data/serializer/BlobRefSerializer.java b/paimon-common/src/main/java/org/apache/paimon/data/serializer/BlobRefSerializer.java new file mode 100644 index 000000000000..91f88aae8e62 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/serializer/BlobRefSerializer.java @@ -0,0 +1,52 @@ +/* + * 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.Blob; +import org.apache.paimon.data.BlobReference; +import org.apache.paimon.data.BlobUtils; +import org.apache.paimon.data.UnresolvedBlob; +import org.apache.paimon.io.DataInputView; +import org.apache.paimon.io.DataOutputView; + +import java.io.IOException; + +/** Type serializer for {@code BLOB_REF}. */ +public class BlobRefSerializer extends SerializerSingleton { + + private static final long serialVersionUID = 1L; + + public static final BlobRefSerializer INSTANCE = new BlobRefSerializer(); + + @Override + public Blob copy(Blob from) { + return from; + } + + @Override + public void serialize(Blob blob, DataOutputView target) throws IOException { + BinarySerializer.INSTANCE.serialize(BlobUtils.serializeBlobReference(blob), target); + } + + @Override + public Blob deserialize(DataInputView source) throws IOException { + byte[] bytes = BinarySerializer.INSTANCE.deserialize(source); + return new UnresolvedBlob(BlobReference.deserialize(bytes)); + } +} 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 6669f347ff27..9d4c9dba1798 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 @@ -92,6 +92,8 @@ private static Serializer createInternal(DataType type) { return VariantSerializer.INSTANCE; case BLOB: return BlobSerializer.INSTANCE; + case BLOB_REF: + return BlobRefSerializer.INSTANCE; default: throw new UnsupportedOperationException( "Unsupported type '" + type + "' to get internal serializer"); 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 4183bfbb2bf8..57fcc8665b97 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 @@ -21,6 +21,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -176,4 +177,9 @@ public final R visit(VariantType rowType) { public final R visit(BlobType blobType) { throw new UnsupportedOperationException("Does not support type blob"); } + + @Override + public final R visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException("Does not support type blob ref"); + } } 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 322847f849ab..722ab63bc4f0 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 @@ -23,6 +23,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -172,6 +173,11 @@ public FastHash visit(BlobType blobType) { throw new UnsupportedOperationException("Does not support type blob"); } + @Override + public FastHash visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException("Does not support type blob_ref"); + } + @Override public FastHash visit(ArrayType arrayType) { throw new UnsupportedOperationException("Does not support type array"); 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 241dc6100379..254204dc2511 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 @@ -25,6 +25,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -272,6 +273,11 @@ public HProcessFunction visit(BlobType blobType) { throw new RuntimeException("Unsupported type"); } + @Override + public HProcessFunction visit(BlobRefType blobRefType) { + throw new RuntimeException("Unsupported type"); + } + @Override public HProcessFunction visit(ArrayType arrayType) { 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 1d40fe75e776..f95e767cb5ae 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 @@ -26,6 +26,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -360,6 +361,11 @@ public ZProcessFunction visit(BlobType blobType) { throw new UnsupportedOperationException("Does not support type blob"); } + @Override + public ZProcessFunction visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException("Does not support type blob_ref"); + } + @Override public ZProcessFunction visit(ArrayType arrayType) { 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 dbac55a07dde..94c71b6346ec 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 @@ -18,6 +18,7 @@ package org.apache.paimon.types; +import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataGetters; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; @@ -229,7 +230,21 @@ public BiFunction visit(BlobType blobType) { if (row.isNullAt(index)) { return NULL_SIZE; } else { - return Math.toIntExact(row.getVariant(index).sizeInBytes()); + return row.getBlob(index).toData().length; + } + }; + } + + @Override + public BiFunction visit(BlobRefType blobRefType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } + try { + return row.getBinary(index).length; + } catch (ClassCastException | UnsupportedOperationException e) { + return BlobUtils.serializeBlobReference(row.getBlob(index)).length; } }; } 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 99e8fd455c41..6ea9a0a7b52f 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 @@ -45,6 +45,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -336,6 +337,11 @@ public ColumnVector visit(BlobType blobType) { throw new UnsupportedOperationException("BlobType is not supported."); } + @Override + public ColumnVector visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException("BlobRefType is not supported."); + } + @Override public ColumnVector visit(ArrayType arrayType) { return new ArrayColumnVector() { diff --git a/paimon-common/src/test/java/org/apache/paimon/data/BlobReferenceTest.java b/paimon-common/src/test/java/org/apache/paimon/data/BlobReferenceTest.java new file mode 100644 index 000000000000..7bbb7965ba95 --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/data/BlobReferenceTest.java @@ -0,0 +1,71 @@ +/* + * 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.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link BlobReference}. */ +public class BlobReferenceTest { + + @Test + public void testSerializeAndDeserialize() { + BlobReference reference = new BlobReference("default.source", 7, 5L); + + BlobReference deserialized = BlobReference.deserialize(reference.serialize()); + + assertThat(deserialized.tableName()).isEqualTo("default.source"); + assertThat(deserialized.fieldId()).isEqualTo(7); + assertThat(deserialized.rowId()).isEqualTo(5L); + } + + @Test + public void testRejectUnexpectedVersion() { + BlobReference reference = new BlobReference("default.source", 7, 5L); + byte[] bytes = reference.serialize(); + bytes[0] = 3; + + assertThatThrownBy(() -> BlobReference.deserialize(bytes)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Expecting BlobReference version to be 1"); + } + + @Test + public void testEquality() { + BlobReference a = new BlobReference("default.source", 7, 5L); + BlobReference b = new BlobReference("default.source", 7, 5L); + BlobReference c = new BlobReference("default.source", 8, 5L); + + assertThat(a).isEqualTo(b); + assertThat(a.hashCode()).isEqualTo(b.hashCode()); + assertThat(a).isNotEqualTo(c); + } + + @Test + public void testIsBlobReference() { + BlobReference reference = new BlobReference("default.source", 7, 5L); + byte[] bytes = reference.serialize(); + + assertThat(BlobReference.isBlobReference(bytes)).isTrue(); + assertThat(BlobReference.isBlobReference(null)).isFalse(); + assertThat(BlobReference.isBlobReference(new byte[] {1, 2, 3})).isFalse(); + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java b/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java index cfdae649c190..728eb84e56e9 100644 --- a/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java @@ -36,6 +36,11 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + /** Test for InternalRowToSizeVisitor. */ public class InternalRowToSizeVisitorTest { @@ -192,4 +197,18 @@ void testCalculatorSize() { Assertions.assertThat(feildSizeCalculator.get(23).apply(row, 23)).isEqualTo(0); } + + @Test + void testBlobRefSizeUsesSerializedReferenceBytes() { + DataGetters row = mock(DataGetters.class); + byte[] referenceBytes = new byte[] {1, 2, 3, 4}; + when(row.isNullAt(0)).thenReturn(false); + when(row.getBinary(0)).thenReturn(referenceBytes); + + int size = new InternalRowToSizeVisitor().visit(DataTypes.BLOB_REF()).apply(row, 0); + + Assertions.assertThat(size).isEqualTo(referenceBytes.length); + verify(row).getBinary(0); + verify(row, never()).getBlob(0); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java index 7f8715ab0846..652017a85e2a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java @@ -32,7 +32,7 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.Collections; +import java.util.ArrayList; import java.util.List; import java.util.function.Function; import java.util.function.Supplier; @@ -106,6 +106,10 @@ public DataFileMeta result() throws IOException { ? DataFileIndexWriter.EMPTY_RESULT : dataFileIndexWriter.result(); String externalPath = isExternalPath ? path.toString() : null; + List extraFiles = new ArrayList<>(); + if (indexResult.independentIndexFile() != null) { + extraFiles.add(indexResult.independentIndexFile()); + } return DataFileMeta.forAppend( path.getName(), fileSize, @@ -114,9 +118,7 @@ public DataFileMeta result() throws IOException { seqNumCounter.getValue() - super.recordCount(), seqNumCounter.getValue() - 1, schemaId, - indexResult.independentIndexFile() == null - ? Collections.emptyList() - : Collections.singletonList(indexResult.independentIndexFile()), + extraFiles, indexResult.embeddedIndexBytes(), fileSource, statsPair.getKey(), diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java index 271709c47ef5..cab6a28d7fac 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java @@ -165,6 +165,7 @@ public static void validateTableSchema(TableSchema schema) { FileFormat fileFormat = FileFormat.fromIdentifier(options.formatType(), new Options(schema.options())); RowType tableRowType = new RowType(schema.fields()); + validateNestedBlobRefFields(tableRowType); Set blobDescriptorFields = validateBlobDescriptorFields(tableRowType, options); validateBlobExternalStorageFields(tableRowType, options, blobDescriptorFields); @@ -672,19 +673,22 @@ private static void validateRowTracking(TableSchema schema, CoreOptions options) List fields = schema.fields(); List blobNames = fields.stream() - .filter(field -> field.type().is(DataTypeRoot.BLOB)) + .filter( + field -> + field.type().is(DataTypeRoot.BLOB) + || field.type().is(DataTypeRoot.BLOB_REF)) .map(DataField::name) .collect(Collectors.toList()); if (!blobNames.isEmpty()) { checkArgument( options.dataEvolutionEnabled(), - "Data evolution config must enabled for table with BLOB type column."); + "Data evolution config must enabled for table with BLOB or BLOB_REF type column."); checkArgument( fields.size() > blobNames.size(), - "Table with BLOB type column must have other normal columns."); + "Table with BLOB or BLOB_REF type column must have other normal columns."); checkArgument( blobNames.stream().noneMatch(schema.partitionKeys()::contains), - "The BLOB type column can not be part of partition keys."); + "The BLOB or BLOB_REF type column can not be part of partition keys."); } FileFormat vectorFileFormat = vectorFileFormat(options); @@ -702,6 +706,49 @@ private static void validateRowTracking(TableSchema schema, CoreOptions options) } } + private static void validateNestedBlobRefFields(RowType rowType) { + for (DataField field : rowType.getFields()) { + checkArgument( + !containsNestedBlobRef(field.type()), + "Nested BLOB_REF type is not supported. Field '%s' contains a nested BLOB_REF.", + field.name()); + } + } + + private static boolean containsNestedBlobRef(DataType dataType) { + switch (dataType.getTypeRoot()) { + case ARRAY: + DataType arrayElementType = ((ArrayType) dataType).getElementType(); + return arrayElementType.is(DataTypeRoot.BLOB_REF) + || containsNestedBlobRef(arrayElementType); + case MULTISET: + DataType multisetElementType = ((MultisetType) dataType).getElementType(); + return multisetElementType.is(DataTypeRoot.BLOB_REF) + || containsNestedBlobRef(multisetElementType); + case MAP: + MapType mapType = (MapType) dataType; + return mapType.getKeyType().is(DataTypeRoot.BLOB_REF) + || containsNestedBlobRef(mapType.getKeyType()) + || mapType.getValueType().is(DataTypeRoot.BLOB_REF) + || containsNestedBlobRef(mapType.getValueType()); + case ROW: + for (DataField field : ((RowType) dataType).getFields()) { + if (field.type().is(DataTypeRoot.BLOB_REF) + || containsNestedBlobRef(field.type())) { + return true; + } + } + return false; + case VECTOR: + DataType vectorElementType = + ((org.apache.paimon.types.VectorType) dataType).getElementType(); + return vectorElementType.is(DataTypeRoot.BLOB_REF) + || containsNestedBlobRef(vectorElementType); + default: + return false; + } + } + private static Set validateBlobDescriptorFields(RowType rowType, CoreOptions options) { Set blobFieldNames = rowType.getFields().stream() diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java index 327810b881bc..8cb70824cd45 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java @@ -123,7 +123,8 @@ public InnerTableRead newRead() { new AppendTableRawFileSplitReadProvider( () -> store().newRead(), config)); } - return new AppendTableRead(providerFactories, schema()); + return new AppendTableRead( + providerFactories, schema(), catalogEnvironment().catalogContext()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java index a2fee49bfb88..411bcf6767de 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java @@ -147,7 +147,10 @@ protected BiConsumer nonPartitionFilterConsumer() { @Override public InnerTableRead newRead() { return new KeyValueTableRead( - () -> store().newRead(), () -> store().newBatchRawFileRead(), schema()); + () -> store().newRead(), + () -> store().newBatchRawFileRead(), + schema(), + catalogEnvironment().catalogContext()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java index ca5af88f40bc..20be987f3ab1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java @@ -18,20 +18,30 @@ package org.apache.paimon.table.source; +import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.TableQueryAuthResult; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobReference; +import org.apache.paimon.data.BlobReferenceResolver; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.UnresolvedBlob; import org.apache.paimon.disk.IOManager; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateProjectionConverter; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.types.DataTypeRoot; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BlobReferenceLookup; import org.apache.paimon.utils.ListUtils; import org.apache.paimon.utils.ProjectedRow; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Optional; import java.util.Set; @@ -45,9 +55,11 @@ public abstract class AbstractDataTableRead implements InnerTableRead { private boolean executeFilter = false; private Predicate predicate; private final TableSchema schema; + private final CatalogContext catalogContext; - public AbstractDataTableRead(TableSchema schema) { + public AbstractDataTableRead(TableSchema schema, CatalogContext catalogContext) { this.schema = schema; + this.catalogContext = catalogContext; } public abstract void applyReadType(RowType readType); @@ -96,6 +108,20 @@ public final RecordReader createReader(Split split) throws IOExcept split = authSplit.split(); authResult = authSplit.authResult(); } + + // Check if this split has BLOB_REF fields that need resolving + if (catalogContext != null) { + RowType rowType = this.readType == null ? schema.logicalRowType() : this.readType; + int[] blobRefFields = + rowType.getFields().stream() + .filter(field -> field.type().is(DataTypeRoot.BLOB_REF)) + .mapToInt(field -> rowType.getFieldIndex(field.name())) + .toArray(); + if (blobRefFields.length > 0) { + return createBlobRefReader(split, authResult, blobRefFields); + } + } + RecordReader reader; if (authResult == null) { reader = reader(split); @@ -105,10 +131,52 @@ public final RecordReader createReader(Split split) throws IOExcept if (executeFilter) { reader = executeFilter(reader); } - return reader; } + private RecordReader createBlobRefReader( + Split split, @Nullable TableQueryAuthResult authResult, int[] blobRefFields) + throws IOException { + // Pre-scan: read the split once to collect all BlobReferences via getBlob() + LinkedHashSet references = new LinkedHashSet<>(); + RecordReader prescanReader = + authResult == null ? reader(split) : authedReader(split, authResult); + try { + prescanReader.forEachRemaining( + row -> { + for (int field : blobRefFields) { + if (row.isNullAt(field)) { + continue; + } + Blob blob = row.getBlob(field); + if (blob instanceof UnresolvedBlob) { + references.add(((UnresolvedBlob) blob).reference()); + } + } + }); + } finally { + prescanReader.close(); + } + + // Build the resolver from collected references + List refList = new ArrayList<>(references); + BlobReferenceResolver resolver = + BlobReferenceLookup.createResolver(catalogContext, refList); + + // Second pass: wrap each row so getBlob() resolves UnresolvedBlob → real Blob + RecordReader reader = + authResult == null ? reader(split) : authedReader(split, authResult); + if (executeFilter) { + reader = executeFilter(reader); + } + Set blobRefFieldSet = new HashSet<>(); + for (int f : blobRefFields) { + blobRefFieldSet.add(f); + } + return reader.transform( + row -> new BlobRefResolvingRow(row, blobRefFieldSet, resolver)); + } + private RecordReader authedReader(Split split, TableQueryAuthResult authResult) throws IOException { RecordReader reader; diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java index 1a9ed9b4bee2..388ef9344c1e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java @@ -18,6 +18,7 @@ package org.apache.paimon.table.source; +import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.data.InternalRow; import org.apache.paimon.operation.MergeFileSplitRead; import org.apache.paimon.operation.SplitRead; @@ -51,8 +52,9 @@ public final class AppendTableRead extends AbstractDataTableRead { public AppendTableRead( List> providerFactories, - TableSchema schema) { - super(schema); + TableSchema schema, + CatalogContext catalogContext) { + super(schema, catalogContext); this.readProviders = providerFactories.stream() .map(factory -> factory.apply(this::config)) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java b/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java new file mode 100644 index 000000000000..270aa2469791 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java @@ -0,0 +1,161 @@ +/* + * 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.table.source; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobReferenceResolver; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalVector; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.data.UnresolvedBlob; +import org.apache.paimon.data.variant.Variant; +import org.apache.paimon.types.RowKind; + +import java.util.Set; + +/** + * {@link InternalRow} wrapper that resolves {@link UnresolvedBlob} to real {@link Blob} via a + * {@link BlobReferenceResolver} when {@link #getBlob(int)} is called. + */ +class BlobRefResolvingRow implements InternalRow { + + private final InternalRow wrapped; + private final Set blobRefFields; + private final BlobReferenceResolver resolver; + + BlobRefResolvingRow( + InternalRow wrapped, Set blobRefFields, BlobReferenceResolver resolver) { + this.wrapped = wrapped; + this.blobRefFields = blobRefFields; + this.resolver = resolver; + } + + @Override + public int getFieldCount() { + return wrapped.getFieldCount(); + } + + @Override + public RowKind getRowKind() { + return wrapped.getRowKind(); + } + + @Override + public void setRowKind(RowKind kind) { + wrapped.setRowKind(kind); + } + + @Override + public boolean isNullAt(int pos) { + return wrapped.isNullAt(pos); + } + + @Override + public boolean getBoolean(int pos) { + return wrapped.getBoolean(pos); + } + + @Override + public byte getByte(int pos) { + return wrapped.getByte(pos); + } + + @Override + public short getShort(int pos) { + return wrapped.getShort(pos); + } + + @Override + public int getInt(int pos) { + return wrapped.getInt(pos); + } + + @Override + public long getLong(int pos) { + return wrapped.getLong(pos); + } + + @Override + public float getFloat(int pos) { + return wrapped.getFloat(pos); + } + + @Override + public double getDouble(int pos) { + return wrapped.getDouble(pos); + } + + @Override + public BinaryString getString(int pos) { + return wrapped.getString(pos); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + return wrapped.getDecimal(pos, precision, scale); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + return wrapped.getTimestamp(pos, precision); + } + + @Override + public byte[] getBinary(int pos) { + return wrapped.getBinary(pos); + } + + @Override + public Variant getVariant(int pos) { + return wrapped.getVariant(pos); + } + + @Override + public Blob getBlob(int pos) { + Blob blob = wrapped.getBlob(pos); + if (blobRefFields.contains(pos) && blob instanceof UnresolvedBlob) { + return resolver.resolve(((UnresolvedBlob) blob).reference()); + } + return blob; + } + + @Override + public InternalRow getRow(int pos, int numFields) { + return wrapped.getRow(pos, numFields); + } + + @Override + public InternalArray getArray(int pos) { + return wrapped.getArray(pos); + } + + @Override + public InternalVector getVector(int pos) { + return wrapped.getVector(pos); + } + + @Override + public InternalMap getMap(int pos) { + return wrapped.getMap(pos); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java index fda7d70ffdf6..ac83737afdc8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.KeyValue; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; import org.apache.paimon.operation.MergeFileSplitRead; @@ -63,8 +64,9 @@ public final class KeyValueTableRead extends AbstractDataTableRead { public KeyValueTableRead( Supplier mergeReadSupplier, Supplier batchRawReadSupplier, - TableSchema schema) { - super(schema); + TableSchema schema, + CatalogContext catalogContext) { + super(schema, catalogContext); this.readProviders = Arrays.asList( new PrimaryKeyTableRawFileSplitReadProvider( diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java new file mode 100644 index 000000000000..6bed9de582f7 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java @@ -0,0 +1,254 @@ +/* + * 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.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobReference; +import org.apache.paimon.data.BlobReferenceResolver; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeSet; + +/** + * Utilities for resolving {@link BlobReference} through table metadata. + * + *

The preload phase only caches lightweight {@link BlobDescriptor} (uri + offset + length) + * rather than the actual blob data, so memory usage stays small even when a data file contains a + * large number of blob references. + */ +public class BlobReferenceLookup { + + private static final Logger LOG = LoggerFactory.getLogger(BlobReferenceLookup.class); + + /** + * Creates a resolver backed by a preloaded descriptor cache. The given references are batch + * scanned from the upstream tables to obtain their {@link BlobDescriptor}s. On a cache hit the + * descriptor is used to construct a {@link Blob} directly. On a cache miss the resolver falls + * back to scanning the upstream table individually. + */ + public static BlobReferenceResolver createResolver( + CatalogContext catalogContext, List references) { + if (references.isEmpty()) { + return createResolver(catalogContext); + } + Map cached = preloadDescriptors(catalogContext, references); + UriReaderFactory uriReaderFactory = new UriReaderFactory(catalogContext); + if (cached.isEmpty()) { + return createResolver(catalogContext); + } + return reference -> { + BlobDescriptor descriptor = cached.get(reference); + if (descriptor != null) { + return Blob.fromDescriptor(uriReaderFactory.create(descriptor.uri()), descriptor); + } + return resolve(catalogContext, reference); + }; + } + + /** Creates a simple resolver that scans the upstream table for each reference individually. */ + public static BlobReferenceResolver createResolver(CatalogContext catalogContext) { + return reference -> resolve(catalogContext, reference); + } + + public static Blob resolve(CatalogContext catalogContext, BlobReference reference) { + try (Catalog catalog = CatalogFactory.createCatalog(catalogContext)) { + Table table = catalog.getTable(Identifier.fromString(reference.tableName())); + if (!table.rowType().containsField(reference.fieldId())) { + throw new IllegalArgumentException( + "Cannot find blob fieldId " + + reference.fieldId() + + " in upstream table " + + reference.tableName() + + "."); + } + int fieldPos = table.rowType().getFieldIndexByFieldId(reference.fieldId()); + + ReadBuilder readBuilder = + table.newReadBuilder() + .withProjection(new int[] {fieldPos}) + .withRowRanges( + Collections.singletonList( + new Range(reference.rowId(), reference.rowId()))); + + try (RecordReader reader = + readBuilder.newRead().createReader(readBuilder.newScan().plan())) { + RecordReader.RecordIterator batch; + while ((batch = reader.readBatch()) != null) { + try { + InternalRow row; + while ((row = batch.next()) != null) { + return row.getBlob(0); + } + } finally { + batch.releaseBatch(); + } + } + } + + throw new IllegalStateException( + "Cannot resolve blob reference for table " + + reference.tableName() + + ", rowId " + + reference.rowId() + + ", fieldId " + + reference.fieldId() + + "."); + } catch (Exception e) { + throw new RuntimeException("Failed to resolve blob reference.", e); + } + } + + private static Map preloadDescriptors( + CatalogContext catalogContext, List references) { + try (Catalog catalog = CatalogFactory.createCatalog(catalogContext)) { + return loadReferencedDescriptors(catalog, references); + } catch (Exception e) { + LOG.warn("Failed to preload blob descriptors. Falling back to per-reference scan.", e); + return Collections.emptyMap(); + } + } + + private static Map loadReferencedDescriptors( + Catalog catalog, Collection references) throws Exception { + Map grouped = new HashMap<>(); + for (BlobReference reference : references) { + grouped.computeIfAbsent(reference.tableName(), TableReferences::new).add(reference); + } + Map resolved = new HashMap<>(); + for (TableReferences tableReferences : grouped.values()) { + loadTableDescriptors(catalog, tableReferences, resolved); + } + return resolved; + } + + private static void loadTableDescriptors( + Catalog catalog, + TableReferences tableReferences, + Map resolved) + throws Exception { + Table table = catalog.getTable(Identifier.fromString(tableReferences.tableName)); + + List fields = new ArrayList<>(tableReferences.referencesByField.size()); + TreeSet rowIds = new TreeSet<>(); + for (Map.Entry> entry : + tableReferences.referencesByField.entrySet()) { + int fieldId = entry.getKey(); + if (!table.rowType().containsField(fieldId)) { + throw new IllegalArgumentException( + "Cannot find blob fieldId " + + fieldId + + " in upstream table " + + tableReferences.tableName + + "."); + } + int fieldPos = table.rowType().getFieldIndexByFieldId(fieldId); + fields.add(new FieldRead(fieldId, fieldPos, table.rowType().getFields().get(fieldPos))); + for (BlobReference reference : entry.getValue()) { + rowIds.add(reference.rowId()); + } + } + + Collections.sort(fields, (left, right) -> Integer.compare(left.fieldPos, right.fieldPos)); + + List readFields = new ArrayList<>(fields.size()); + for (FieldRead field : fields) { + readFields.add(field.field); + } + + ReadBuilder readBuilder = + table.newReadBuilder() + .withReadType(SpecialFields.rowTypeWithRowId(new RowType(readFields))) + .withRowRanges(Range.toRanges(rowIds)); + + try (RecordReader reader = + readBuilder.newRead().createReader(readBuilder.newScan().plan())) { + RecordReader.RecordIterator batch; + while ((batch = reader.readBatch()) != null) { + try { + InternalRow row; + while ((row = batch.next()) != null) { + long rowId = row.getLong(fields.size()); + for (int i = 0; i < fields.size(); i++) { + Blob blob = row.getBlob(i); + if (blob != null) { + resolved.put( + new BlobReference( + tableReferences.tableName, + fields.get(i).fieldId, + rowId), + blob.toDescriptor()); + } + } + } + } finally { + batch.releaseBatch(); + } + } + } + } + + private static class TableReferences { + private final String tableName; + private final Map> referencesByField = new HashMap<>(); + + private TableReferences(String tableName) { + this.tableName = tableName; + } + + private void add(BlobReference reference) { + referencesByField + .computeIfAbsent(reference.fieldId(), unused -> new ArrayList<>()) + .add(reference); + } + } + + private static class FieldRead { + private final int fieldId; + private final int fieldPos; + private final DataField field; + + private FieldRead(int fieldId, int fieldPos, DataField field) { + this.fieldId = fieldId; + this.fieldPos = fieldPos; + this.field = field; + } + } + + private BlobReferenceLookup() {} +} diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java index e3019485bb2d..7203aeeb4008 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobData; import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.fs.FileIO; @@ -39,6 +40,8 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.TableTestBase; +import org.apache.paimon.table.sink.BatchWriteBuilder; +import org.apache.paimon.table.sink.BatchTableCommit; import org.apache.paimon.table.sink.BatchTableWrite; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.StreamTableWrite; @@ -749,6 +752,112 @@ void testRenameBlobColumnShouldFail() throws Exception { false)) .isInstanceOf(UnsupportedOperationException.class) .hasMessageContaining("Cannot rename BLOB column"); + + public void testBlobRefE2E() throws Exception { + // 1. Create upstream table with BLOB field and write data + String upstreamTableName = "UpstreamBlob"; + Schema.Builder upstreamSchema = Schema.newBuilder(); + upstreamSchema.column("id", DataTypes.INT()); + upstreamSchema.column("name", DataTypes.STRING()); + upstreamSchema.column("image", DataTypes.BLOB()); + upstreamSchema.option(CoreOptions.TARGET_FILE_SIZE.key(), "25 MB"); + upstreamSchema.option(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + upstreamSchema.option(CoreOptions.DATA_EVOLUTION_ENABLED.key(), "true"); + catalog.createTable(identifier(upstreamTableName), upstreamSchema.build(), true); + + FileStoreTable upstreamTable = + (FileStoreTable) catalog.getTable(identifier(upstreamTableName)); + + byte[] imageBytes1 = randomBytes(); + byte[] imageBytes2 = randomBytes(); + + BatchWriteBuilder upstreamWriteBuilder = upstreamTable.newBatchWriteBuilder(); + try (BatchTableWrite write = upstreamWriteBuilder.newWrite(); + BatchTableCommit commit = upstreamWriteBuilder.newCommit()) { + write.write( + GenericRow.of(1, BinaryString.fromString("row1"), new BlobData(imageBytes1))); + write.write( + GenericRow.of(2, BinaryString.fromString("row2"), new BlobData(imageBytes2))); + commit.commit(write.prepareCommit()); + } + + // 2. Get field ID for the "image" blob column + int imageFieldId = + upstreamTable.rowType().getFields().stream() + .filter(f -> f.name().equals("image")) + .findFirst() + .orElseThrow(() -> new RuntimeException("image field not found")) + .id(); + + // Read upstream with _ROW_ID to get actual row IDs + RowTrackingTable upstreamRowTracking = new RowTrackingTable(upstreamTable); + // schema: 0=id, 1=name, 2=image, 3=_ROW_ID, 4=_SEQUENCE_NUMBER + ReadBuilder rowIdReader = + upstreamRowTracking.newReadBuilder().withProjection(new int[] {0, 2, 3}); + // maps: upstream id -> (rowId, blobData) + java.util.Map idToRowId = new java.util.HashMap<>(); + java.util.Map idToBlob = new java.util.HashMap<>(); + rowIdReader + .newRead() + .createReader(rowIdReader.newScan().plan()) + .forEachRemaining( + row -> { + int id = row.getInt(0); + byte[] blobData = row.getBlob(1).toData(); + long rowId = row.getLong(2); + idToRowId.put(id, rowId); + idToBlob.put(id, blobData); + }); + assertThat(idToRowId.size()).isEqualTo(2); + + // 3. Create downstream table with BLOB_REF field + String downstreamTableName = "DownstreamRef"; + Schema.Builder downstreamSchema = Schema.newBuilder(); + downstreamSchema.column("id", DataTypes.INT()); + downstreamSchema.column("label", DataTypes.STRING()); + downstreamSchema.column("image_ref", DataTypes.BLOB_REF()); + downstreamSchema.option(CoreOptions.TARGET_FILE_SIZE.key(), "25 MB"); + downstreamSchema.option(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + downstreamSchema.option(CoreOptions.DATA_EVOLUTION_ENABLED.key(), "true"); + catalog.createTable(identifier(downstreamTableName), downstreamSchema.build(), true); + + FileStoreTable downstreamTable = + (FileStoreTable) catalog.getTable(identifier(downstreamTableName)); + + // 4. Write blob references using actual row IDs from upstream + String upstreamFullName = database + "." + upstreamTableName; + BlobReference ref1 = new BlobReference(upstreamFullName, imageFieldId, idToRowId.get(1)); + BlobReference ref2 = new BlobReference(upstreamFullName, imageFieldId, idToRowId.get(2)); + + BatchWriteBuilder downstreamWriteBuilder = downstreamTable.newBatchWriteBuilder(); + try (BatchTableWrite write = downstreamWriteBuilder.newWrite(); + BatchTableCommit commit = downstreamWriteBuilder.newCommit()) { + write.write( + GenericRow.of( + 1, BinaryString.fromString("label1"), Blob.fromReference(ref1))); + write.write( + GenericRow.of( + 2, BinaryString.fromString("label2"), Blob.fromReference(ref2))); + commit.commit(write.prepareCommit()); + } + + // 5. Read downstream table — blob references should resolve from upstream + ReadBuilder downstreamReadBuilder = downstreamTable.newReadBuilder(); + List downstreamRows = new ArrayList<>(); + downstreamReadBuilder + .newRead() + .createReader(downstreamReadBuilder.newScan().plan()) + .forEachRemaining(downstreamRows::add); + + assertThat(downstreamRows.size()).isEqualTo(2); + + for (InternalRow row : downstreamRows) { + int id = row.getInt(0); + Blob blob = row.getBlob(2); + assertThat(blob).isNotNull(); + // The resolved blob data should match the original upstream data + assertThat(blob.toData()).isEqualTo(idToBlob.get(id)); + } } private void createExternalStorageTable() throws Exception { diff --git a/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java new file mode 100644 index 000000000000..07916cd3bc15 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java @@ -0,0 +1,93 @@ +/* + * 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.io; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobReference; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.fileindex.FileIndexOptions; +import org.apache.paimon.format.FileFormat; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.options.Options; +import org.apache.paimon.statistics.NoneSimpleColStatsCollector; +import org.apache.paimon.statistics.SimpleColStatsCollector; +import org.apache.paimon.types.BlobRefType; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.LongCounter; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link RowDataFileWriter}. */ +public class RowDataFileWriterTest { + + private static final RowType SCHEMA = + RowType.of( + new DataType[] {new IntType(), new BlobRefType()}, new String[] {"id", "ref"}); + + @TempDir java.nio.file.Path tempDir; + + @Test + public void testWriteBlobRefDataFile() throws Exception { + FileFormat fileFormat = FileFormat.fromIdentifier("parquet", new Options()); + Path dataPath = new Path(tempDir.toUri().toString(), "data.parquet"); + BlobReference reference = new BlobReference("default.upstream", 7, 11L); + + RowDataFileWriter writer = + new RowDataFileWriter( + LocalFileIO.create(), + RollingFileWriter.createFileWriterContext( + fileFormat, + SCHEMA, + new SimpleColStatsCollector.Factory[] { + NoneSimpleColStatsCollector::new, + NoneSimpleColStatsCollector::new + }, + CoreOptions.FILE_COMPRESSION.defaultValue()), + dataPath, + SCHEMA, + 0L, + () -> new LongCounter(0), + new FileIndexOptions(), + FileSource.APPEND, + false, + false, + false, + SCHEMA.getFieldNames()); + + writer.write(GenericRow.of(1, Blob.fromReference(reference))); + writer.close(); + + DataFileMeta meta = writer.result(); + + // No .blobref extra files should be produced + assertThat( + meta.extraFiles().stream() + .noneMatch(f -> f.endsWith(".blobref"))) + .isTrue(); + assertThat(meta.rowCount()).isEqualTo(1); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/BlobRefSchemaValidationTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/BlobRefSchemaValidationTest.java new file mode 100644 index 000000000000..fcc345d719a9 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/schema/BlobRefSchemaValidationTest.java @@ -0,0 +1,69 @@ +/* + * 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.schema; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyList; +import static org.apache.paimon.CoreOptions.BUCKET; +import static org.apache.paimon.schema.SchemaValidation.validateTableSchema; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for BLOB_REF-specific schema validation. */ +public class BlobRefSchemaValidationTest { + + @Test + public void testNestedBlobRefTableSchema() { + Map options = new HashMap<>(); + options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + options.put(CoreOptions.DATA_EVOLUTION_ENABLED.key(), "true"); + options.put(BUCKET.key(), String.valueOf(-1)); + + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField( + 1, + "f1", + DataTypes.ROW(DataTypes.FIELD(2, "nested", DataTypes.BLOB_REF())))); + + assertThatThrownBy( + () -> + validateTableSchema( + new TableSchema( + 1, + fields, + 10, + emptyList(), + emptyList(), + options, + ""))) + .hasMessage( + "Nested BLOB_REF type is not supported. Field 'f1' contains a nested BLOB_REF."); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java new file mode 100644 index 000000000000..974c6f7e5ba8 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.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.utils; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobReference; +import org.apache.paimon.data.BlobReferenceResolver; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.disk.IOManager; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.metrics.MetricRegistry; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.source.TableRead; +import org.apache.paimon.table.source.TableScan; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.OptionalLong; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** Tests for {@link BlobReferenceLookup}. */ +public class BlobReferenceLookupTest { + + @TempDir java.nio.file.Path tempDir; + + @Test + public void testResolveByFieldIdAfterRename() throws Exception { + CatalogContext context = mock(CatalogContext.class); + Catalog catalog = mock(Catalog.class); + Table table = mock(Table.class); + ReadBuilder readBuilder = mock(ReadBuilder.class); + TableScan scan = mock(TableScan.class); + TableScan.Plan plan = mock(TableScan.Plan.class); + + byte[] payload = new byte[] {1, 2, 3}; + InternalRow row = GenericRow.of(Blob.fromData(payload)); + Split split = new TestSplit(); + + when(catalog.getTable(any())).thenReturn(table); + when(table.rowType()) + .thenReturn( + new RowType( + Collections.singletonList( + new DataField(7, "blob_renamed", DataTypes.BLOB())))); + when(table.newReadBuilder()).thenReturn(readBuilder); + when(readBuilder.withProjection(any(int[].class))).thenReturn(readBuilder); + when(readBuilder.withRowRanges(anyList())).thenReturn(readBuilder); + when(readBuilder.newRead()) + .thenReturn( + new ListRowTableRead(split, Collections.singletonList(row))); + when(readBuilder.newScan()).thenReturn(scan); + when(scan.plan()).thenReturn(plan); + when(plan.splits()).thenReturn(Collections.singletonList(split)); + + BlobReference reference = new BlobReference("default.source", 7, 12L); + + try (MockedStatic mockedCatalogFactory = + Mockito.mockStatic(CatalogFactory.class)) { + mockedCatalogFactory + .when(() -> CatalogFactory.createCatalog(context)) + .thenReturn(catalog); + + Blob resolved = BlobReferenceLookup.resolve(context, reference); + assertThat(resolved.toData()).isEqualTo(payload); + } + } + + @Test + public void testCreateResolverPreloadsDescriptors() throws Exception { + CatalogContext context = + CatalogContext.create(new Path(tempDir.toUri().toString())); + Catalog catalog = mock(Catalog.class); + Table table = mock(Table.class); + ReadBuilder readBuilder = mock(ReadBuilder.class); + TableScan scan = mock(TableScan.class); + TableScan.Plan plan = mock(TableScan.Plan.class); + + byte[] leftPayload = new byte[] {1, 2, 3}; + byte[] rightPayload = new byte[] {4, 5, 6}; + Path leftBlobPath = new Path(tempDir.toUri().toString(), "left.blob"); + Path rightBlobPath = new Path(tempDir.toUri().toString(), "right.blob"); + LocalFileIO fileIO = LocalFileIO.create(); + try (org.apache.paimon.fs.PositionOutputStream out = + fileIO.newOutputStream(leftBlobPath, false)) { + out.write(leftPayload); + } + try (org.apache.paimon.fs.PositionOutputStream out = + fileIO.newOutputStream(rightBlobPath, false)) { + out.write(rightPayload); + } + + BlobDescriptor leftDescriptor = + new BlobDescriptor(leftBlobPath.toString(), 0L, leftPayload.length); + BlobDescriptor rightDescriptor = + new BlobDescriptor(rightBlobPath.toString(), 0L, rightPayload.length); + + Blob leftBlob = + Blob.fromDescriptor(UriReader.fromFile(LocalFileIO.create()), leftDescriptor); + Blob rightBlob = + Blob.fromDescriptor(UriReader.fromFile(LocalFileIO.create()), rightDescriptor); + + BlobReference leftReference = new BlobReference("default.source", 7, 12L); + BlobReference rightReference = new BlobReference("default.source", 8, 12L); + + Split readerSplit = new TestSplit(); + InternalRow preloadRow = GenericRow.of(leftBlob, rightBlob, 12L); + + when(catalog.getTable(any())).thenReturn(table); + when(table.rowType()) + .thenReturn( + new RowType( + java.util.Arrays.asList( + new DataField(7, "blob_left", DataTypes.BLOB()), + new DataField(8, "blob_right", DataTypes.BLOB())))); + when(table.newReadBuilder()).thenReturn(readBuilder); + when(readBuilder.withReadType(any(RowType.class))).thenReturn(readBuilder); + when(readBuilder.withRowRanges(anyList())).thenReturn(readBuilder); + when(readBuilder.newRead()) + .thenAnswer( + invocation -> + new ListRowTableRead( + readerSplit, + Collections.singletonList(preloadRow))); + when(readBuilder.newScan()).thenReturn(scan); + when(scan.plan()).thenReturn(plan); + when(plan.splits()).thenReturn(Collections.singletonList(readerSplit)); + + try (MockedStatic mockedCatalogFactory = + Mockito.mockStatic(CatalogFactory.class)) { + mockedCatalogFactory + .when(() -> CatalogFactory.createCatalog(context)) + .thenReturn(catalog); + + BlobReferenceResolver resolver = + BlobReferenceLookup.createResolver( + context, + java.util.Arrays.asList(leftReference, rightReference)); + + assertThat(resolver.resolve(leftReference).toData()).isEqualTo(leftPayload); + assertThat(resolver.resolve(rightReference).toData()).isEqualTo(rightPayload); + assertThat(resolver.resolve(new BlobReference("default.source", 7, 12L)).toData()) + .isEqualTo(leftPayload); + + verify(table, times(1)).newReadBuilder(); + } + } + + private static class ListRowTableRead implements TableRead { + + private final Split split; + private final List rows; + + private ListRowTableRead(Split split, List rows) { + this.split = split; + this.rows = rows; + } + + @Override + public TableRead withMetricRegistry(MetricRegistry registry) { + return this; + } + + @Override + public TableRead executeFilter() { + return this; + } + + @Override + public TableRead withIOManager(IOManager ioManager) { + return this; + } + + @Override + public RecordReader createReader(Split split) { + return new RecordReader() { + + private boolean emitted = false; + + @Nullable + @Override + public RecordIterator readBatch() { + if (emitted) { + return null; + } + emitted = true; + return new RecordIterator() { + + private int next = 0; + + @Nullable + @Override + public InternalRow next() { + return next < rows.size() ? rows.get(next++) : null; + } + + @Override + public void releaseBatch() {} + }; + } + + @Override + public void close() throws IOException {} + }; + } + } + + private static class TestSplit implements Split { + + @Override + public long rowCount() { + return 1L; + } + + @Override + public OptionalLong mergedRowCount() { + return OptionalLong.of(1L); + } + } +} 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 92ae714ca577..21cb45cf0d93 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 @@ -21,6 +21,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -155,6 +156,11 @@ public LogicalType visit(BlobType blobType) { org.apache.flink.table.types.logical.VarBinaryType.MAX_LENGTH); } + @Override + public LogicalType visit(BlobRefType blobRefType) { + return new org.apache.flink.table.types.logical.VarBinaryType(BlobType.DEFAULT_SIZE); + } + @Override public LogicalType visit(ArrayType arrayType) { return new org.apache.flink.table.types.logical.ArrayType( 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 5f59063668a5..80d1d14d2d2f 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 @@ -153,6 +153,7 @@ import static org.apache.paimon.catalog.Catalog.SYSTEM_DATABASE_NAME; import static org.apache.paimon.catalog.Catalog.TOTAL_SIZE_PROP; import static org.apache.paimon.flink.FlinkCatalogOptions.DISABLE_CREATE_TABLE_IN_DEFAULT_DB; +import static org.apache.paimon.flink.LogicalTypeConversion.toBlobRefType; import static org.apache.paimon.flink.LogicalTypeConversion.toBlobType; import static org.apache.paimon.flink.LogicalTypeConversion.toDataType; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; @@ -1038,6 +1039,7 @@ public static Schema fromCatalogTable(CatalogBaseTable catalogTable) { Map options = new HashMap<>(catalogTable.getOptions()); List blobFields = CoreOptions.blobField(options); + List blobRefFields = CoreOptions.blobRefField(options); if (!blobFields.isEmpty()) { checkArgument( options.containsKey(CoreOptions.DATA_EVOLUTION_ENABLED.key()), @@ -1047,6 +1049,15 @@ public static Schema fromCatalogTable(CatalogBaseTable catalogTable) { + CoreOptions.DATA_EVOLUTION_ENABLED.key() + "'"); } + if (!blobRefFields.isEmpty()) { + checkArgument( + options.containsKey(CoreOptions.DATA_EVOLUTION_ENABLED.key()), + "When setting '" + + CoreOptions.BLOB_REF_FIELD.key() + + "', you must also set '" + + CoreOptions.DATA_EVOLUTION_ENABLED.key() + + "'"); + } // Serialize virtual columns and watermark to the options // This is what Flink SQL needs, the storage itself does not need them options.putAll(columnOptions(schema)); @@ -1077,9 +1088,13 @@ private static org.apache.paimon.types.DataType resolveDataType( org.apache.flink.table.types.logical.LogicalType logicalType, Map options) { List blobFields = CoreOptions.blobField(options); + List blobRefFields = CoreOptions.blobRefField(options); if (blobFields.contains(fieldName)) { return toBlobType(logicalType); } + if (blobRefFields.contains(fieldName)) { + return toBlobRefType(logicalType); + } Set vectorFields = CoreOptions.vectorField(options); if (vectorFields.contains(fieldName)) { return toVectorType(fieldName, logicalType, options); 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 ad2132e8c1eb..018cf3aa1867 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 @@ -22,7 +22,7 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobData; -import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; @@ -32,7 +32,6 @@ import org.apache.paimon.data.variant.GenericVariant; import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; -import org.apache.paimon.utils.UriReader; import org.apache.paimon.utils.UriReaderFactory; import org.apache.flink.table.data.DecimalData; @@ -55,7 +54,8 @@ public FlinkRowWrapper(org.apache.flink.table.data.RowData row) { public FlinkRowWrapper(org.apache.flink.table.data.RowData row, CatalogContext catalogContext) { this.row = row; - this.uriReaderFactory = new UriReaderFactory(catalogContext); + this.uriReaderFactory = + catalogContext == null ? null : new UriReaderFactory(catalogContext); } @Override @@ -142,15 +142,8 @@ public Variant getVariant(int pos) { @Override public Blob getBlob(int pos) { - byte[] bytes = row.getBinary(pos); - boolean blobDes = BlobDescriptor.isBlobDescriptor(bytes); - if (blobDes) { - BlobDescriptor blobDescriptor = BlobDescriptor.deserialize(bytes); - UriReader uriReader = uriReaderFactory.create(blobDescriptor.uri()); - return Blob.fromDescriptor(uriReader, blobDescriptor); - } else { - return new BlobData(bytes); - } + return BlobUtils.fromBytes( + row.getBinary(pos), uriReaderFactory, null); } @Override 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 556dbd95ff31..1cd0168c332d 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 @@ -19,6 +19,7 @@ package org.apache.paimon.flink; import org.apache.paimon.CoreOptions; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -53,6 +54,13 @@ public static BlobType toBlobType(LogicalType logicalType) { return new BlobType(); } + public static BlobRefType toBlobRefType(LogicalType logicalType) { + checkArgument( + logicalType instanceof BinaryType || logicalType instanceof VarBinaryType, + "Expected BinaryType or VarBinaryType, but got: " + logicalType); + return new BlobRefType(); + } + public static VectorType toVectorType( String fieldName, org.apache.flink.table.types.logical.LogicalType logicalType, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java index e4870de58336..76d83393b3d4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.lookup; +import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; import org.apache.paimon.operation.MergeFileSplitRead; @@ -41,8 +42,9 @@ public class LookupCompactDiffRead extends AbstractDataTableRead { private final SplitRead fullPhaseMergeRead; private final SplitRead incrementalDiffRead; - public LookupCompactDiffRead(MergeFileSplitRead mergeRead, TableSchema schema) { - super(schema); + public LookupCompactDiffRead( + MergeFileSplitRead mergeRead, TableSchema schema, CatalogContext catalogContext) { + super(schema, catalogContext); this.incrementalDiffRead = new IncrementalCompactDiffSplitRead(mergeRead); this.fullPhaseMergeRead = SplitRead.convert( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java index 353c99d2b1f1..4e355db1e8f2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java @@ -64,7 +64,9 @@ public InnerTableRead newRead() { return wrapped.newRead(); case COMPACT_DELTA_MONITOR: return new LookupCompactDiffRead( - ((KeyValueFileStore) wrapped.store()).newRead(), wrapped.schema()); + ((KeyValueFileStore) wrapped.store()).newRead(), + wrapped.schema(), + wrapped.catalogEnvironment().catalogContext()); default: throw new UnsupportedOperationException( "Unknown lookup stream scan mode: " + lookupScanMode.name()); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitReader.java index b49b9adb9476..eff22090735c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitReader.java @@ -277,7 +277,8 @@ private FileStoreRecordIterator(@Nullable RowType rowType) { private Set blobFieldIndex(RowType rowType) { Set result = new HashSet<>(); for (int i = 0; i < rowType.getFieldCount(); i++) { - if (rowType.getTypeAt(i).getTypeRoot() == DataTypeRoot.BLOB) { + if (rowType.getTypeAt(i).getTypeRoot() == DataTypeRoot.BLOB + || rowType.getTypeAt(i).getTypeRoot() == DataTypeRoot.BLOB_REF) { result.add(i); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index 8114ac17eb38..a0d5f98ba0e0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -152,7 +152,7 @@ public KeyValueTableRead createReadWithKey() { FileFormatDiscover.of(options), pathFactory, options); - return new KeyValueTableRead(() -> read, () -> rawFileRead, null); + return new KeyValueTableRead(() -> read, () -> rawFileRead, schema, null); } public List writeFiles( diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroSchemaConverter.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroSchemaConverter.java index eb55a86c5b66..251a973b3e0c 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroSchemaConverter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroSchemaConverter.java @@ -106,6 +106,7 @@ public static Schema convertToSchema( case BINARY: case VARBINARY: case BLOB: + case BLOB_REF: Schema binary = SchemaBuilder.builder().bytesType(); return nullable ? nullableSchema(binary) : binary; case TIMESTAMP_WITHOUT_TIME_ZONE: diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldReaderFactory.java index 9aa663df8946..47f12d45684b 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldReaderFactory.java @@ -89,10 +89,13 @@ public FieldReaderFactory(@Nullable UriReader uriReader) { @Override public FieldReader primitive(Schema primitive, DataType type) { - if (primitive.getType() == Schema.Type.BYTES - && type != null - && type.getTypeRoot() == DataTypeRoot.BLOB) { - return new BlobDescriptorBytesReader(uriReader); + if (primitive.getType() == Schema.Type.BYTES && type != null) { + if (type.getTypeRoot() == DataTypeRoot.BLOB) { + return new BlobDescriptorBytesReader(uriReader); + } + if (type.getTypeRoot() == DataTypeRoot.BLOB_REF) { + return BYTES_READER; + } } return AvroSchemaVisitor.super.primitive(primitive, type); } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java index 6eb81cb7f5d1..d48becb7ed46 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataGetters; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.GenericRow; @@ -93,12 +94,24 @@ public class FieldWriterFactory implements AvroSchemaVisitor { } }; + private static final FieldWriter BLOB_REFERENCE_BYTES_WRITER = + (container, i, encoder) -> { + Blob blob = container.getBlob(i); + if (blob == null) { + throw new IllegalArgumentException("Null blob_ref is not allowed."); + } + encoder.writeBytes(BlobUtils.serializeBlobReference(blob)); + }; + @Override public FieldWriter primitive(Schema primitive, DataType type) { - if (primitive.getType() == Schema.Type.BYTES - && type != null - && type.getTypeRoot() == DataTypeRoot.BLOB) { - return BLOB_DESCRIPTOR_BYTES_WRITER; + if (primitive.getType() == Schema.Type.BYTES && type != null) { + if (type.getTypeRoot() == DataTypeRoot.BLOB) { + return BLOB_DESCRIPTOR_BYTES_WRITER; + } + if (type.getTypeRoot() == DataTypeRoot.BLOB_REF) { + return BLOB_REFERENCE_BYTES_WRITER; + } } return AvroSchemaVisitor.super.primitive(primitive, type); } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcTypeUtil.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcTypeUtil.java index 4b80827d1bb3..4fe8b5999b3f 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcTypeUtil.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcTypeUtil.java @@ -68,6 +68,7 @@ static TypeDescription convertToOrcType(DataType type, int fieldId, int depth) { return TypeDescription.createBoolean() .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); case BLOB: + case BLOB_REF: return TypeDescription.createBinary() .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); case VARBINARY: 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 443c2410cbd2..8248937b5f14 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 @@ -28,6 +28,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -264,6 +265,18 @@ public FieldWriter visit(BlobType blobType) { }; } + @Override + public FieldWriter visit(BlobRefType blobRefType) { + return (rowId, column, getters, columnId) -> { + BytesColumnVector vector = (BytesColumnVector) column; + byte[] bytes = + org.apache.paimon.data.BlobUtils.serializeBlobReference( + getters.getBlob(columnId)); + vector.setVal(rowId, bytes, 0, bytes.length); + return bytes.length; + }; + } + @Override public FieldWriter visit(DecimalType decimalType) { return (rowId, column, getters, columnId) -> { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java index 37a69fe9aebd..102aa0b2b709 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java @@ -91,6 +91,7 @@ public static Type convertToParquetType(String name, DataType type, int fieldId, case BINARY: case VARBINARY: case BLOB: + case BLOB_REF: return Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) .named(name) .withId(fieldId); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReaderUtil.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReaderUtil.java index a2741f869ab6..a285491219a6 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReaderUtil.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReaderUtil.java @@ -101,6 +101,7 @@ public static WritableColumnVector createWritableColumnVector( case VARCHAR: case VARBINARY: case BLOB: + case BLOB_REF: return new HeapBytesVector(batchSize); case BINARY: return new HeapBytesVector(batchSize); @@ -178,7 +179,8 @@ public static ColumnVector createReadableColumnVector( case TIMESTAMP_WITH_LOCAL_TIME_ZONE: return new ParquetTimestampVector(writableVector); case BLOB: - // Physical representation is bytes; higher-level Row#getBlob() handles descriptor. + case BLOB_REF: + // Physical representation is bytes; higher-level Row#getBlob() materializes them. return writableVector; case ARRAY: return new CastedArrayColumnVector( 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 0abf78fd2747..2f2582b401e6 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 @@ -36,6 +36,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -230,6 +231,11 @@ public UpdaterFactory visit(BlobType blobType) { }; } + @Override + public UpdaterFactory visit(BlobRefType blobRefType) { + return visit(new BlobType(blobRefType.isNullable())); + } + @Override public UpdaterFactory visit(ArrayType arrayType) { throw new RuntimeException("Array type is not supported"); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java index a7241147e68a..b668a0d912c9 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; @@ -109,6 +110,8 @@ private FieldWriter createWriter(DataType t, Type type) { return new BinaryWriter(); case BLOB: return new BlobDescriptorWriter(); + case BLOB_REF: + return new BlobReferenceWriter(); case DECIMAL: DecimalType decimalType = (DecimalType) t; return createDecimalWriter(decimalType.getPrecision(), decimalType.getScale()); @@ -344,6 +347,25 @@ private void writeBlob(Blob blob) { } } + /** Writes BLOB_REF as serialized {@link org.apache.paimon.data.BlobReference} bytes. */ + private class BlobReferenceWriter implements FieldWriter { + + @Override + public void write(InternalRow row, int ordinal) { + writeBlob(row.getBlob(ordinal)); + } + + @Override + public void write(InternalArray arrayData, int ordinal) { + throw new UnsupportedOperationException("BLOB_REF in array is not supported."); + } + + private void writeBlob(Blob blob) { + byte[] bytes = BlobUtils.serializeBlobReference(blob); + recordConsumer.addBinary(Binary.fromReusedByteArray(bytes)); + } + } + private class IntWriter implements FieldWriter { @Override 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 dacd12f492c1..70c865f0b5f9 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 @@ -29,6 +29,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -414,6 +415,11 @@ public Operators.Column visit(BlobType blobType) { throw new UnsupportedOperationException(); } + @Override + public Operators.Column visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException(); + } + // ===================== can not support ========================= @Override diff --git a/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java index cb3d7de27da5..797756224b2a 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java @@ -18,6 +18,8 @@ package org.apache.paimon.format.avro; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.format.FileFormat; @@ -76,6 +78,7 @@ public void testSupportedDataTypes() { dataFields.add(new DataField(index++, "varchar_type", DataTypes.VARCHAR(20))); dataFields.add(new DataField(index++, "binary_type", DataTypes.BINARY(20))); dataFields.add(new DataField(index++, "varbinary_type", DataTypes.VARBINARY(20))); + dataFields.add(new DataField(index++, "blob_ref_type", DataTypes.BLOB_REF())); dataFields.add(new DataField(index++, "timestamp_type", DataTypes.TIMESTAMP(3))); dataFields.add(new DataField(index++, "date_type", DataTypes.DATE())); dataFields.add(new DataField(index++, "decimal_type", DataTypes.DECIMAL(10, 3))); @@ -210,4 +213,29 @@ void testCompression() throws IOException { .hasMessageContaining("Unrecognized codec: unsupported"); } } + + @Test + void testBlobRefRoundTrip() throws IOException { + RowType rowType = DataTypes.ROW(DataTypes.FIELD(0, "blob_ref", DataTypes.BLOB_REF())); + BlobReference reference = new BlobReference("default.t", 7, 11L); + Blob blob = Blob.fromReference(reference); + + FileFormat format = new AvroFileFormat(new FormatContext(new Options(), 1024, 1024)); + LocalFileIO fileIO = LocalFileIO.create(); + Path file = new Path(new Path(tempPath.toUri()), UUID.randomUUID().toString()); + + try (PositionOutputStream out = fileIO.newOutputStream(file, false)) { + FormatWriter writer = format.createWriterFactory(rowType).create(out, "zstd"); + writer.addElement(GenericRow.of(blob)); + writer.close(); + } + + try (RecordReader reader = + format.createReaderFactory(rowType, rowType, new ArrayList<>()) + .createReader( + new FormatReaderContext(fileIO, file, fileIO.getFileSize(file)))) { + InternalRow row = reader.readBatch().next(); + assertThat(row.getBinary(0)).isEqualTo(reference.serialize()); + } + } } diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcTypeUtilTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcTypeUtilTest.java index 5669ac33d443..5c36e14cfd1a 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcTypeUtilTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcTypeUtilTest.java @@ -60,6 +60,7 @@ void testDataTypeToOrcType() { test("varchar(123)", DataTypes.VARCHAR(123)); test("string", DataTypes.STRING()); test("binary", DataTypes.BYTES()); + test("binary", DataTypes.BLOB_REF()); test("tinyint", DataTypes.TINYINT()); test("smallint", DataTypes.SMALLINT()); test("int", DataTypes.INT()); diff --git a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetSchemaConverterTest.java b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetSchemaConverterTest.java index bfbdaed7c4a3..a312d8867b53 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetSchemaConverterTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetSchemaConverterTest.java @@ -25,6 +25,7 @@ import org.apache.paimon.types.RowType; import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; import org.junit.jupiter.api.Test; import java.util.Arrays; @@ -106,4 +107,15 @@ public void testPaimonParquetSchemaConvert() { RowType rowType = convertToPaimonRowType(messageType); assertThat(ALL_TYPES).isEqualTo(rowType); } + + @Test + public void testBlobRefSchemaConvertToBinary() { + MessageType messageType = + convertToParquetMessageType( + new RowType( + Arrays.asList(new DataField(0, "blob_ref", DataTypes.BLOB_REF())))); + + assertThat(messageType.getType("blob_ref").asPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.BINARY); + } } diff --git a/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/HiveTypeUtils.java b/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/HiveTypeUtils.java index e4799341d1dc..bcb48dffb485 100644 --- a/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/HiveTypeUtils.java +++ b/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/HiveTypeUtils.java @@ -22,6 +22,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -235,6 +236,11 @@ public TypeInfo visit(BlobType blobType) { return TypeInfoFactory.binaryTypeInfo; } + @Override + public TypeInfo visit(BlobRefType blobRefType) { + return TypeInfoFactory.binaryTypeInfo; + } + @Override protected TypeInfo defaultMethod(org.apache.paimon.types.DataType dataType) { throw new UnsupportedOperationException("Unsupported type: " + dataType); 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 64b4e2887f82..9421de6b60a3 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 @@ -27,6 +27,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -188,6 +189,11 @@ public Void visit(BlobType blobType) { return null; } + @Override + public Void visit(BlobRefType blobRefType) { + return null; + } + @Override public Void visit(ArrayType arrayType) { return null; diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index 6ef853eda870..7a07f4fb5ef8 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -41,6 +41,7 @@ import org.apache.paimon.table.iceberg.IcebergTable; import org.apache.paimon.table.lance.LanceTable; import org.apache.paimon.table.object.ObjectTable; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; @@ -457,6 +458,7 @@ private Schema toInitialSchema( StructType schema, Transform[] partitions, Map properties) { Map normalizedProperties = new HashMap<>(properties); List blobFields = CoreOptions.blobField(properties); + List blobRefFields = CoreOptions.blobRefField(properties); String provider = properties.get(TableCatalog.PROP_PROVIDER); if (!usePaimon(provider)) { if (isFormatTable(provider)) { @@ -495,6 +497,11 @@ private Schema toInitialSchema( field.dataType() instanceof org.apache.spark.sql.types.BinaryType, "The type of blob field must be binary"); type = new BlobType(); + } else if (blobRefFields.contains(name)) { + checkArgument( + field.dataType() instanceof org.apache.spark.sql.types.BinaryType, + "The type of blob ref field must be binary"); + type = new BlobRefType(); } else { type = toPaimonType(field.dataType()).copy(field.nullable()); } 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 ffd077741c9f..48e93f8bbc3c 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 @@ -22,7 +22,7 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobData; -import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; @@ -32,7 +32,6 @@ import org.apache.paimon.data.variant.Variant; import org.apache.paimon.spark.util.shim.TypeUtils$; import org.apache.paimon.types.RowKind; -import org.apache.paimon.utils.UriReader; import org.apache.paimon.utils.UriReaderFactory; import org.apache.spark.sql.catalyst.util.ArrayData; @@ -246,15 +245,8 @@ public Blob getBlob(int pos) { if (actualPos == -1 || internalRow.isNullAt(actualPos)) { return null; } - byte[] bytes = internalRow.getBinary(actualPos); - boolean blobDes = BlobDescriptor.isBlobDescriptor(bytes); - if (blobDes) { - BlobDescriptor blobDescriptor = BlobDescriptor.deserialize(bytes); - UriReader uriReader = uriReaderFactory.create(blobDescriptor.uri()); - return Blob.fromDescriptor(uriReader, blobDescriptor); - } else { - return new BlobData(bytes); - } + return BlobUtils.fromBytes( + internalRow.getBinary(actualPos), uriReaderFactory, null); } @Override 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 36b5624ff52f..3bc132810788 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 @@ -22,7 +22,7 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobData; -import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; @@ -38,7 +38,6 @@ import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.DateTimeUtils; -import org.apache.paimon.utils.UriReader; import org.apache.paimon.utils.UriReaderFactory; import org.apache.spark.sql.Row; @@ -72,7 +71,8 @@ public SparkRow(RowType type, Row row, RowKind rowkind, CatalogContext catalogCo this.type = type; this.row = row; this.rowKind = rowkind; - this.uriReaderFactory = new UriReaderFactory(catalogContext); + this.uriReaderFactory = + catalogContext == null ? null : new UriReaderFactory(catalogContext); } @Override @@ -161,15 +161,7 @@ public Variant getVariant(int i) { @Override public Blob getBlob(int i) { - byte[] bytes = row.getAs(i); - boolean blobDes = BlobDescriptor.isBlobDescriptor(bytes); - if (blobDes) { - BlobDescriptor blobDescriptor = BlobDescriptor.deserialize(bytes); - UriReader uriReader = uriReaderFactory.create(blobDescriptor.uri()); - return Blob.fromDescriptor(uriReader, blobDescriptor); - } else { - return new BlobData(bytes); - } + return BlobUtils.fromBytes(row.getAs(i), uriReaderFactory, null); } @Override diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTypeUtils.java b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTypeUtils.java index dc2f8b30acab..823534deea7c 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTypeUtils.java +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTypeUtils.java @@ -24,6 +24,7 @@ import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; import org.apache.paimon.types.BlobType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; import org.apache.paimon.types.DataField; @@ -167,6 +168,11 @@ public DataType visit(BlobType blobType) { return DataTypes.BinaryType; } + @Override + public DataType visit(BlobRefType blobRefType) { + return DataTypes.BinaryType; + } + @Override public DataType visit(VarBinaryType varBinaryType) { return DataTypes.BinaryType; diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala index ae504b24120f..4c34814df4f2 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala @@ -48,7 +48,10 @@ object SparkInternalRow { var i: Int = 0 val blobFields = new mutable.HashSet[Int]() while (i < rowType.getFieldCount) { - if (rowType.getTypeAt(i).getTypeRoot.equals(DataTypeRoot.BLOB)) { + if ( + rowType.getTypeAt(i).getTypeRoot.equals(DataTypeRoot.BLOB) || + rowType.getTypeAt(i).getTypeRoot.equals(DataTypeRoot.BLOB_REF) + ) { blobFields.add(i) } i += 1 diff --git a/paimon-vortex/paimon-vortex-format/src/main/java/org/apache/paimon/format/vortex/VortexFileFormat.java b/paimon-vortex/paimon-vortex-format/src/main/java/org/apache/paimon/format/vortex/VortexFileFormat.java index eda8a3944ca9..d6191af2204d 100644 --- a/paimon-vortex/paimon-vortex-format/src/main/java/org/apache/paimon/format/vortex/VortexFileFormat.java +++ b/paimon-vortex/paimon-vortex-format/src/main/java/org/apache/paimon/format/vortex/VortexFileFormat.java @@ -27,6 +27,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BlobRefType; import org.apache.paimon.types.BlobType; import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.CharType; @@ -187,6 +188,12 @@ public Void visit(BlobType blobType) { "Vortex file format does not support type BLOB"); } + @Override + public Void visit(BlobRefType blobRefType) { + throw new UnsupportedOperationException( + "Vortex file format does not support type BLOB_REF"); + } + @Override public Void visit(ArrayType arrayType) { return null; From 721a3ab6153c44018333ea34e30456a3760b5db3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Thu, 16 Apr 2026 17:21:59 +0800 Subject: [PATCH 02/19] Fix minus --- .../paimon/data/BlobReferenceHolder.java | 25 +++++++++++ .../org/apache/paimon/data/BlobUtils.java | 10 ++--- .../org/apache/paimon/data/GenericRow.java | 9 +++- .../apache/paimon/data/UnresolvedBlob.java | 2 +- .../apache/paimon/utils/InternalRowUtils.java | 1 + .../table/source/AbstractDataTableRead.java | 3 +- .../table/source/BlobRefResolvingRow.java | 43 ++++++++++++++++++- .../apache/paimon/append/BlobTableTest.java | 6 +-- .../paimon/io/RowDataFileWriterTest.java | 5 +-- .../paimon/utils/BlobReferenceLookupTest.java | 12 ++---- .../apache/paimon/flink/FlinkRowWrapper.java | 3 +- .../paimon/spark/SparkInternalRowWrapper.java | 3 +- 12 files changed, 91 insertions(+), 31 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceHolder.java diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceHolder.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceHolder.java new file mode 100644 index 000000000000..c5863bf5dffe --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceHolder.java @@ -0,0 +1,25 @@ +/* + * 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; + +/** A {@link Blob} that carries a {@link BlobReference} for re-serialization. */ +public interface BlobReferenceHolder { + + BlobReference reference(); +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java index 210c13e0f1d5..813e9a649d9b 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java @@ -28,9 +28,7 @@ public class BlobUtils { public static Blob fromBytes( - byte[] bytes, - @Nullable UriReaderFactory uriReaderFactory, - @Nullable FileIO fileIO) { + byte[] bytes, @Nullable UriReaderFactory uriReaderFactory, @Nullable FileIO fileIO) { if (bytes == null) { return null; } @@ -52,11 +50,11 @@ public static Blob fromBytes( } public static byte[] serializeBlobReference(Blob blob) { - if (blob instanceof UnresolvedBlob) { - return ((UnresolvedBlob) blob).reference().serialize(); + if (blob instanceof BlobReferenceHolder) { + return ((BlobReferenceHolder) blob).reference().serialize(); } throw new IllegalArgumentException( - "BLOB_REF fields only accept UnresolvedBlob values, but found " + "BLOB_REF fields only accept blobs with a BlobReference, but found " + blob.getClass().getSimpleName() + "."); } 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 10aefbafdd07..293f47bf445e 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 @@ -194,7 +194,14 @@ public Variant getVariant(int pos) { @Override public Blob getBlob(int pos) { - return (Blob) this.fields[pos]; + Object value = this.fields[pos]; + if (value instanceof Blob) { + return (Blob) value; + } + if (value instanceof byte[]) { + return BlobUtils.fromBytes((byte[]) value, null, null); + } + throw new ClassCastException("Cannot cast " + value.getClass().getName() + " to Blob"); } @Override diff --git a/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java b/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java index 6ba949d92576..a9bb9d02d298 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java @@ -27,7 +27,7 @@ * An unresolved {@link Blob} that only holds a {@link BlobReference}. It cannot provide data * directly — callers must resolve it through a {@link BlobReferenceResolver} first. */ -public class UnresolvedBlob implements Blob, Serializable { +public class UnresolvedBlob implements Blob, BlobReferenceHolder, Serializable { private static final long serialVersionUID = 1L; 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 4cfe35e39851..e2ff28c7bab1 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 @@ -331,6 +331,7 @@ public static Object get(DataGetters dataGetters, int pos, DataType fieldType) { case VARIANT: return dataGetters.getVariant(pos); case BLOB: + case BLOB_REF: return dataGetters.getBlob(pos); default: throw new UnsupportedOperationException("Unsupported type: " + fieldType); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java index 20be987f3ab1..cdf76db4f737 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java @@ -173,8 +173,7 @@ private RecordReader createBlobRefReader( for (int f : blobRefFields) { blobRefFieldSet.add(f); } - return reader.transform( - row -> new BlobRefResolvingRow(row, blobRefFieldSet, resolver)); + return reader.transform(row -> new BlobRefResolvingRow(row, blobRefFieldSet, resolver)); } private RecordReader authedReader(Split split, TableQueryAuthResult authResult) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java b/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java index 270aa2469791..fe0d1988a7fe 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java @@ -20,6 +20,9 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobReference; +import org.apache.paimon.data.BlobReferenceHolder; import org.apache.paimon.data.BlobReferenceResolver; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; @@ -29,8 +32,10 @@ import org.apache.paimon.data.Timestamp; import org.apache.paimon.data.UnresolvedBlob; import org.apache.paimon.data.variant.Variant; +import org.apache.paimon.fs.SeekableInputStream; import org.apache.paimon.types.RowKind; +import java.io.IOException; import java.util.Set; /** @@ -134,11 +139,47 @@ public Variant getVariant(int pos) { public Blob getBlob(int pos) { Blob blob = wrapped.getBlob(pos); if (blobRefFields.contains(pos) && blob instanceof UnresolvedBlob) { - return resolver.resolve(((UnresolvedBlob) blob).reference()); + BlobReference reference = ((UnresolvedBlob) blob).reference(); + Blob resolved = resolver.resolve(reference); + return new ResolvedBlobRef(reference, resolved); } return blob; } + /** + * A resolved blob that carries both the original {@link BlobReference} (for re-serialization) + * and the resolved {@link Blob} (for data access). + */ + static class ResolvedBlobRef implements Blob, BlobReferenceHolder { + + private final BlobReference reference; + private final Blob resolved; + + ResolvedBlobRef(BlobReference reference, Blob resolved) { + this.reference = reference; + this.resolved = resolved; + } + + public BlobReference reference() { + return reference; + } + + @Override + public byte[] toData() { + return resolved.toData(); + } + + @Override + public BlobDescriptor toDescriptor() { + return resolved.toDescriptor(); + } + + @Override + public SeekableInputStream newInputStream() throws IOException { + return resolved.newInputStream(); + } + } + @Override public InternalRow getRow(int pos, int numFields) { return wrapped.getRow(pos, numFields); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java index 7203aeeb4008..4f35dce66efc 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java @@ -833,11 +833,9 @@ public void testBlobRefE2E() throws Exception { try (BatchTableWrite write = downstreamWriteBuilder.newWrite(); BatchTableCommit commit = downstreamWriteBuilder.newCommit()) { write.write( - GenericRow.of( - 1, BinaryString.fromString("label1"), Blob.fromReference(ref1))); + GenericRow.of(1, BinaryString.fromString("label1"), Blob.fromReference(ref1))); write.write( - GenericRow.of( - 2, BinaryString.fromString("label2"), Blob.fromReference(ref2))); + GenericRow.of(2, BinaryString.fromString("label2"), Blob.fromReference(ref2))); commit.commit(write.prepareCommit()); } diff --git a/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java index 07916cd3bc15..d2db4ec3e2c9 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java @@ -84,10 +84,7 @@ public void testWriteBlobRefDataFile() throws Exception { DataFileMeta meta = writer.result(); // No .blobref extra files should be produced - assertThat( - meta.extraFiles().stream() - .noneMatch(f -> f.endsWith(".blobref"))) - .isTrue(); + assertThat(meta.extraFiles().stream().noneMatch(f -> f.endsWith(".blobref"))).isTrue(); assertThat(meta.rowCount()).isEqualTo(1); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java index 974c6f7e5ba8..a787b2cc6547 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java @@ -89,8 +89,7 @@ public void testResolveByFieldIdAfterRename() throws Exception { when(readBuilder.withProjection(any(int[].class))).thenReturn(readBuilder); when(readBuilder.withRowRanges(anyList())).thenReturn(readBuilder); when(readBuilder.newRead()) - .thenReturn( - new ListRowTableRead(split, Collections.singletonList(row))); + .thenReturn(new ListRowTableRead(split, Collections.singletonList(row))); when(readBuilder.newScan()).thenReturn(scan); when(scan.plan()).thenReturn(plan); when(plan.splits()).thenReturn(Collections.singletonList(split)); @@ -110,8 +109,7 @@ public void testResolveByFieldIdAfterRename() throws Exception { @Test public void testCreateResolverPreloadsDescriptors() throws Exception { - CatalogContext context = - CatalogContext.create(new Path(tempDir.toUri().toString())); + CatalogContext context = CatalogContext.create(new Path(tempDir.toUri().toString())); Catalog catalog = mock(Catalog.class); Table table = mock(Table.class); ReadBuilder readBuilder = mock(ReadBuilder.class); @@ -162,8 +160,7 @@ public void testCreateResolverPreloadsDescriptors() throws Exception { .thenAnswer( invocation -> new ListRowTableRead( - readerSplit, - Collections.singletonList(preloadRow))); + readerSplit, Collections.singletonList(preloadRow))); when(readBuilder.newScan()).thenReturn(scan); when(scan.plan()).thenReturn(plan); when(plan.splits()).thenReturn(Collections.singletonList(readerSplit)); @@ -176,8 +173,7 @@ public void testCreateResolverPreloadsDescriptors() throws Exception { BlobReferenceResolver resolver = BlobReferenceLookup.createResolver( - context, - java.util.Arrays.asList(leftReference, rightReference)); + context, java.util.Arrays.asList(leftReference, rightReference)); assertThat(resolver.resolve(leftReference).toData()).isEqualTo(leftPayload); assertThat(resolver.resolve(rightReference).toData()).isEqualTo(rightPayload); 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 018cf3aa1867..71f5304643a7 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 @@ -142,8 +142,7 @@ public Variant getVariant(int pos) { @Override public Blob getBlob(int pos) { - return BlobUtils.fromBytes( - row.getBinary(pos), uriReaderFactory, null); + return BlobUtils.fromBytes(row.getBinary(pos), uriReaderFactory, null); } @Override 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 48e93f8bbc3c..14b7331be319 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 @@ -245,8 +245,7 @@ public Blob getBlob(int pos) { if (actualPos == -1 || internalRow.isNullAt(actualPos)) { return null; } - return BlobUtils.fromBytes( - internalRow.getBinary(actualPos), uriReaderFactory, null); + return BlobUtils.fromBytes(internalRow.getBinary(actualPos), uriReaderFactory, null); } @Override From c385e88b7023de118c0669a30ad58c65aa470958 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Thu, 16 Apr 2026 17:36:44 +0800 Subject: [PATCH 03/19] Fix minus --- .../main/java/org/apache/paimon/utils/BlobReferenceLookup.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java index 6bed9de582f7..ea16ada4d8db 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java @@ -18,6 +18,7 @@ package org.apache.paimon.utils; +import java.util.Comparator; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; @@ -184,7 +185,7 @@ private static void loadTableDescriptors( } } - Collections.sort(fields, (left, right) -> Integer.compare(left.fieldPos, right.fieldPos)); + Collections.sort(fields, Comparator.comparingInt(left -> left.fieldPos)); List readFields = new ArrayList<>(fields.size()); for (FieldRead field : fields) { From d74de7c392b7e29d2f44d84d4d726dd103544ae5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Thu, 16 Apr 2026 17:40:05 +0800 Subject: [PATCH 04/19] Fix compile --- .../src/test/java/org/apache/paimon/append/BlobTableTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java index 4f35dce66efc..90394253dc8a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java @@ -752,7 +752,9 @@ void testRenameBlobColumnShouldFail() throws Exception { false)) .isInstanceOf(UnsupportedOperationException.class) .hasMessageContaining("Cannot rename BLOB column"); + } + @Test public void testBlobRefE2E() throws Exception { // 1. Create upstream table with BLOB field and write data String upstreamTableName = "UpstreamBlob"; From 2eb0e0597e96596adcbe3d4dfc6a087ae8c49031 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Thu, 16 Apr 2026 22:20:16 +0800 Subject: [PATCH 05/19] Fix minus --- .../arrow/ArrowFieldTypeConversion.java | 3 +- .../Arrow2PaimonVectorConverter.java | 2 +- .../ArrowFieldWriterFactoryVisitor.java | 2 +- .../arrow/writer/ArrowFieldWriters.java | 37 +++++++++ .../ArrowVectorizedBatchConverterTest.java | 75 +++++++++++++++++++ .../OneElementFieldVectorGeneratorTest.java | 32 ++++++++ .../table/AppendOnlyFileStoreTable.java | 5 +- .../table/PrimaryKeyFileStoreTable.java | 9 ++- .../table/source/AbstractDataTableRead.java | 51 ++++++++++--- .../paimon/table/source/AppendTableRead.java | 6 +- .../table/source/KeyValueTableRead.java | 15 +++- .../paimon/utils/BlobReferenceLookup.java | 2 +- .../apache/paimon/append/BlobTableTest.java | 2 +- .../source/TestChangelogDataReadWrite.java | 2 +- 14 files changed, 222 insertions(+), 21 deletions(-) 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 37b36a24d154..aca278588b7d 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 @@ -166,7 +166,8 @@ public FieldType visit(BlobType blobType) { @Override public FieldType visit(BlobRefType blobRefType) { - throw new UnsupportedOperationException(); + return new FieldType( + blobRefType.isNullable(), Types.MinorType.VARBINARY.getType(), null); } private TimeUnit getTimeUnit(int precision) { 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 d8672dfdc23b..6c2008452276 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 @@ -450,7 +450,7 @@ public Arrow2PaimonVectorConverter visit(BlobType blobType) { @Override public Arrow2PaimonVectorConverter visit(BlobRefType blobRefType) { - throw new UnsupportedOperationException(); + return visit(new VarBinaryType()); } @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 b4d38e2dae61..287a33f0382f 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 @@ -159,7 +159,7 @@ public ArrowFieldWriterFactory visit(BlobType blobType) { @Override public ArrowFieldWriterFactory visit(BlobRefType blobRefType) { - throw new UnsupportedOperationException("Doesn't support BlobRefType."); + return ArrowFieldWriters.BlobRefWriter::new; } @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 2999acdaf651..20a9dd5e4a77 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 @@ -20,6 +20,7 @@ import org.apache.paimon.arrow.ArrowUtils; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataGetters; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalArray; @@ -199,6 +200,42 @@ protected void doWrite(int rowIndex, DataGetters getters, int pos) { } } + /** + * Writer for BLOB_REF. The batch path is identical to {@link BinaryWriter} (columnar data is + * already serialized bytes). The row-by-row path serializes the {@link + * org.apache.paimon.data.Blob} via {@link BlobUtils#serializeBlobReference}. + */ + public static class BlobRefWriter extends ArrowFieldWriter { + + public BlobRefWriter(FieldVector fieldVector, boolean isNullable) { + super(fieldVector, isNullable); + } + + @Override + protected void doWrite( + ColumnVector columnVector, + @Nullable int[] pickedInColumn, + int startIndex, + int batchRows) { + VarBinaryVector varBinaryVector = (VarBinaryVector) fieldVector; + for (int i = 0; i < batchRows; i++) { + int row = getRowNumber(startIndex, i, pickedInColumn); + if (columnVector.isNullAt(row)) { + varBinaryVector.setNull(i); + } else { + byte[] value = ((BytesColumnVector) columnVector).getBytes(row).getBytes(); + varBinaryVector.setSafe(i, value); + } + } + } + + @Override + protected void doWrite(int rowIndex, DataGetters getters, int pos) { + byte[] bytes = BlobUtils.serializeBlobReference(getters.getBlob(pos)); + ((VarBinaryVector) fieldVector).setSafe(rowIndex, bytes); + } + } + /** Writer for DECIMAL. */ public static class DecimalWriter extends ArrowFieldWriter { diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverterTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverterTest.java index ec484de64bd8..06d0f6dc8cb1 100644 --- a/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverterTest.java +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverterTest.java @@ -20,18 +20,21 @@ import org.apache.paimon.arrow.ArrowUtils; import org.apache.paimon.arrow.writer.ArrowFieldWriter; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.columnar.ColumnVector; import org.apache.paimon.data.columnar.ColumnarVec; import org.apache.paimon.data.columnar.VecColumnVector; import org.apache.paimon.data.columnar.VectorizedColumnBatch; +import org.apache.paimon.data.columnar.heap.HeapBytesVector; import org.apache.paimon.data.columnar.heap.HeapFloatVector; import org.apache.paimon.reader.VectorizedRecordIterator; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.complex.FixedSizeListVector; import org.junit.jupiter.api.Test; @@ -129,6 +132,78 @@ public void testVectorColumnWriteWithPickedInColumn() { } } + @Test + public void testBlobRefColumnWriteAndReadBack() { + RowType rowType = RowType.of(DataTypes.BLOB_REF()); + try (RootAllocator allocator = new RootAllocator()) { + VectorSchemaRoot vsr = ArrowUtils.createVectorSchemaRoot(rowType, allocator); + ArrowFieldWriter[] fieldWriters = ArrowUtils.createArrowFieldWriters(vsr, rowType); + + // Prepare serialized BlobReference bytes + BlobReference ref0 = new BlobReference("default.upstream", 7, 100L); + BlobReference ref1 = new BlobReference("default.upstream", 8, 200L); + byte[] bytes0 = ref0.serialize(); + byte[] bytes1 = ref1.serialize(); + + int rows = 3; // row 0 = ref0, row 1 = null, row 2 = ref1 + HeapBytesVector bytesVector = new HeapBytesVector(rows); + bytesVector.appendByteArray(bytes0, 0, bytes0.length); + bytesVector.appendByteArray(new byte[0], 0, 0); // placeholder for null + bytesVector.appendByteArray(bytes1, 0, bytes1.length); + bytesVector.setNullAt(1); + + VectorizedColumnBatch batch = + new VectorizedColumnBatch(new ColumnVector[] {bytesVector}); + batch.setNumRows(rows); + + ArrowVectorizedBatchConverter converter = + new ArrowVectorizedBatchConverter(vsr, fieldWriters); + converter.reset( + new VectorizedRecordIterator() { + @Override + public VectorizedColumnBatch batch() { + return batch; + } + + @Override + public InternalRow next() { + return null; + } + + @Override + public void releaseBatch() {} + }); + converter.next(rows); + + // Verify the Arrow vector contains the correct binary data + VarBinaryVector arrowVector = (VarBinaryVector) vsr.getVector(0); + assertThat(arrowVector.isNull(0)).isFalse(); + assertThat(arrowVector.isNull(1)).isTrue(); + assertThat(arrowVector.isNull(2)).isFalse(); + + // Read back and verify the bytes can be deserialized to BlobReference + BlobReference readRef0 = BlobReference.deserialize(arrowVector.getObject(0)); + assertThat(readRef0.tableName()).isEqualTo("default.upstream"); + assertThat(readRef0.fieldId()).isEqualTo(7); + assertThat(readRef0.rowId()).isEqualTo(100L); + + BlobReference readRef2 = BlobReference.deserialize(arrowVector.getObject(2)); + assertThat(readRef2.tableName()).isEqualTo("default.upstream"); + assertThat(readRef2.fieldId()).isEqualTo(8); + assertThat(readRef2.rowId()).isEqualTo(200L); + + // Also verify the Arrow2Paimon round-trip + Arrow2PaimonVectorConverter paimonConverter = + Arrow2PaimonVectorConverter.construct(DataTypes.BLOB_REF()); + ColumnVector paimonVector = paimonConverter.convertVector(arrowVector); + assertThat(paimonVector.isNullAt(0)).isFalse(); + assertThat(paimonVector.isNullAt(1)).isTrue(); + assertThat(paimonVector.isNullAt(2)).isFalse(); + + converter.close(); + } + } + private static class TestVecColumnVectorWithNulls implements VecColumnVector { private final ColumnVector data; diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/OneElementFieldVectorGeneratorTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/OneElementFieldVectorGeneratorTest.java index a417133da1a1..32bab40e3445 100644 --- a/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/OneElementFieldVectorGeneratorTest.java +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/OneElementFieldVectorGeneratorTest.java @@ -20,6 +20,8 @@ import org.apache.paimon.arrow.reader.ArrowBatchReader; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.Timestamp; @@ -95,4 +97,34 @@ public void testFunction() { } } } + + @Test + public void testBlobRef() { + try (RootAllocator rootAllocator = new RootAllocator()) { + DataField dataField = new DataField(0, "ref", DataTypes.BLOB_REF()); + BlobReference reference = new BlobReference("default.upstream", 7, 42L); + Blob value = Blob.fromReference(reference); + + OneElementFieldVectorGenerator generator = + new OneElementFieldVectorGenerator(rootAllocator, dataField, value); + try (FieldVector fieldVector = generator.get(3)) { + Assertions.assertThat(fieldVector.getValueCount()).isEqualTo(3); + + // Read back via ArrowBatchReader — BLOB_REF comes back as binary bytes + ArrowBatchReader reader = + new ArrowBatchReader(new RowType(Arrays.asList(dataField)), true); + Iterable it = + reader.readBatch(new VectorSchemaRoot(Arrays.asList(fieldVector))); + it.forEach( + row -> { + byte[] bytes = row.getBinary(0); + BlobReference readRef = BlobReference.deserialize(bytes); + Assertions.assertThat(readRef.tableName()) + .isEqualTo("default.upstream"); + Assertions.assertThat(readRef.fieldId()).isEqualTo(7); + Assertions.assertThat(readRef.rowId()).isEqualTo(42L); + }); + } + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java index 8cb70824cd45..5b3c2600f397 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java @@ -124,7 +124,10 @@ public InnerTableRead newRead() { () -> store().newRead(), config)); } return new AppendTableRead( - providerFactories, schema(), catalogEnvironment().catalogContext()); + providerFactories, + schema(), + catalogEnvironment().catalogContext(), + () -> new AppendTableRead(providerFactories, schema(), null, null)); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java index 411bcf6767de..52b6faa74acf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java @@ -150,7 +150,14 @@ public InnerTableRead newRead() { () -> store().newRead(), () -> store().newBatchRawFileRead(), schema(), - catalogEnvironment().catalogContext()); + catalogEnvironment().catalogContext(), + () -> + new KeyValueTableRead( + () -> store().newRead(), + () -> store().newBatchRawFileRead(), + schema(), + null, + null)); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java index cdf76db4f737..8a8afd0afc6e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java @@ -45,6 +45,7 @@ import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.function.Supplier; import static org.apache.paimon.predicate.PredicateVisitor.collectFieldNames; @@ -55,17 +56,34 @@ public abstract class AbstractDataTableRead implements InnerTableRead { private boolean executeFilter = false; private Predicate predicate; private final TableSchema schema; - private final CatalogContext catalogContext; + @Nullable private final CatalogContext catalogContext; + @Nullable private final Supplier readFactory; - public AbstractDataTableRead(TableSchema schema, CatalogContext catalogContext) { + public AbstractDataTableRead(TableSchema schema, @Nullable CatalogContext catalogContext) { + this(schema, catalogContext, null); + } + + public AbstractDataTableRead( + TableSchema schema, + @Nullable CatalogContext catalogContext, + @Nullable Supplier readFactory) { this.schema = schema; this.catalogContext = catalogContext; + this.readFactory = readFactory; } public abstract void applyReadType(RowType readType); public abstract RecordReader reader(Split split) throws IOException; + /** + * Hook for subclasses to transfer additional runtime state (e.g. forceKeepDelete, ioManager) to + * the independent pre-scan read used for BLOB_REF reference collection. Called after + * withReadType and withFilter have already been set. Subclasses should NOT set limit or topN on + * the prescan read, as the pre-scan must cover all rows. + */ + protected void configurePrescanRead(InnerTableRead prescanRead) {} + @Override public TableRead withIOManager(IOManager ioManager) { return this; @@ -118,6 +136,11 @@ public final RecordReader createReader(Split split) throws IOExcept .mapToInt(field -> rowType.getFieldIndex(field.name())) .toArray(); if (blobRefFields.length > 0) { + if (readFactory == null) { + throw new IllegalStateException( + "Cannot read BLOB_REF fields without a readFactory. " + + "The table must provide a readFactory to support BLOB_REF resolution."); + } return createBlobRefReader(split, authResult, blobRefFields); } } @@ -137,18 +160,28 @@ public final RecordReader createReader(Split split) throws IOExcept private RecordReader createBlobRefReader( Split split, @Nullable TableQueryAuthResult authResult, int[] blobRefFields) throws IOException { - // Pre-scan: read the split once to collect all BlobReferences via getBlob() + // Pre-scan: use an independent read instance to read only BLOB_REF columns. + // Transfer predicate to narrow the scan range, but NOT limit/topN since the + // pre-scan must cover all rows that the second pass might return. + RowType rowType = this.readType == null ? schema.logicalRowType() : this.readType; + RowType blobRefOnlyType = rowType.project(blobRefFields); + InnerTableRead prescanRead = readFactory.get(); + prescanRead.withReadType(blobRefOnlyType); + if (predicate != null) { + prescanRead.withFilter(predicate); + } + configurePrescanRead(prescanRead); + Split prescanSplit = authResult != null ? new QueryAuthSplit(split, authResult) : split; LinkedHashSet references = new LinkedHashSet<>(); - RecordReader prescanReader = - authResult == null ? reader(split) : authedReader(split, authResult); + RecordReader prescanReader = prescanRead.createReader(prescanSplit); try { prescanReader.forEachRemaining( row -> { - for (int field : blobRefFields) { - if (row.isNullAt(field)) { + for (int i = 0; i < blobRefFields.length; i++) { + if (row.isNullAt(i)) { continue; } - Blob blob = row.getBlob(field); + Blob blob = row.getBlob(i); if (blob instanceof UnresolvedBlob) { references.add(((UnresolvedBlob) blob).reference()); } @@ -163,7 +196,7 @@ private RecordReader createBlobRefReader( BlobReferenceResolver resolver = BlobReferenceLookup.createResolver(catalogContext, refList); - // Second pass: wrap each row so getBlob() resolves UnresolvedBlob → real Blob + // Second pass: read all columns, wrap each row to resolve UnresolvedBlob RecordReader reader = authResult == null ? reader(split) : authedReader(split, authResult); if (executeFilter) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java index 388ef9344c1e..5501e37d9e7b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java @@ -36,6 +36,7 @@ import java.util.ArrayList; import java.util.List; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; /** @@ -53,8 +54,9 @@ public final class AppendTableRead extends AbstractDataTableRead { public AppendTableRead( List> providerFactories, TableSchema schema, - CatalogContext catalogContext) { - super(schema, catalogContext); + CatalogContext catalogContext, + @Nullable Supplier readFactory) { + super(schema, catalogContext, readFactory); this.readProviders = providerFactories.stream() .map(factory -> factory.apply(this::config)) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java index ac83737afdc8..b5dd6cef309c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java @@ -65,8 +65,9 @@ public KeyValueTableRead( Supplier mergeReadSupplier, Supplier batchRawReadSupplier, TableSchema schema, - CatalogContext catalogContext) { - super(schema, catalogContext); + CatalogContext catalogContext, + @Nullable Supplier readFactory) { + super(schema, catalogContext, readFactory); this.readProviders = Arrays.asList( new PrimaryKeyTableRawFileSplitReadProvider( @@ -102,6 +103,16 @@ private void config(SplitRead read) { read.withFilter(predicate).withIOManager(ioManager); } + @Override + protected void configurePrescanRead(InnerTableRead prescanRead) { + if (forceKeepDelete) { + prescanRead.forceKeepDelete(); + } + if (ioManager != null) { + prescanRead.withIOManager(ioManager); + } + } + @Override public void applyReadType(RowType readType) { initialized().forEach(r -> r.withReadType(readType)); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java index ea16ada4d8db..af8b2be35c96 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java @@ -18,7 +18,6 @@ package org.apache.paimon.utils; -import java.util.Comparator; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; @@ -41,6 +40,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java index 90394253dc8a..731134b7b295 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java @@ -40,9 +40,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.TableTestBase; -import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.sink.BatchTableCommit; import org.apache.paimon.table.sink.BatchTableWrite; +import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.StreamTableWrite; import org.apache.paimon.table.sink.StreamWriteBuilder; diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index a0d5f98ba0e0..f110c7e31488 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -152,7 +152,7 @@ public KeyValueTableRead createReadWithKey() { FileFormatDiscover.of(options), pathFactory, options); - return new KeyValueTableRead(() -> read, () -> rawFileRead, schema, null); + return new KeyValueTableRead(() -> read, () -> rawFileRead, schema, null, null); } public List writeFiles( From 95c8c597e73ef609e2bbcd89a8da0b5d4b469f21 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Thu, 16 Apr 2026 23:18:52 +0800 Subject: [PATCH 06/19] Fix test --- .../scala/org/apache/paimon/codegen/GenerateUtils.scala | 4 ++-- .../paimon/codegen/EqualiserCodeGeneratorTest.java | 9 +++++++++ .../src/main/java/org/apache/paimon/data/BinaryRow.java | 2 +- .../src/main/java/org/apache/paimon/data/NestedRow.java | 2 +- .../java/org/apache/paimon/utils/TypeCheckUtils.java | 3 ++- 5 files changed, 15 insertions(+), 5 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 967d58ad30db..6e5d073fccc8 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 @@ -385,7 +385,7 @@ object GenerateUtils { case MULTISET | MAP => className[InternalMap] case ROW => className[InternalRow] case VARIANT => className[Variant] - case BLOB => className[Blob] + case BLOB | BLOB_REF => className[Blob] case _ => throw new IllegalArgumentException("Illegal type: " + t) } @@ -428,7 +428,7 @@ object GenerateUtils { s"$rowTerm.getRow($indexTerm, ${getFieldCount(t)})" case VARIANT => s"$rowTerm.getVariant($indexTerm)" - case BLOB => + case BLOB | BLOB_REF => s"$rowTerm.getBlob($indexTerm)" case _ => throw new IllegalArgumentException("Illegal type: " + t) 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 7e977291e81f..cbc0cbde9c52 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 @@ -22,7 +22,9 @@ import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.BinaryWriter; +import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobData; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.GenericMap; import org.apache.paimon.data.GenericRow; @@ -206,6 +208,13 @@ public class EqualiserCodeGeneratorTest { Pair.of( new BlobData(new byte[] {1, 2, 3}), new BlobData(new byte[] {4, 5, 6})))); + TEST_DATA.put( + DataTypeRoot.BLOB_REF, + new GeneratedData( + DataTypes.BLOB_REF(), + Pair.of( + Blob.fromReference(new BlobReference("default.t1", 1, 0L)), + Blob.fromReference(new BlobReference("default.t2", 2, 1L))))); } @ParameterizedTest 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 ff5406f7b326..05d005558af5 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 @@ -346,7 +346,7 @@ public Variant getVariant(int pos) { @Override public Blob getBlob(int pos) { - return new BlobData(getBinary(pos)); + return BlobUtils.fromBytes(getBinary(pos), null, null); } @Override 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 afc4f0c47fb0..447700efe77a 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 @@ -291,7 +291,7 @@ public Variant getVariant(int pos) { @Override public Blob getBlob(int pos) { - return new BlobData(getBinary(pos)); + return BlobUtils.fromBytes(getBinary(pos), null, null); } @Override 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 c1520be34107..b4b22517865a 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 @@ -24,6 +24,7 @@ import static org.apache.paimon.types.DataTypeRoot.ARRAY; import static org.apache.paimon.types.DataTypeRoot.BIGINT; import static org.apache.paimon.types.DataTypeRoot.BLOB; +import static org.apache.paimon.types.DataTypeRoot.BLOB_REF; import static org.apache.paimon.types.DataTypeRoot.BOOLEAN; import static org.apache.paimon.types.DataTypeRoot.DECIMAL; import static org.apache.paimon.types.DataTypeRoot.INTEGER; @@ -107,7 +108,7 @@ public static boolean isVariant(DataType type) { } public static boolean isBlob(DataType type) { - return type.getTypeRoot() == BLOB; + return type.getTypeRoot() == BLOB || type.getTypeRoot() == BLOB_REF; } public static boolean isComparable(DataType type) { From 4e03b00c00e9e5405b0f0b6197670c0ea8b35807 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Fri, 17 Apr 2026 15:13:19 +0800 Subject: [PATCH 07/19] Fix test --- .../java/org/apache/paimon/schema/SchemaValidationTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java index 2f7ffbb6d937..a6570bc0e872 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java @@ -144,13 +144,13 @@ public void testBlobTableSchema() { options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); assertThatThrownBy(() -> validateBlobSchema(options, emptyList())) - .hasMessage("Data evolution config must enabled for table with BLOB type column."); + .hasMessage("Data evolution config must enabled for table with BLOB or BLOB_REF type column."); options.clear(); options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); options.put(CoreOptions.DATA_EVOLUTION_ENABLED.key(), "true"); assertThatThrownBy(() -> validateBlobSchema(options, singletonList("f2"))) - .hasMessage("The BLOB type column can not be part of partition keys."); + .hasMessage("The BLOB or BLOB_REF type column can not be part of partition keys."); assertThatThrownBy( () -> { @@ -164,7 +164,7 @@ public void testBlobTableSchema() { options, "")); }) - .hasMessage("Table with BLOB type column must have other normal columns."); + .hasMessage("Table with BLOB or BLOB_REF type column must have other normal columns."); } @Test From 1ee173fd650c8cefe093be008782506505e45885 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Fri, 17 Apr 2026 15:21:22 +0800 Subject: [PATCH 08/19] Checkstyle --- .../java/org/apache/paimon/schema/SchemaValidationTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java index a6570bc0e872..9b931181f27f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java @@ -144,7 +144,8 @@ public void testBlobTableSchema() { options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); assertThatThrownBy(() -> validateBlobSchema(options, emptyList())) - .hasMessage("Data evolution config must enabled for table with BLOB or BLOB_REF type column."); + .hasMessage( + "Data evolution config must enabled for table with BLOB or BLOB_REF type column."); options.clear(); options.put(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); @@ -164,7 +165,8 @@ public void testBlobTableSchema() { options, "")); }) - .hasMessage("Table with BLOB or BLOB_REF type column must have other normal columns."); + .hasMessage( + "Table with BLOB or BLOB_REF type column must have other normal columns."); } @Test From bee42eeacecb558fd009de34b36feb1adcfe3df0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Mon, 20 Apr 2026 00:37:57 +0800 Subject: [PATCH 09/19] Fix minus --- .../arrow/writer/ArrowFieldWriters.java | 3 +- .../OneElementFieldVectorGeneratorTest.java | 3 +- .../apache/paimon/codegen/GenerateUtils.scala | 7 +- .../org/apache/paimon/data/BinaryRow.java | 6 ++ .../org/apache/paimon/data/BinaryWriter.java | 4 +- .../java/org/apache/paimon/data/Blob.java | 4 +- .../java/org/apache/paimon/data/BlobRef.java | 69 ++++++++++++++--- .../paimon/data/BlobReferenceHolder.java | 25 ------- .../paimon/data/BlobReferenceResolver.java | 4 +- .../org/apache/paimon/data/BlobUtils.java | 18 ++--- .../org/apache/paimon/data/DataGetters.java | 6 ++ .../org/apache/paimon/data/GenericRow.java | 15 ++-- .../org/apache/paimon/data/InternalRow.java | 7 +- .../org/apache/paimon/data/NestedRow.java | 6 ++ .../apache/paimon/data/UnresolvedBlob.java | 75 ------------------- .../paimon/data/columnar/ColumnarRow.java | 11 +++ .../data/serializer/BlobRefSerializer.java | 16 ++-- .../types/InternalRowToSizeVisitor.java | 6 +- .../apache/paimon/utils/InternalRowUtils.java | 3 +- .../types/InternalRowToSizeVisitorTest.java | 14 ++-- .../table/source/AbstractDataTableRead.java | 9 +-- .../table/source/BlobRefResolvingRow.java | 52 ++----------- .../paimon/utils/BlobReferenceLookup.java | 31 ++++++-- .../apache/paimon/append/BlobTableTest.java | 9 ++- .../paimon/io/RowDataFileWriterTest.java | 2 +- .../paimon/utils/BlobReferenceLookupTest.java | 17 +++-- .../format/avro/FieldWriterFactory.java | 7 +- .../format/orc/writer/FieldWriterFactory.java | 4 +- .../parquet/writer/ParquetRowDataWriter.java | 8 +- .../format/avro/AvroFileFormatTest.java | 3 +- 30 files changed, 202 insertions(+), 242 deletions(-) delete mode 100644 paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceHolder.java delete mode 100644 paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java 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 20a9dd5e4a77..6a977e65f655 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 @@ -20,7 +20,6 @@ import org.apache.paimon.arrow.ArrowUtils; import org.apache.paimon.data.BinaryString; -import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataGetters; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalArray; @@ -231,7 +230,7 @@ protected void doWrite( @Override protected void doWrite(int rowIndex, DataGetters getters, int pos) { - byte[] bytes = BlobUtils.serializeBlobReference(getters.getBlob(pos)); + byte[] bytes = getters.getBlobRef(pos).reference().serialize(); ((VarBinaryVector) fieldVector).setSafe(rowIndex, bytes); } } diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/OneElementFieldVectorGeneratorTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/OneElementFieldVectorGeneratorTest.java index 32bab40e3445..ff4ea1e0ed67 100644 --- a/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/OneElementFieldVectorGeneratorTest.java +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/OneElementFieldVectorGeneratorTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.arrow.reader.ArrowBatchReader; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; @@ -103,7 +104,7 @@ public void testBlobRef() { try (RootAllocator rootAllocator = new RootAllocator()) { DataField dataField = new DataField(0, "ref", DataTypes.BLOB_REF()); BlobReference reference = new BlobReference("default.upstream", 7, 42L); - Blob value = Blob.fromReference(reference); + BlobRef value = Blob.fromReference(reference); OneElementFieldVectorGenerator generator = new OneElementFieldVectorGenerator(rootAllocator, dataField, value); 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 6e5d073fccc8..e4a3fb6f9551 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 @@ -385,7 +385,8 @@ object GenerateUtils { case MULTISET | MAP => className[InternalMap] case ROW => className[InternalRow] case VARIANT => className[Variant] - case BLOB | BLOB_REF => className[Blob] + case BLOB => className[Blob] + case BLOB_REF => className[BlobRef] case _ => throw new IllegalArgumentException("Illegal type: " + t) } @@ -428,8 +429,10 @@ object GenerateUtils { s"$rowTerm.getRow($indexTerm, ${getFieldCount(t)})" case VARIANT => s"$rowTerm.getVariant($indexTerm)" - case BLOB | BLOB_REF => + case BLOB => s"$rowTerm.getBlob($indexTerm)" + case BLOB_REF => + s"$rowTerm.getBlobRef($indexTerm)" case _ => throw new IllegalArgumentException("Illegal type: " + t) } 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 05d005558af5..3c2aecc3dc6d 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 @@ -349,6 +349,12 @@ public Blob getBlob(int pos) { return BlobUtils.fromBytes(getBinary(pos), null, null); } + @Override + public BlobRef getBlobRef(int pos) { + byte[] bytes = getBinary(pos); + return new BlobRef(BlobReference.deserialize(bytes)); + } + @Override public InternalArray getArray(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 b22336e31b8d..fbb1570d3517 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 @@ -158,7 +158,7 @@ static void write( writer.writeBlob(pos, (Blob) o); break; case BLOB_REF: - byte[] refBytes = BlobUtils.serializeBlobReference((Blob) o); + byte[] refBytes = ((BlobRef) o).reference().serialize(); writer.writeBinary(pos, refBytes, 0, refBytes.length); break; default: @@ -247,7 +247,7 @@ static ValueSetter createValueSetter(DataType elementType, Serializer seriali return (writer, pos, value) -> writer.writeBlob(pos, (Blob) value); case BLOB_REF: return (writer, pos, value) -> { - byte[] bytes = BlobUtils.serializeBlobReference((Blob) value); + byte[] bytes = ((BlobRef) value).reference().serialize(); writer.writeBinary(pos, bytes, 0, bytes.length); }; default: diff --git a/paimon-common/src/main/java/org/apache/paimon/data/Blob.java b/paimon-common/src/main/java/org/apache/paimon/data/Blob.java index 2ab095d23a9e..8eed2733c671 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/Blob.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/Blob.java @@ -65,8 +65,8 @@ static Blob fromDescriptor(UriReader reader, BlobDescriptor descriptor) { return new BlobRef(reader, descriptor); } - static Blob fromReference(BlobReference reference) { - return new UnresolvedBlob(reference); + static BlobRef fromReference(BlobReference reference) { + return new BlobRef(reference); } static Blob fromInputStream(Supplier supplier) { diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobRef.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobRef.java index 0248454ee90e..ae5fa5f1f8d8 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BlobRef.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobRef.java @@ -24,25 +24,61 @@ import org.apache.paimon.utils.IOUtils; import org.apache.paimon.utils.UriReader; +import javax.annotation.Nullable; + import java.io.IOException; +import java.io.Serializable; import java.util.Objects; /** - * A {@link Blob} refers blob in {@link BlobDescriptor}. + * A {@link Blob} that can represent both descriptor-backed blobs (for BLOB type) and + * reference-based blobs (for BLOB_REF type). + * + *

For BLOB type: created via {@link #BlobRef(UriReader, BlobDescriptor)}, always resolved. + * + *

For BLOB_REF type: created via {@link #BlobRef(BlobReference)}, initially unresolved. + * Call {@link #resolve(UriReader, BlobDescriptor)} to make it readable. * * @since 1.4.0 */ @Public -public class BlobRef implements Blob { +public class BlobRef implements Blob, Serializable { + + private static final long serialVersionUID = 1L; - private final UriReader uriReader; - private final BlobDescriptor descriptor; + @Nullable private final BlobReference reference; + @Nullable private UriReader uriReader; + @Nullable private BlobDescriptor descriptor; + /** Creates a resolved descriptor-backed blob (for BLOB type). */ public BlobRef(UriReader uriReader, BlobDescriptor descriptor) { + this.reference = null; this.uriReader = uriReader; this.descriptor = descriptor; } + /** Creates an unresolved blob ref (for BLOB_REF type). */ + public BlobRef(BlobReference reference) { + this.reference = reference; + this.uriReader = null; + this.descriptor = null; + } + + @Nullable + public BlobReference reference() { + return reference; + } + + public boolean isResolved() { + return uriReader != null && descriptor != null; + } + + /** Resolves this blob ref in place by setting the reader and descriptor. */ + public void resolve(UriReader reader, BlobDescriptor desc) { + this.uriReader = reader; + this.descriptor = desc; + } + @Override public byte[] toData() { try { @@ -54,15 +90,21 @@ public byte[] toData() { @Override public BlobDescriptor toDescriptor() { - return descriptor; + if (descriptor != null) { + return descriptor; + } + throw new IllegalStateException("BlobRef is not resolved."); } @Override public SeekableInputStream newInputStream() throws IOException { - return new OffsetSeekableInputStream( - uriReader.newInputStream(descriptor.uri()), - descriptor.offset(), - descriptor.length()); + if (uriReader != null && descriptor != null) { + return new OffsetSeekableInputStream( + uriReader.newInputStream(descriptor.uri()), + descriptor.offset(), + descriptor.length()); + } + throw new IllegalStateException("BlobRef is not resolved."); } @Override @@ -70,12 +112,15 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - BlobRef blobRef = (BlobRef) o; - return Objects.deepEquals(descriptor, blobRef.descriptor); + BlobRef that = (BlobRef) o; + if (reference != null) { + return Objects.equals(reference, that.reference); + } + return Objects.equals(descriptor, that.descriptor); } @Override public int hashCode() { - return descriptor.hashCode(); + return reference != null ? Objects.hash(reference) : Objects.hash(descriptor); } } diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceHolder.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceHolder.java deleted file mode 100644 index c5863bf5dffe..000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceHolder.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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; - -/** A {@link Blob} that carries a {@link BlobReference} for re-serialization. */ -public interface BlobReferenceHolder { - - BlobReference reference(); -} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceResolver.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceResolver.java index 9e0d78485542..dd263edef718 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceResolver.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobReferenceResolver.java @@ -20,9 +20,9 @@ import java.io.Serializable; -/** Resolves a {@link BlobReference} through fallback metadata. */ +/** Resolves a {@link BlobRef} by setting its reader and descriptor in place. */ @FunctionalInterface public interface BlobReferenceResolver extends Serializable { - Blob resolve(BlobReference reference); + void resolve(BlobRef blobRef); } diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java index 813e9a649d9b..02c02d07e4ec 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobUtils.java @@ -27,16 +27,16 @@ /** Utilities for decoding and encoding blob-related bytes. */ public class BlobUtils { + /** + * Decodes blob bytes for BLOB type fields. For BLOB_REF type, use {@link + * DataGetters#getBlobRef(int)} instead. + */ public static Blob fromBytes( byte[] bytes, @Nullable UriReaderFactory uriReaderFactory, @Nullable FileIO fileIO) { if (bytes == null) { return null; } - if (BlobReference.isBlobReference(bytes)) { - return new UnresolvedBlob(BlobReference.deserialize(bytes)); - } - if (BlobDescriptor.isBlobDescriptor(bytes)) { BlobDescriptor descriptor = BlobDescriptor.deserialize(bytes); UriReader reader = @@ -49,14 +49,8 @@ public static Blob fromBytes( return new BlobData(bytes); } - public static byte[] serializeBlobReference(Blob blob) { - if (blob instanceof BlobReferenceHolder) { - return ((BlobReferenceHolder) blob).reference().serialize(); - } - throw new IllegalArgumentException( - "BLOB_REF fields only accept blobs with a BlobReference, but found " - + blob.getClass().getSimpleName() - + "."); + public static byte[] serializeBlobReference(BlobRef blobRef) { + return blobRef.reference().serialize(); } private BlobUtils() {} 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 1043b7e3ba4f..43df3fa8f8df 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 @@ -81,6 +81,12 @@ public interface DataGetters { /** Returns the blob value at the given position. */ Blob getBlob(int pos); + /** Returns the blob ref value at the given position. For BLOB_REF type fields only. */ + default BlobRef getBlobRef(int pos) { + throw new UnsupportedOperationException( + getClass().getSimpleName() + " does not support getBlobRef."); + } + /** Returns the array value at the given position. */ InternalArray getArray(int 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 293f47bf445e..4ceb893ee04f 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 @@ -194,14 +194,17 @@ public Variant getVariant(int pos) { @Override public Blob getBlob(int pos) { + return (Blob) this.fields[pos]; + } + + @Override + public BlobRef getBlobRef(int pos) { Object value = this.fields[pos]; - if (value instanceof Blob) { - return (Blob) value; - } - if (value instanceof byte[]) { - return BlobUtils.fromBytes((byte[]) value, null, null); + if (value instanceof BlobRef) { + return (BlobRef) value; } - throw new ClassCastException("Cannot cast " + value.getClass().getName() + " to Blob"); + throw new ClassCastException( + "Cannot cast " + value.getClass().getName() + " to BlobRef"); } @Override 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 f4e9e6960b6b..e466111e0072 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 @@ -147,8 +147,9 @@ static Class getDataClass(DataType type) { case TIMESTAMP_WITH_LOCAL_TIME_ZONE: return Timestamp.class; case BLOB: - case BLOB_REF: return Blob.class; + case BLOB_REF: + return BlobRef.class; case ARRAY: return InternalArray.class; case MULTISET: @@ -231,9 +232,11 @@ static FieldGetter createFieldGetter(DataType fieldType, int fieldPos) { fieldGetter = row -> row.getVariant(fieldPos); break; case BLOB: - case BLOB_REF: fieldGetter = row -> row.getBlob(fieldPos); break; + case BLOB_REF: + fieldGetter = row -> row.getBlobRef(fieldPos); + break; default: String msg = String.format( 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 447700efe77a..531e193409a3 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 @@ -294,6 +294,12 @@ public Blob getBlob(int pos) { return BlobUtils.fromBytes(getBinary(pos), null, null); } + @Override + public BlobRef getBlobRef(int pos) { + byte[] bytes = getBinary(pos); + return new BlobRef(BlobReference.deserialize(bytes)); + } + @Override public InternalRow getRow(int pos, int numFields) { assertIndexIsValid(pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java b/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java deleted file mode 100644 index a9bb9d02d298..000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/data/UnresolvedBlob.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.fs.SeekableInputStream; - -import java.io.Serializable; -import java.util.Objects; - -/** - * An unresolved {@link Blob} that only holds a {@link BlobReference}. It cannot provide data - * directly — callers must resolve it through a {@link BlobReferenceResolver} first. - */ -public class UnresolvedBlob implements Blob, BlobReferenceHolder, Serializable { - - private static final long serialVersionUID = 1L; - - private final BlobReference reference; - - public UnresolvedBlob(BlobReference reference) { - this.reference = reference; - } - - public BlobReference reference() { - return reference; - } - - @Override - public byte[] toData() { - throw new IllegalStateException( - "UnresolvedBlob cannot provide data. Resolve it first via BlobReferenceResolver."); - } - - @Override - public BlobDescriptor toDescriptor() { - throw new IllegalStateException( - "UnresolvedBlob cannot provide descriptor. Resolve it first via BlobReferenceResolver."); - } - - @Override - public SeekableInputStream newInputStream() { - throw new IllegalStateException( - "UnresolvedBlob cannot provide stream. Resolve it first via BlobReferenceResolver."); - } - - @Override - public boolean equals(Object o) { - if (o == null || getClass() != o.getClass()) { - return false; - } - UnresolvedBlob that = (UnresolvedBlob) o; - return Objects.equals(reference, that.reference); - } - - @Override - public int hashCode() { - return Objects.hash(reference); - } -} 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 79a97915efe7..34d59d1f6a7a 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 @@ -20,6 +20,8 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobRef; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataSetters; import org.apache.paimon.data.Decimal; @@ -164,6 +166,15 @@ public Blob getBlob(int pos) { return BlobUtils.fromBytes(bytes, null, fileIO); } + @Override + public BlobRef getBlobRef(int pos) { + byte[] bytes = getBinary(pos); + if (bytes == null) { + return null; + } + return new BlobRef(BlobReference.deserialize(bytes)); + } + @Override public InternalRow getRow(int pos, int numFields) { return vectorizedColumnBatch.getRow(rowId, pos); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/serializer/BlobRefSerializer.java b/paimon-common/src/main/java/org/apache/paimon/data/serializer/BlobRefSerializer.java index 91f88aae8e62..9d40468fdb11 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/serializer/BlobRefSerializer.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/serializer/BlobRefSerializer.java @@ -18,35 +18,33 @@ package org.apache.paimon.data.serializer; -import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobReference; -import org.apache.paimon.data.BlobUtils; -import org.apache.paimon.data.UnresolvedBlob; import org.apache.paimon.io.DataInputView; import org.apache.paimon.io.DataOutputView; import java.io.IOException; /** Type serializer for {@code BLOB_REF}. */ -public class BlobRefSerializer extends SerializerSingleton { +public class BlobRefSerializer extends SerializerSingleton { private static final long serialVersionUID = 1L; public static final BlobRefSerializer INSTANCE = new BlobRefSerializer(); @Override - public Blob copy(Blob from) { + public BlobRef copy(BlobRef from) { return from; } @Override - public void serialize(Blob blob, DataOutputView target) throws IOException { - BinarySerializer.INSTANCE.serialize(BlobUtils.serializeBlobReference(blob), target); + public void serialize(BlobRef blobRef, DataOutputView target) throws IOException { + BinarySerializer.INSTANCE.serialize(blobRef.reference().serialize(), target); } @Override - public Blob deserialize(DataInputView source) throws IOException { + public BlobRef deserialize(DataInputView source) throws IOException { byte[] bytes = BinarySerializer.INSTANCE.deserialize(source); - return new UnresolvedBlob(BlobReference.deserialize(bytes)); + return new BlobRef(BlobReference.deserialize(bytes)); } } 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 94c71b6346ec..d69982f4836a 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 @@ -241,11 +241,7 @@ public BiFunction visit(BlobRefType blobRefType) if (row.isNullAt(index)) { return NULL_SIZE; } - try { - return row.getBinary(index).length; - } catch (ClassCastException | UnsupportedOperationException e) { - return BlobUtils.serializeBlobReference(row.getBlob(index)).length; - } + return row.getBlobRef(index).reference().serialize().length; }; } 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 e2ff28c7bab1..407df59555f2 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 @@ -331,8 +331,9 @@ public static Object get(DataGetters dataGetters, int pos, DataType fieldType) { case VARIANT: return dataGetters.getVariant(pos); case BLOB: - case BLOB_REF: return dataGetters.getBlob(pos); + case BLOB_REF: + return dataGetters.getBlobRef(pos); default: throw new UnsupportedOperationException("Unsupported type: " + fieldType); } diff --git a/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java b/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java index 728eb84e56e9..3357b54bdc3f 100644 --- a/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java @@ -19,6 +19,9 @@ package org.apache.paimon.types; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobRef; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.DataGetters; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.GenericArray; @@ -200,15 +203,16 @@ void testCalculatorSize() { @Test void testBlobRefSizeUsesSerializedReferenceBytes() { + BlobReference reference = new BlobReference("default.t", 1, 0L); + BlobRef blobRef = Blob.fromReference(reference); DataGetters row = mock(DataGetters.class); - byte[] referenceBytes = new byte[] {1, 2, 3, 4}; when(row.isNullAt(0)).thenReturn(false); - when(row.getBinary(0)).thenReturn(referenceBytes); + when(row.getBlobRef(0)).thenReturn(blobRef); int size = new InternalRowToSizeVisitor().visit(DataTypes.BLOB_REF()).apply(row, 0); - Assertions.assertThat(size).isEqualTo(referenceBytes.length); - verify(row).getBinary(0); - verify(row, never()).getBlob(0); + Assertions.assertThat(size).isEqualTo(reference.serialize().length); + verify(row).getBlobRef(0); + verify(row, never()).getBinary(0); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java index 8a8afd0afc6e..b3aa9524ed73 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java @@ -20,11 +20,10 @@ import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.TableQueryAuthResult; -import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.BlobReferenceResolver; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.data.UnresolvedBlob; import org.apache.paimon.disk.IOManager; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateProjectionConverter; @@ -181,10 +180,8 @@ private RecordReader createBlobRefReader( if (row.isNullAt(i)) { continue; } - Blob blob = row.getBlob(i); - if (blob instanceof UnresolvedBlob) { - references.add(((UnresolvedBlob) blob).reference()); - } + BlobRef blobRef = row.getBlobRef(i); + references.add(blobRef.reference()); } }); } finally { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java b/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java index fe0d1988a7fe..2b7484b6c84b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java @@ -20,9 +20,7 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; -import org.apache.paimon.data.BlobDescriptor; -import org.apache.paimon.data.BlobReference; -import org.apache.paimon.data.BlobReferenceHolder; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobReferenceResolver; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; @@ -30,12 +28,9 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.InternalVector; import org.apache.paimon.data.Timestamp; -import org.apache.paimon.data.UnresolvedBlob; import org.apache.paimon.data.variant.Variant; -import org.apache.paimon.fs.SeekableInputStream; import org.apache.paimon.types.RowKind; -import java.io.IOException; import java.util.Set; /** @@ -137,47 +132,16 @@ public Variant getVariant(int pos) { @Override public Blob getBlob(int pos) { - Blob blob = wrapped.getBlob(pos); - if (blobRefFields.contains(pos) && blob instanceof UnresolvedBlob) { - BlobReference reference = ((UnresolvedBlob) blob).reference(); - Blob resolved = resolver.resolve(reference); - return new ResolvedBlobRef(reference, resolved); - } - return blob; + return wrapped.getBlob(pos); } - /** - * A resolved blob that carries both the original {@link BlobReference} (for re-serialization) - * and the resolved {@link Blob} (for data access). - */ - static class ResolvedBlobRef implements Blob, BlobReferenceHolder { - - private final BlobReference reference; - private final Blob resolved; - - ResolvedBlobRef(BlobReference reference, Blob resolved) { - this.reference = reference; - this.resolved = resolved; - } - - public BlobReference reference() { - return reference; - } - - @Override - public byte[] toData() { - return resolved.toData(); - } - - @Override - public BlobDescriptor toDescriptor() { - return resolved.toDescriptor(); - } - - @Override - public SeekableInputStream newInputStream() throws IOException { - return resolved.newInputStream(); + @Override + public BlobRef getBlobRef(int pos) { + BlobRef blobRef = wrapped.getBlobRef(pos); + if (blobRefFields.contains(pos) && !blobRef.isResolved()) { + resolver.resolve(blobRef); } + return blobRef; } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java index af8b2be35c96..ffa1367367a6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java @@ -24,6 +24,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.BlobReferenceResolver; import org.apache.paimon.data.InternalRow; @@ -73,21 +74,39 @@ public static BlobReferenceResolver createResolver( if (cached.isEmpty()) { return createResolver(catalogContext); } - return reference -> { - BlobDescriptor descriptor = cached.get(reference); + return blobRef -> { + BlobDescriptor descriptor = cached.get(blobRef.reference()); if (descriptor != null) { - return Blob.fromDescriptor(uriReaderFactory.create(descriptor.uri()), descriptor); + blobRef.resolve(uriReaderFactory.create(descriptor.uri()), descriptor); + } else { + resolveFromUpstream(catalogContext, blobRef); } - return resolve(catalogContext, reference); }; } /** Creates a simple resolver that scans the upstream table for each reference individually. */ public static BlobReferenceResolver createResolver(CatalogContext catalogContext) { - return reference -> resolve(catalogContext, reference); + return blobRef -> resolveFromUpstream(catalogContext, blobRef); } - public static Blob resolve(CatalogContext catalogContext, BlobReference reference) { + private static void resolveFromUpstream(CatalogContext catalogContext, BlobRef blobRef) { + BlobReference reference = blobRef.reference(); + Blob resolved = resolveBlob(catalogContext, reference); + try { + BlobDescriptor desc = resolved.toDescriptor(); + UriReaderFactory uriReaderFactory = new UriReaderFactory(catalogContext); + blobRef.resolve(uriReaderFactory.create(desc.uri()), desc); + } catch (Exception e) { + // Inline blob — store the raw data directly + byte[] data = resolved.toData(); + blobRef.resolve( + uri -> new org.apache.paimon.fs.ByteArraySeekableStream(data), + new BlobDescriptor("memory://inline", 0, data.length)); + } + } + + /** Scans the upstream table to get the blob for a single reference. */ + static Blob resolveBlob(CatalogContext catalogContext, BlobReference reference) { try (Catalog catalog = CatalogFactory.createCatalog(catalogContext)) { Table table = catalog.getTable(Identifier.fromString(reference.tableName())); if (!table.rowType().containsField(reference.fieldId())) { diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java index 731134b7b295..e12540f8c73f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobData; import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; @@ -853,10 +854,10 @@ public void testBlobRefE2E() throws Exception { for (InternalRow row : downstreamRows) { int id = row.getInt(0); - Blob blob = row.getBlob(2); - assertThat(blob).isNotNull(); - // The resolved blob data should match the original upstream data - assertThat(blob.toData()).isEqualTo(idToBlob.get(id)); + BlobRef blobRef = row.getBlobRef(2); + assertThat(blobRef).isNotNull(); + assertThat(blobRef.isResolved()).isTrue(); + assertThat(blobRef.toData()).isEqualTo(idToBlob.get(id)); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java index d2db4ec3e2c9..0e9e10aa2147 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/RowDataFileWriterTest.java @@ -51,7 +51,7 @@ public class RowDataFileWriterTest { @TempDir java.nio.file.Path tempDir; @Test - public void testWriteBlobRefDataFile() throws Exception { + public void testWriteBlobRefFile() throws Exception { FileFormat fileFormat = FileFormat.fromIdentifier("parquet", new Options()); Path dataPath = new Path(tempDir.toUri().toString(), "data.parquet"); BlobReference reference = new BlobReference("default.upstream", 7, 11L); diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java index a787b2cc6547..70ce49389007 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java @@ -23,6 +23,7 @@ import org.apache.paimon.catalog.CatalogFactory; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.BlobReferenceResolver; import org.apache.paimon.data.GenericRow; @@ -102,7 +103,7 @@ public void testResolveByFieldIdAfterRename() throws Exception { .when(() -> CatalogFactory.createCatalog(context)) .thenReturn(catalog); - Blob resolved = BlobReferenceLookup.resolve(context, reference); + Blob resolved = BlobReferenceLookup.resolveBlob(context, reference); assertThat(resolved.toData()).isEqualTo(payload); } } @@ -175,10 +176,16 @@ public void testCreateResolverPreloadsDescriptors() throws Exception { BlobReferenceLookup.createResolver( context, java.util.Arrays.asList(leftReference, rightReference)); - assertThat(resolver.resolve(leftReference).toData()).isEqualTo(leftPayload); - assertThat(resolver.resolve(rightReference).toData()).isEqualTo(rightPayload); - assertThat(resolver.resolve(new BlobReference("default.source", 7, 12L)).toData()) - .isEqualTo(leftPayload); + BlobRef leftBlobRef = new BlobRef(leftReference); + BlobRef rightBlobRef = new BlobRef(rightReference); + resolver.resolve(leftBlobRef); + resolver.resolve(rightBlobRef); + assertThat(leftBlobRef.toData()).isEqualTo(leftPayload); + assertThat(rightBlobRef.toData()).isEqualTo(rightPayload); + + BlobRef anotherLeft = new BlobRef(new BlobReference("default.source", 7, 12L)); + resolver.resolve(anotherLeft); + assertThat(anotherLeft.toData()).isEqualTo(leftPayload); verify(table, times(1)).newReadBuilder(); } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java index d48becb7ed46..c9a1eec92b7f 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataGetters; import org.apache.paimon.data.Decimal; @@ -96,11 +97,11 @@ public class FieldWriterFactory implements AvroSchemaVisitor { private static final FieldWriter BLOB_REFERENCE_BYTES_WRITER = (container, i, encoder) -> { - Blob blob = container.getBlob(i); - if (blob == null) { + BlobRef blobRef = container.getBlobRef(i); + if (blobRef == null) { throw new IllegalArgumentException("Null blob_ref is not allowed."); } - encoder.writeBytes(BlobUtils.serializeBlobReference(blob)); + encoder.writeBytes(blobRef.reference().serialize()); }; @Override 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 8248937b5f14..c77f1339c5c3 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 @@ -269,9 +269,7 @@ public FieldWriter visit(BlobType blobType) { public FieldWriter visit(BlobRefType blobRefType) { return (rowId, column, getters, columnId) -> { BytesColumnVector vector = (BytesColumnVector) column; - byte[] bytes = - org.apache.paimon.data.BlobUtils.serializeBlobReference( - getters.getBlob(columnId)); + byte[] bytes = getters.getBlobRef(columnId).reference().serialize(); vector.setVal(rowId, bytes, 0, bytes.length); return bytes.length; }; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java index b668a0d912c9..1ddd00396242 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java @@ -352,18 +352,14 @@ private class BlobReferenceWriter implements FieldWriter { @Override public void write(InternalRow row, int ordinal) { - writeBlob(row.getBlob(ordinal)); + byte[] bytes = row.getBlobRef(ordinal).reference().serialize(); + recordConsumer.addBinary(Binary.fromReusedByteArray(bytes)); } @Override public void write(InternalArray arrayData, int ordinal) { throw new UnsupportedOperationException("BLOB_REF in array is not supported."); } - - private void writeBlob(Blob blob) { - byte[] bytes = BlobUtils.serializeBlobReference(blob); - recordConsumer.addBinary(Binary.fromReusedByteArray(bytes)); - } } private class IntWriter implements FieldWriter { diff --git a/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java index 797756224b2a..e14dc8d49d28 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.format.avro; import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; @@ -218,7 +219,7 @@ void testCompression() throws IOException { void testBlobRefRoundTrip() throws IOException { RowType rowType = DataTypes.ROW(DataTypes.FIELD(0, "blob_ref", DataTypes.BLOB_REF())); BlobReference reference = new BlobReference("default.t", 7, 11L); - Blob blob = Blob.fromReference(reference); + BlobRef blob = Blob.fromReference(reference); FileFormat format = new AvroFileFormat(new FormatContext(new Options(), 1024, 1024)); LocalFileIO fileIO = LocalFileIO.create(); From 3e792b1428a164f08608abccef56bae21a9d7a8f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Mon, 20 Apr 2026 09:11:01 +0800 Subject: [PATCH 10/19] Fix minus --- .../src/main/java/org/apache/paimon/data/BlobRef.java | 4 ++-- .../src/main/java/org/apache/paimon/data/GenericRow.java | 3 +-- .../org/apache/paimon/types/InternalRowToSizeVisitor.java | 1 - .../org/apache/paimon/format/avro/FieldWriterFactory.java | 1 - .../paimon/format/parquet/writer/ParquetRowDataWriter.java | 1 - 5 files changed, 3 insertions(+), 7 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobRef.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobRef.java index ae5fa5f1f8d8..aeefb2bebac2 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BlobRef.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobRef.java @@ -36,8 +36,8 @@ * *

For BLOB type: created via {@link #BlobRef(UriReader, BlobDescriptor)}, always resolved. * - *

For BLOB_REF type: created via {@link #BlobRef(BlobReference)}, initially unresolved. - * Call {@link #resolve(UriReader, BlobDescriptor)} to make it readable. + *

For BLOB_REF type: created via {@link #BlobRef(BlobReference)}, initially unresolved. Call + * {@link #resolve(UriReader, BlobDescriptor)} to make it readable. * * @since 1.4.0 */ 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 4ceb893ee04f..b3d5541d13ac 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 @@ -203,8 +203,7 @@ public BlobRef getBlobRef(int pos) { if (value instanceof BlobRef) { return (BlobRef) value; } - throw new ClassCastException( - "Cannot cast " + value.getClass().getName() + " to BlobRef"); + throw new ClassCastException("Cannot cast " + value.getClass().getName() + " to BlobRef"); } @Override 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 d69982f4836a..e3dab7d7517d 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 @@ -18,7 +18,6 @@ package org.apache.paimon.types; -import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataGetters; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java index c9a1eec92b7f..07856e6ca421 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldWriterFactory.java @@ -22,7 +22,6 @@ import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobDescriptor; import org.apache.paimon.data.BlobRef; -import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.DataGetters; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.GenericRow; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java index 1ddd00396242..43c6be2e2df5 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/ParquetRowDataWriter.java @@ -22,7 +22,6 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobDescriptor; -import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; From 9eb6530c0bf8747543bcb84f682454f39b0ca887 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Mon, 20 Apr 2026 10:07:42 +0800 Subject: [PATCH 11/19] Fix docs --- docs/layouts/shortcodes/generated/core_configuration.html | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 0071c264376f..1ad6b008ef98 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -86,6 +86,12 @@ String Specifies column names that should be stored as blob type. This is used when you want to treat a BYTES column as a BLOB. + +

blob-ref-field
+ (none) + String + Specifies column names that should be stored as blob reference type. This is used when you want to treat a BYTES column as a BLOB_REF. +
blob.split-by-file-size
(none) From cd083cbb0f0149d90fca6d95a2ff68376a0f0fb5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Mon, 20 Apr 2026 12:25:51 +0800 Subject: [PATCH 12/19] Fix test --- .../table/source/BlobRefResolvingRow.java | 4 ++-- .../paimon/utils/BlobReferenceLookup.java | 9 +++++-- .../apache/paimon/append/BlobTableTest.java | 24 ++++++++----------- .../paimon/format/blob/BlobFileMeta.java | 2 +- 4 files changed, 20 insertions(+), 19 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java b/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java index 2b7484b6c84b..b6aa4d2f0405 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/BlobRefResolvingRow.java @@ -34,8 +34,8 @@ import java.util.Set; /** - * {@link InternalRow} wrapper that resolves {@link UnresolvedBlob} to real {@link Blob} via a - * {@link BlobReferenceResolver} when {@link #getBlob(int)} is called. + * {@link InternalRow} wrapper that resolves UnresolvedBlob to real {@link Blob} via a {@link + * BlobReferenceResolver} when {@link #getBlob(int)} is called. */ class BlobRefResolvingRow implements InternalRow { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java index ffa1367367a6..f2de8f0e0aa5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java @@ -18,6 +18,7 @@ package org.apache.paimon.utils; +import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; @@ -160,7 +161,7 @@ private static Map preloadDescriptors( return loadReferencedDescriptors(catalog, references); } catch (Exception e) { LOG.warn("Failed to preload blob descriptors. Falling back to per-reference scan.", e); - return Collections.emptyMap(); + throw new RuntimeException(e); } } @@ -182,7 +183,11 @@ private static void loadTableDescriptors( TableReferences tableReferences, Map resolved) throws Exception { - Table table = catalog.getTable(Identifier.fromString(tableReferences.tableName)); + Table table = + catalog.getTable(Identifier.fromString(tableReferences.tableName)) + .copy( + Collections.singletonMap( + CoreOptions.BLOB_AS_DESCRIPTOR.key(), "true")); List fields = new ArrayList<>(tableReferences.referencesByField.size()); TreeSet rowIds = new TreeSet<>(); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java index e12540f8c73f..f71e33a85911 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java @@ -829,8 +829,8 @@ public void testBlobRefE2E() throws Exception { // 4. Write blob references using actual row IDs from upstream String upstreamFullName = database + "." + upstreamTableName; - BlobReference ref1 = new BlobReference(upstreamFullName, imageFieldId, idToRowId.get(1)); - BlobReference ref2 = new BlobReference(upstreamFullName, imageFieldId, idToRowId.get(2)); + BlobReference ref1 = new BlobReference(upstreamFullName, imageFieldId, 0L); + BlobReference ref2 = new BlobReference(upstreamFullName, imageFieldId, 1L); BatchWriteBuilder downstreamWriteBuilder = downstreamTable.newBatchWriteBuilder(); try (BatchTableWrite write = downstreamWriteBuilder.newWrite(); @@ -844,21 +844,17 @@ public void testBlobRefE2E() throws Exception { // 5. Read downstream table — blob references should resolve from upstream ReadBuilder downstreamReadBuilder = downstreamTable.newReadBuilder(); - List downstreamRows = new ArrayList<>(); downstreamReadBuilder .newRead() .createReader(downstreamReadBuilder.newScan().plan()) - .forEachRemaining(downstreamRows::add); - - assertThat(downstreamRows.size()).isEqualTo(2); - - for (InternalRow row : downstreamRows) { - int id = row.getInt(0); - BlobRef blobRef = row.getBlobRef(2); - assertThat(blobRef).isNotNull(); - assertThat(blobRef.isResolved()).isTrue(); - assertThat(blobRef.toData()).isEqualTo(idToBlob.get(id)); - } + .forEachRemaining( + row -> { + int id = row.getInt(0); + BlobRef blobRef = row.getBlobRef(2); + assertThat(blobRef).isNotNull(); + assertThat(blobRef.isResolved()).isTrue(); + assertThat(blobRef.toData()).isEqualTo(idToBlob.get(id)); + }); } private void createExternalStorageTable() throws Exception { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/blob/BlobFileMeta.java b/paimon-format/src/main/java/org/apache/paimon/format/blob/BlobFileMeta.java index 02579b959793..27a1ed56170a 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/blob/BlobFileMeta.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/blob/BlobFileMeta.java @@ -98,7 +98,7 @@ public long blobOffset(int i) { } public int returnedPosition(int i) { - return returnedPositions == null ? i : returnedPositions[i - 1]; + return returnedPositions == null ? i - 1 : returnedPositions[i - 1]; } public int recordNumber() { From 13a0bb01b115da48d3c20757a39770f2efae295a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Mon, 20 Apr 2026 14:21:10 +0800 Subject: [PATCH 13/19] Fix minus --- .../org/apache/paimon/data/BlobReference.java | 11 ++ .../paimon/utils/BlobReferenceLookup.java | 112 +++--------------- .../apache/paimon/append/BlobTableTest.java | 4 +- .../paimon/utils/BlobReferenceLookupTest.java | 48 ++------ 4 files changed, 36 insertions(+), 139 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BlobReference.java b/paimon-common/src/main/java/org/apache/paimon/data/BlobReference.java index d4a7926bd514..028db0dad514 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BlobReference.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BlobReference.java @@ -142,4 +142,15 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(tableName, fieldId, rowId); } + + @Override + public String toString() { + return "BlobReference{table=" + + tableName + + ", fieldId=" + + fieldId + + ", rowId=" + + rowId + + "}"; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java index f2de8f0e0aa5..c926b532a0aa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java @@ -36,9 +36,6 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.RowType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -49,119 +46,42 @@ import java.util.TreeSet; /** - * Utilities for resolving {@link BlobReference} through table metadata. - * - *

The preload phase only caches lightweight {@link BlobDescriptor} (uri + offset + length) - * rather than the actual blob data, so memory usage stays small even when a data file contains a - * large number of blob references. + * Batch-preloads {@link BlobDescriptor}s for a set of {@link BlobReference}s by scanning the + * upstream tables once. The preloaded descriptors are lightweight (uri + offset + length) so memory + * stays small even for large numbers of references. */ public class BlobReferenceLookup { - private static final Logger LOG = LoggerFactory.getLogger(BlobReferenceLookup.class); - /** - * Creates a resolver backed by a preloaded descriptor cache. The given references are batch - * scanned from the upstream tables to obtain their {@link BlobDescriptor}s. On a cache hit the - * descriptor is used to construct a {@link Blob} directly. On a cache miss the resolver falls - * back to scanning the upstream table individually. + * Creates a resolver that resolves {@link BlobRef}s using a preloaded descriptor cache. All + * given references are batch-scanned from the upstream tables upfront. */ public static BlobReferenceResolver createResolver( CatalogContext catalogContext, List references) { - if (references.isEmpty()) { - return createResolver(catalogContext); - } Map cached = preloadDescriptors(catalogContext, references); UriReaderFactory uriReaderFactory = new UriReaderFactory(catalogContext); - if (cached.isEmpty()) { - return createResolver(catalogContext); - } return blobRef -> { BlobDescriptor descriptor = cached.get(blobRef.reference()); - if (descriptor != null) { - blobRef.resolve(uriReaderFactory.create(descriptor.uri()), descriptor); - } else { - resolveFromUpstream(catalogContext, blobRef); + if (descriptor == null) { + throw new IllegalStateException( + "BlobReference not found in preloaded cache: " + + blobRef.reference() + + ". Cache keys: " + + cached.keySet()); } + blobRef.resolve(uriReaderFactory.create(descriptor.uri()), descriptor); }; } - /** Creates a simple resolver that scans the upstream table for each reference individually. */ - public static BlobReferenceResolver createResolver(CatalogContext catalogContext) { - return blobRef -> resolveFromUpstream(catalogContext, blobRef); - } - - private static void resolveFromUpstream(CatalogContext catalogContext, BlobRef blobRef) { - BlobReference reference = blobRef.reference(); - Blob resolved = resolveBlob(catalogContext, reference); - try { - BlobDescriptor desc = resolved.toDescriptor(); - UriReaderFactory uriReaderFactory = new UriReaderFactory(catalogContext); - blobRef.resolve(uriReaderFactory.create(desc.uri()), desc); - } catch (Exception e) { - // Inline blob — store the raw data directly - byte[] data = resolved.toData(); - blobRef.resolve( - uri -> new org.apache.paimon.fs.ByteArraySeekableStream(data), - new BlobDescriptor("memory://inline", 0, data.length)); - } - } - - /** Scans the upstream table to get the blob for a single reference. */ - static Blob resolveBlob(CatalogContext catalogContext, BlobReference reference) { - try (Catalog catalog = CatalogFactory.createCatalog(catalogContext)) { - Table table = catalog.getTable(Identifier.fromString(reference.tableName())); - if (!table.rowType().containsField(reference.fieldId())) { - throw new IllegalArgumentException( - "Cannot find blob fieldId " - + reference.fieldId() - + " in upstream table " - + reference.tableName() - + "."); - } - int fieldPos = table.rowType().getFieldIndexByFieldId(reference.fieldId()); - - ReadBuilder readBuilder = - table.newReadBuilder() - .withProjection(new int[] {fieldPos}) - .withRowRanges( - Collections.singletonList( - new Range(reference.rowId(), reference.rowId()))); - - try (RecordReader reader = - readBuilder.newRead().createReader(readBuilder.newScan().plan())) { - RecordReader.RecordIterator batch; - while ((batch = reader.readBatch()) != null) { - try { - InternalRow row; - while ((row = batch.next()) != null) { - return row.getBlob(0); - } - } finally { - batch.releaseBatch(); - } - } - } - - throw new IllegalStateException( - "Cannot resolve blob reference for table " - + reference.tableName() - + ", rowId " - + reference.rowId() - + ", fieldId " - + reference.fieldId() - + "."); - } catch (Exception e) { - throw new RuntimeException("Failed to resolve blob reference.", e); - } - } - private static Map preloadDescriptors( CatalogContext catalogContext, List references) { + if (references.isEmpty()) { + return Collections.emptyMap(); + } try (Catalog catalog = CatalogFactory.createCatalog(catalogContext)) { return loadReferencedDescriptors(catalog, references); } catch (Exception e) { - LOG.warn("Failed to preload blob descriptors. Falling back to per-reference scan.", e); - throw new RuntimeException(e); + throw new RuntimeException("Failed to preload blob descriptors.", e); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java index f71e33a85911..fa1cafe8c451 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BlobTableTest.java @@ -829,8 +829,8 @@ public void testBlobRefE2E() throws Exception { // 4. Write blob references using actual row IDs from upstream String upstreamFullName = database + "." + upstreamTableName; - BlobReference ref1 = new BlobReference(upstreamFullName, imageFieldId, 0L); - BlobReference ref2 = new BlobReference(upstreamFullName, imageFieldId, 1L); + BlobReference ref1 = new BlobReference(upstreamFullName, imageFieldId, idToRowId.get(1)); + BlobReference ref2 = new BlobReference(upstreamFullName, imageFieldId, idToRowId.get(2)); BatchWriteBuilder downstreamWriteBuilder = downstreamTable.newBatchWriteBuilder(); try (BatchTableWrite write = downstreamWriteBuilder.newWrite(); diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java index 70ce49389007..c4dfb191c443 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java @@ -57,6 +57,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -67,47 +68,6 @@ public class BlobReferenceLookupTest { @TempDir java.nio.file.Path tempDir; - @Test - public void testResolveByFieldIdAfterRename() throws Exception { - CatalogContext context = mock(CatalogContext.class); - Catalog catalog = mock(Catalog.class); - Table table = mock(Table.class); - ReadBuilder readBuilder = mock(ReadBuilder.class); - TableScan scan = mock(TableScan.class); - TableScan.Plan plan = mock(TableScan.Plan.class); - - byte[] payload = new byte[] {1, 2, 3}; - InternalRow row = GenericRow.of(Blob.fromData(payload)); - Split split = new TestSplit(); - - when(catalog.getTable(any())).thenReturn(table); - when(table.rowType()) - .thenReturn( - new RowType( - Collections.singletonList( - new DataField(7, "blob_renamed", DataTypes.BLOB())))); - when(table.newReadBuilder()).thenReturn(readBuilder); - when(readBuilder.withProjection(any(int[].class))).thenReturn(readBuilder); - when(readBuilder.withRowRanges(anyList())).thenReturn(readBuilder); - when(readBuilder.newRead()) - .thenReturn(new ListRowTableRead(split, Collections.singletonList(row))); - when(readBuilder.newScan()).thenReturn(scan); - when(scan.plan()).thenReturn(plan); - when(plan.splits()).thenReturn(Collections.singletonList(split)); - - BlobReference reference = new BlobReference("default.source", 7, 12L); - - try (MockedStatic mockedCatalogFactory = - Mockito.mockStatic(CatalogFactory.class)) { - mockedCatalogFactory - .when(() -> CatalogFactory.createCatalog(context)) - .thenReturn(catalog); - - Blob resolved = BlobReferenceLookup.resolveBlob(context, reference); - assertThat(resolved.toData()).isEqualTo(payload); - } - } - @Test public void testCreateResolverPreloadsDescriptors() throws Exception { CatalogContext context = CatalogContext.create(new Path(tempDir.toUri().toString())); @@ -117,6 +77,7 @@ public void testCreateResolverPreloadsDescriptors() throws Exception { TableScan scan = mock(TableScan.class); TableScan.Plan plan = mock(TableScan.Plan.class); + // Write real blob files byte[] leftPayload = new byte[] {1, 2, 3}; byte[] rightPayload = new byte[] {4, 5, 6}; Path leftBlobPath = new Path(tempDir.toUri().toString(), "left.blob"); @@ -148,6 +109,7 @@ public void testCreateResolverPreloadsDescriptors() throws Exception { InternalRow preloadRow = GenericRow.of(leftBlob, rightBlob, 12L); when(catalog.getTable(any())).thenReturn(table); + when(table.copy(anyMap())).thenReturn(table); when(table.rowType()) .thenReturn( new RowType( @@ -180,13 +142,17 @@ public void testCreateResolverPreloadsDescriptors() throws Exception { BlobRef rightBlobRef = new BlobRef(rightReference); resolver.resolve(leftBlobRef); resolver.resolve(rightBlobRef); + assertThat(leftBlobRef.isResolved()).isTrue(); + assertThat(rightBlobRef.isResolved()).isTrue(); assertThat(leftBlobRef.toData()).isEqualTo(leftPayload); assertThat(rightBlobRef.toData()).isEqualTo(rightPayload); + // Same coordinates → same result BlobRef anotherLeft = new BlobRef(new BlobReference("default.source", 7, 12L)); resolver.resolve(anotherLeft); assertThat(anotherLeft.toData()).isEqualTo(leftPayload); + // Only one readBuilder should have been created (batch preload) verify(table, times(1)).newReadBuilder(); } } From ef66dd3ca3822318040b8dd20bfa2395680cd3ee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Mon, 20 Apr 2026 14:32:51 +0800 Subject: [PATCH 14/19] Fix minus --- .../paimon/table/source/AbstractDataTableRead.java | 9 --------- .../apache/paimon/table/source/KeyValueTableRead.java | 10 ---------- 2 files changed, 19 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java index b3aa9524ed73..4492ed5d4518 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java @@ -75,14 +75,6 @@ public AbstractDataTableRead( public abstract RecordReader reader(Split split) throws IOException; - /** - * Hook for subclasses to transfer additional runtime state (e.g. forceKeepDelete, ioManager) to - * the independent pre-scan read used for BLOB_REF reference collection. Called after - * withReadType and withFilter have already been set. Subclasses should NOT set limit or topN on - * the prescan read, as the pre-scan must cover all rows. - */ - protected void configurePrescanRead(InnerTableRead prescanRead) {} - @Override public TableRead withIOManager(IOManager ioManager) { return this; @@ -169,7 +161,6 @@ private RecordReader createBlobRefReader( if (predicate != null) { prescanRead.withFilter(predicate); } - configurePrescanRead(prescanRead); Split prescanSplit = authResult != null ? new QueryAuthSplit(split, authResult) : split; LinkedHashSet references = new LinkedHashSet<>(); RecordReader prescanReader = prescanRead.createReader(prescanSplit); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java index b5dd6cef309c..1d9a4d9b51b5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java @@ -103,16 +103,6 @@ private void config(SplitRead read) { read.withFilter(predicate).withIOManager(ioManager); } - @Override - protected void configurePrescanRead(InnerTableRead prescanRead) { - if (forceKeepDelete) { - prescanRead.forceKeepDelete(); - } - if (ioManager != null) { - prescanRead.withIOManager(ioManager); - } - } - @Override public void applyReadType(RowType readType) { initialized().forEach(r -> r.withReadType(readType)); From 53f5fd7595e2efc2da62fcfdf6dd3ad9a61e78d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Tue, 21 Apr 2026 11:10:22 +0800 Subject: [PATCH 15/19] Flink Support --- .../paimon/flink/FlinkRowDataWithBlob.java | 13 ++++- .../apache/paimon/flink/FlinkRowWrapper.java | 8 ++++ .../flink/function/BlobReferenceFunction.java | 38 +++++++++++++++ .../flink/function/BuiltInFunctions.java | 1 + .../source/FileStoreSourceSplitReader.java | 19 +++++--- .../apache/paimon/flink/BlobTableITCase.java | 48 +++++++++++++++++++ 6 files changed, 119 insertions(+), 8 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/function/BlobReferenceFunction.java diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowDataWithBlob.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowDataWithBlob.java index dc38588a7ce4..5b70fe6b8f93 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowDataWithBlob.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkRowDataWithBlob.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink; import org.apache.paimon.data.Blob; +import org.apache.paimon.data.BlobRef; import org.apache.paimon.data.InternalRow; import java.util.Set; @@ -27,18 +28,26 @@ public class FlinkRowDataWithBlob extends FlinkRowData { private final Set blobFields; + private final Set blobRefFields; private final boolean blobAsDescriptor; public FlinkRowDataWithBlob( - InternalRow row, Set blobFields, boolean blobAsDescriptor) { + InternalRow row, + Set blobFields, + Set blobRefFields, + boolean blobAsDescriptor) { super(row); this.blobFields = blobFields; + this.blobRefFields = blobRefFields; this.blobAsDescriptor = blobAsDescriptor; } @Override public byte[] getBinary(int pos) { - if (blobFields.contains(pos)) { + if (blobRefFields.contains(pos)) { + BlobRef blobRef = row.getBlobRef(pos); + return blobAsDescriptor ? blobRef.toDescriptor().serialize() : blobRef.toData(); + } else if (blobFields.contains(pos)) { Blob blob = row.getBlob(pos); return blobAsDescriptor ? blob.toDescriptor().serialize() : blob.toData(); } else { 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 71f5304643a7..9ef6e1719fa4 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 @@ -22,6 +22,8 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobData; +import org.apache.paimon.data.BlobRef; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; @@ -145,6 +147,12 @@ public Blob getBlob(int pos) { return BlobUtils.fromBytes(row.getBinary(pos), uriReaderFactory, null); } + @Override + public BlobRef getBlobRef(int pos) { + byte[] bytes = row.getBinary(pos); + return new BlobRef(BlobReference.deserialize(bytes)); + } + @Override public InternalArray getArray(int pos) { return new FlinkArrayWrapper(row.getArray(pos)); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/function/BlobReferenceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/function/BlobReferenceFunction.java new file mode 100644 index 000000000000..d2a54cd3146d --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/function/BlobReferenceFunction.java @@ -0,0 +1,38 @@ +/* + * 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.function; + +import org.apache.paimon.data.BlobReference; + +import org.apache.flink.table.functions.ScalarFunction; + +/** + * Flink scalar function that constructs a serialized {@link BlobReference}. + * + *

Usage: {@code sys.blob_reference('db.table', 7, 11)} + */ +public class BlobReferenceFunction extends ScalarFunction { + + public byte[] eval(String tableName, int fieldId, long rowId) { + if (tableName == null) { + return null; + } + return new BlobReference(tableName, fieldId, rowId).serialize(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/function/BuiltInFunctions.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/function/BuiltInFunctions.java index a6a94faf6141..4b9965117974 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/function/BuiltInFunctions.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/function/BuiltInFunctions.java @@ -29,6 +29,7 @@ public class BuiltInFunctions { { put("path_to_descriptor", PathToDescriptor.class.getName()); put("descriptor_to_string", DescriptorToString.class.getName()); + put("blob_reference", BlobReferenceFunction.class.getName()); } }; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitReader.java index eff22090735c..0938184a04ec 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitReader.java @@ -269,16 +269,22 @@ private class FileStoreRecordIterator implements BulkFormat.RecordIterator recordAndPosition = new MutableRecordAndPosition<>(); private final Set blobFields; + private final Set blobRefFields; private FileStoreRecordIterator(@Nullable RowType rowType) { - this.blobFields = rowType == null ? Collections.emptySet() : blobFieldIndex(rowType); + if (rowType == null) { + this.blobFields = Collections.emptySet(); + this.blobRefFields = Collections.emptySet(); + } else { + this.blobFields = fieldIndexByType(rowType, DataTypeRoot.BLOB); + this.blobRefFields = fieldIndexByType(rowType, DataTypeRoot.BLOB_REF); + } } - private Set blobFieldIndex(RowType rowType) { + private Set fieldIndexByType(RowType rowType, DataTypeRoot typeRoot) { Set result = new HashSet<>(); for (int i = 0; i < rowType.getFieldCount(); i++) { - if (rowType.getTypeAt(i).getTypeRoot() == DataTypeRoot.BLOB - || rowType.getTypeAt(i).getTypeRoot() == DataTypeRoot.BLOB_REF) { + if (rowType.getTypeAt(i).getTypeRoot() == typeRoot) { result.add(i); } } @@ -308,9 +314,10 @@ public RecordAndPosition next() { } recordAndPosition.setNext( - blobFields.isEmpty() + blobFields.isEmpty() && blobRefFields.isEmpty() ? new FlinkRowData(row) - : new FlinkRowDataWithBlob(row, blobFields, blobAsDescriptor)); + : new FlinkRowDataWithBlob( + row, blobFields, blobRefFields, blobAsDescriptor)); currentNumRead++; if (limiter != null) { limiter.increment(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BlobTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BlobTableITCase.java index e4900207095f..ac282f48bfed 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BlobTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BlobTableITCase.java @@ -374,6 +374,54 @@ public void testBlobTypeSchemaEquals() throws Exception { assertThat(AbstractFlinkTableFactory.schemaEquals(convertedRowType, flinkRowType)).isTrue(); } + @Test + public void testWriteBlobRefWithBuiltInFunction() throws Exception { + // 1. Create upstream blob table and write data + tEnv.executeSql( + "CREATE TABLE upstream_blob (id INT, name STRING, picture BYTES)" + + " WITH ('row-tracking.enabled'='true'," + + " 'data-evolution.enabled'='true'," + + " 'blob-field'='picture')"); + batchSql("INSERT INTO upstream_blob VALUES (1, 'row1', X'48656C6C6F')"); + batchSql("INSERT INTO upstream_blob VALUES (2, 'row2', X'5945')"); + + // 2. Get fieldId for the "picture" column from Paimon schema + FileStoreTable upstreamTable = paimonTable("upstream_blob"); + int pictureFieldId = + upstreamTable.rowType().getFields().stream() + .filter(f -> f.name().equals("picture")) + .findFirst() + .orElseThrow(() -> new RuntimeException("picture field not found")) + .id(); + + // 3. Create downstream blob_ref table + String fullTableName = tEnv.getCurrentDatabase() + ".upstream_blob"; + tEnv.executeSql( + "CREATE TABLE downstream_ref (id INT, label STRING, image_ref BYTES)" + + " WITH ('row-tracking.enabled'='true'," + + " 'data-evolution.enabled'='true'," + + " 'blob-ref-field'='image_ref')"); + + // 4. Insert by reading _ROW_ID from $row_tracking directly in the INSERT + batchSql( + String.format( + "INSERT INTO downstream_ref" + + " SELECT id, name, sys.blob_reference('%s', %d, _ROW_ID)" + + " FROM `upstream_blob$row_tracking`", + fullTableName, pictureFieldId)); + + // 5. Read back — blob references should resolve to upstream blob data + List result = batchSql("SELECT * FROM downstream_ref ORDER BY id"); + assertThat(result).hasSize(2); + assertThat(result.get(0).getField(0)).isEqualTo(1); + assertThat(result.get(0).getField(1)).isEqualTo("row1"); + assertThat((byte[]) result.get(0).getField(2)) + .isEqualTo(new byte[] {72, 101, 108, 108, 111}); + assertThat(result.get(1).getField(0)).isEqualTo(2); + assertThat(result.get(1).getField(1)).isEqualTo("row2"); + assertThat((byte[]) result.get(1).getField(2)).isEqualTo(new byte[] {89, 69}); + } + private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); public static String bytesToHex(byte[] bytes) { From 7c3a294d302150ce6a4da940f8a9f05f3d56f9b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Tue, 21 Apr 2026 18:19:27 +0800 Subject: [PATCH 16/19] Uupdate file io --- .../paimon/utils/BlobReferenceLookup.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java index c926b532a0aa..2ec94009743b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BlobReferenceLookup.java @@ -59,7 +59,8 @@ public class BlobReferenceLookup { public static BlobReferenceResolver createResolver( CatalogContext catalogContext, List references) { Map cached = preloadDescriptors(catalogContext, references); - UriReaderFactory uriReaderFactory = new UriReaderFactory(catalogContext); + Catalog catalog = CatalogFactory.createCatalog(catalogContext); + Map cache = new HashMap<>(); return blobRef -> { BlobDescriptor descriptor = cached.get(blobRef.reference()); if (descriptor == null) { @@ -69,7 +70,19 @@ public static BlobReferenceResolver createResolver( + ". Cache keys: " + cached.keySet()); } - blobRef.resolve(uriReaderFactory.create(descriptor.uri()), descriptor); + UriReader uriReader = + cache.computeIfAbsent( + blobRef.reference().tableName(), + tableName -> { + try { + return UriReader.fromFile( + catalog.getTable(Identifier.fromString(tableName)) + .fileIO()); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + blobRef.resolve(uriReader, descriptor); }; } From aa451dac0010d3531fa5ce559adb05070fdb62e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Tue, 21 Apr 2026 19:07:30 +0800 Subject: [PATCH 17/19] Fix test --- .../java/org/apache/paimon/utils/BlobReferenceLookupTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java index c4dfb191c443..a40825756df3 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/BlobReferenceLookupTest.java @@ -134,6 +134,8 @@ public void testCreateResolverPreloadsDescriptors() throws Exception { .when(() -> CatalogFactory.createCatalog(context)) .thenReturn(catalog); + when(table.fileIO()).thenReturn(LocalFileIO.create()); + BlobReferenceResolver resolver = BlobReferenceLookup.createResolver( context, java.util.Arrays.asList(leftReference, rightReference)); From 201e3713484ca29635fe0cd0fddc22d0cf8aee5f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Tue, 21 Apr 2026 20:03:01 +0800 Subject: [PATCH 18/19] Spark support --- .../format/avro/FieldReaderFactory.java | 20 +++++++++- .../paimon/spark/SparkInternalRowWrapper.java | 12 ++++++ .../paimon/spark/sql/BlobTestBase.scala | 40 +++++++++++++++++++ .../data/Spark3InternalRowWithBlob.scala | 18 +++++---- .../data/Spark4InternalRowWithBlob.scala | 18 +++++---- 5 files changed, 93 insertions(+), 15 deletions(-) diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldReaderFactory.java index 47f12d45684b..0f63e7c7339c 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/FieldReaderFactory.java @@ -22,6 +22,8 @@ import org.apache.paimon.data.BinaryVector; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobDescriptor; +import org.apache.paimon.data.BlobRef; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.GenericArray; import org.apache.paimon.data.GenericMap; @@ -69,6 +71,8 @@ public FieldReaderFactory(@Nullable UriReader uriReader) { private static final FieldReader BYTES_READER = new BytesReader(); + private static final FieldReader BLOB_REFERENCE_READER = new BlobReferenceBytesReader(); + private static final FieldReader BOOLEAN_READER = new BooleanReader(); private static final FieldReader TINYINT_READER = new TinyIntReader(); @@ -94,7 +98,7 @@ public FieldReader primitive(Schema primitive, DataType type) { return new BlobDescriptorBytesReader(uriReader); } if (type.getTypeRoot() == DataTypeRoot.BLOB_REF) { - return BYTES_READER; + return BLOB_REFERENCE_READER; } } return AvroSchemaVisitor.super.primitive(primitive, type); @@ -288,6 +292,20 @@ public void skip(Decoder decoder) throws IOException { } } + private static class BlobReferenceBytesReader implements FieldReader { + + @Override + public Object read(Decoder decoder, Object reuse) throws IOException { + byte[] bytes = decoder.readBytes(null).array(); + return new BlobRef(BlobReference.deserialize(bytes)); + } + + @Override + public void skip(Decoder decoder) throws IOException { + decoder.skipBytes(); + } + } + private static class BooleanReader implements FieldReader { @Override 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 14b7331be319..15d865f82b05 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 @@ -22,6 +22,8 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Blob; import org.apache.paimon.data.BlobData; +import org.apache.paimon.data.BlobRef; +import org.apache.paimon.data.BlobReference; import org.apache.paimon.data.BlobUtils; import org.apache.paimon.data.Decimal; import org.apache.paimon.data.InternalArray; @@ -248,6 +250,16 @@ public Blob getBlob(int pos) { return BlobUtils.fromBytes(internalRow.getBinary(actualPos), uriReaderFactory, null); } + @Override + public BlobRef getBlobRef(int pos) { + int actualPos = getActualFieldPosition(pos); + if (actualPos == -1 || internalRow.isNullAt(actualPos)) { + return null; + } + byte[] bytes = internalRow.getBinary(actualPos); + return new BlobRef(BlobReference.deserialize(bytes)); + } + @Override public InternalArray getArray(int pos) { int actualPos = getActualFieldPosition(pos); diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BlobTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BlobTestBase.scala index 86e0a0dce2f9..4fa8b8aab969 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BlobTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BlobTestBase.scala @@ -32,6 +32,8 @@ import org.apache.spark.sql.Row import java.util import java.util.Random +import scala.collection.JavaConverters._ + class BlobTestBase extends PaimonSparkTestBase { private val RANDOM = new Random @@ -314,6 +316,44 @@ class BlobTestBase extends PaimonSparkTestBase { } } + test("BlobRef: test write and read blob reference") { + withTable("upstream", "downstream") { + // 1. Create upstream blob table and write data + sql( + "CREATE TABLE upstream (id INT, name STRING, picture BINARY) TBLPROPERTIES " + + "('row-tracking.enabled'='true', 'data-evolution.enabled'='true', 'blob-field'='picture')") + sql("INSERT INTO upstream VALUES (1, 'row1', X'48656C6C6F')") + sql("INSERT INTO upstream VALUES (2, 'row2', X'5945')") + + // 2. Get fieldId for the "picture" column + val upstreamTable = loadTable("upstream") + val pictureFieldId = upstreamTable.rowType().getFields.asScala + .find(_.name() == "picture") + .map(_.id()) + .getOrElse(throw new RuntimeException("picture field not found")) + + // 3. Create downstream blob_ref table + val fullTableName = s"$dbName0.upstream" + sql( + "CREATE TABLE downstream (id INT, label STRING, image_ref BINARY) TBLPROPERTIES " + + "('row-tracking.enabled'='true', 'data-evolution.enabled'='true', 'blob-ref-field'='image_ref')") + + // 4. Insert by reading _ROW_ID from $row_tracking directly + sql( + s"INSERT INTO downstream " + + s"SELECT id, name, sys.blob_reference('$fullTableName', $pictureFieldId, _ROW_ID) " + + s"FROM `upstream$$row_tracking`") + + // 5. Read back — blob references should resolve to upstream blob data + checkAnswer( + sql("SELECT * FROM downstream ORDER BY id"), + Seq( + Row(1, "row1", Array[Byte](72, 101, 108, 108, 111)), + Row(2, "row2", Array[Byte](89, 69))) + ) + } + } + private val HEX_ARRAY = "0123456789ABCDEF".toCharArray def bytesToHex(bytes: Array[Byte]): String = { diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRowWithBlob.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRowWithBlob.scala index 6c1dbd9d2120..c44411bceec1 100644 --- a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRowWithBlob.scala +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRowWithBlob.scala @@ -18,7 +18,7 @@ package org.apache.paimon.spark.data -import org.apache.paimon.types.RowType +import org.apache.paimon.types.{DataTypeRoot, RowType} import org.apache.paimon.utils.InternalRowUtils.copyInternalRow import org.apache.spark.sql.catalyst.InternalRow @@ -26,13 +26,17 @@ import org.apache.spark.sql.catalyst.InternalRow class Spark3InternalRowWithBlob(rowType: RowType, blobFields: Set[Int], blobAsDescriptor: Boolean) extends Spark3InternalRow(rowType) { + private val blobRefFields: Set[Int] = blobFields.filter( + i => rowType.getTypeAt(i).getTypeRoot.equals(DataTypeRoot.BLOB_REF)) + private val pureBlobFields: Set[Int] = blobFields -- blobRefFields + override def getBinary(ordinal: Int): Array[Byte] = { - if (blobFields.contains(ordinal)) { - if (blobAsDescriptor) { - row.getBlob(ordinal).toDescriptor.serialize() - } else { - row.getBlob(ordinal).toData - } + if (blobRefFields.contains(ordinal)) { + val blobRef = row.getBlobRef(ordinal) + if (blobAsDescriptor) blobRef.toDescriptor.serialize() else blobRef.toData + } else if (pureBlobFields.contains(ordinal)) { + val blob = row.getBlob(ordinal) + if (blobAsDescriptor) blob.toDescriptor.serialize() else blob.toData } else { super.getBinary(ordinal) } diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRowWithBlob.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRowWithBlob.scala index 2a120e5b4c2a..aef1c97de813 100644 --- a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRowWithBlob.scala +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRowWithBlob.scala @@ -19,7 +19,7 @@ package org.apache.paimon.spark.data import org.apache.paimon.spark.AbstractSparkInternalRow -import org.apache.paimon.types.RowType +import org.apache.paimon.types.{DataTypeRoot, RowType} import org.apache.paimon.utils.InternalRowUtils.copyInternalRow import org.apache.spark.sql.catalyst.InternalRow @@ -28,13 +28,17 @@ import org.apache.spark.unsafe.types.VariantVal class Spark4InternalRowWithBlob(rowType: RowType, blobFields: Set[Int], blobAsDescriptor: Boolean) extends Spark4InternalRow(rowType) { + private val blobRefFields: Set[Int] = blobFields.filter( + i => rowType.getTypeAt(i).getTypeRoot.equals(DataTypeRoot.BLOB_REF)) + private val pureBlobFields: Set[Int] = blobFields -- blobRefFields + override def getBinary(ordinal: Int): Array[Byte] = { - if (blobFields.contains(ordinal)) { - if (blobAsDescriptor) { - row.getBlob(ordinal).toDescriptor.serialize() - } else { - row.getBlob(ordinal).toData - } + if (blobRefFields.contains(ordinal)) { + val blobRef = row.getBlobRef(ordinal) + if (blobAsDescriptor) blobRef.toDescriptor.serialize() else blobRef.toData + } else if (pureBlobFields.contains(ordinal)) { + val blob = row.getBlob(ordinal) + if (blobAsDescriptor) blob.toDescriptor.serialize() else blob.toData } else { super.getBinary(ordinal) } From 225cf6b5de7ba739765149eb5a5bc7b76b6ab7a0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BB=9F=E5=BC=8B?= Date: Tue, 21 Apr 2026 22:26:47 +0800 Subject: [PATCH 19/19] Fix minus --- .../apache/paimon/spark/sql/BlobTestBase.scala | 15 ++++++++------- .../spark/data/Spark3InternalRowWithBlob.scala | 4 ++-- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BlobTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BlobTestBase.scala index 4fa8b8aab969..478c1736f066 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BlobTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BlobTestBase.scala @@ -319,24 +319,25 @@ class BlobTestBase extends PaimonSparkTestBase { test("BlobRef: test write and read blob reference") { withTable("upstream", "downstream") { // 1. Create upstream blob table and write data - sql( - "CREATE TABLE upstream (id INT, name STRING, picture BINARY) TBLPROPERTIES " + - "('row-tracking.enabled'='true', 'data-evolution.enabled'='true', 'blob-field'='picture')") + sql("CREATE TABLE upstream (id INT, name STRING, picture BINARY) TBLPROPERTIES " + + "('row-tracking.enabled'='true', 'data-evolution.enabled'='true', 'blob-field'='picture')") sql("INSERT INTO upstream VALUES (1, 'row1', X'48656C6C6F')") sql("INSERT INTO upstream VALUES (2, 'row2', X'5945')") // 2. Get fieldId for the "picture" column val upstreamTable = loadTable("upstream") - val pictureFieldId = upstreamTable.rowType().getFields.asScala + val pictureFieldId = upstreamTable + .rowType() + .getFields + .asScala .find(_.name() == "picture") .map(_.id()) .getOrElse(throw new RuntimeException("picture field not found")) // 3. Create downstream blob_ref table val fullTableName = s"$dbName0.upstream" - sql( - "CREATE TABLE downstream (id INT, label STRING, image_ref BINARY) TBLPROPERTIES " + - "('row-tracking.enabled'='true', 'data-evolution.enabled'='true', 'blob-ref-field'='image_ref')") + sql("CREATE TABLE downstream (id INT, label STRING, image_ref BINARY) TBLPROPERTIES " + + "('row-tracking.enabled'='true', 'data-evolution.enabled'='true', 'blob-ref-field'='image_ref')") // 4. Insert by reading _ROW_ID from $row_tracking directly sql( diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRowWithBlob.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRowWithBlob.scala index c44411bceec1..3a43ffefb737 100644 --- a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRowWithBlob.scala +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRowWithBlob.scala @@ -26,8 +26,8 @@ import org.apache.spark.sql.catalyst.InternalRow class Spark3InternalRowWithBlob(rowType: RowType, blobFields: Set[Int], blobAsDescriptor: Boolean) extends Spark3InternalRow(rowType) { - private val blobRefFields: Set[Int] = blobFields.filter( - i => rowType.getTypeAt(i).getTypeRoot.equals(DataTypeRoot.BLOB_REF)) + private val blobRefFields: Set[Int] = + blobFields.filter(i => rowType.getTypeAt(i).getTypeRoot.equals(DataTypeRoot.BLOB_REF)) private val pureBlobFields: Set[Int] = blobFields -- blobRefFields override def getBinary(ordinal: Int): Array[Byte] = {