From 80b5d573ca4b3bd9e36043a904e56ecf2438632e Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Tue, 31 Mar 2026 16:15:32 +0800 Subject: [PATCH] [common] Support variant type --- docs/variant-shredding-design.md | 878 +++++++++++++ .../fluss/client/table/scanner/Scan.java | 14 + .../fluss/client/table/scanner/TableScan.java | 57 +- .../client/table/scanner/log/LogFetcher.java | 22 +- .../client/table/scanner/log/LogScan.java | 33 +- .../table/scanner/log/LogScannerImpl.java | 8 +- .../fluss/client/write/RecordAccumulator.java | 99 +- .../client/write/VariantShreddingManager.java | 170 +++ .../client/table/FlussVariantITCase.java | 577 +++++++++ .../scanner/log/LogFetcherFilterITCase.java | 2 + .../table/scanner/log/LogFetcherITCase.java | 4 + .../table/scanner/log/LogFetcherTest.java | 1 + .../apache/fluss/config/ConfigOptions.java | 51 + .../org/apache/fluss/config/TableConfig.java | 45 + .../org/apache/fluss/metadata/TableInfo.java | 38 +- .../fluss/record/DefaultLogRecordBatch.java | 1 + .../DefaultLogRecordBatchStatistics.java | 7 + .../fluss/record/FileLogProjection.java | 211 +++- .../fluss/record/LogRecordReadContext.java | 10 +- .../org/apache/fluss/row/BinaryArray.java | 7 + .../org/apache/fluss/row/BinaryWriter.java | 3 + .../org/apache/fluss/row/DataGetters.java | 4 + .../org/apache/fluss/row/GenericArray.java | 6 + .../java/org/apache/fluss/row/GenericRow.java | 6 + .../org/apache/fluss/row/InternalArray.java | 3 + .../org/apache/fluss/row/InternalRow.java | 6 + .../java/org/apache/fluss/row/PaddingRow.java | 7 + .../org/apache/fluss/row/ProjectedRow.java | 6 + .../apache/fluss/row/aligned/AlignedRow.java | 13 + .../apache/fluss/row/arrow/ArrowReader.java | 22 + .../apache/fluss/row/arrow/ArrowWriter.java | 119 +- .../fluss/row/arrow/ArrowWriterPool.java | 25 +- .../fluss/row/arrow/ArrowWriterProvider.java | 21 + .../vectors/ArrowVariantColumnVector.java | 58 + .../vectors/ShreddedVariantColumnVector.java | 254 ++++ .../writers/ArrowShreddedVariantWriter.java | 507 ++++++++ .../row/arrow/writers/ArrowVariantWriter.java | 50 + .../fluss/row/columnar/ColumnarArray.java | 6 + .../fluss/row/columnar/ColumnarRow.java | 6 + .../row/columnar/VariantColumnVector.java | 25 + .../row/columnar/VectorizedColumnBatch.java | 5 + .../fluss/row/compacted/CompactedRow.java | 6 + .../row/compacted/CompactedRowReader.java | 7 + .../apache/fluss/row/indexed/IndexedRow.java | 8 + .../fluss/types/DataTypeDefaultVisitor.java | 5 + .../apache/fluss/types/DataTypeParser.java | 4 +- .../org/apache/fluss/types/DataTypeRoot.java | 4 +- .../apache/fluss/types/DataTypeVisitor.java | 2 + .../org/apache/fluss/types/DataTypes.java | 10 + .../org/apache/fluss/types/VariantType.java | 65 + .../fluss/types/variant/FieldStatistics.java | 178 +++ .../fluss/types/variant/ShreddedField.java | 87 ++ .../fluss/types/variant/ShreddedVariant.java | 457 +++++++ .../fluss/types/variant/ShreddingSchema.java | 364 ++++++ .../variant/ShreddingSchemaInferrer.java | 336 +++++ .../apache/fluss/types/variant/Variant.java | 396 ++++++ .../fluss/types/variant/VariantBuilder.java | 457 +++++++ .../variant/VariantStatisticsCollector.java | 234 ++++ .../fluss/types/variant/VariantUtil.java | 1088 +++++++++++++++++ .../org/apache/fluss/utils/ArrowUtils.java | 283 ++++- .../fluss/utils/json/DataTypeJsonSerde.java | 3 + .../fluss/record/FileLogProjectionTest.java | 360 +++++- .../apache/fluss/record/LogRecordsTest.java | 22 + .../org/apache/fluss/record/TestData.java | 34 + .../arrow/ShreddedVariantRoundTripTest.java | 459 +++++++ .../types/variant/JsonToVariantTest.java | 460 +++++++ .../variant/ShreddingSchemaInferrerTest.java | 317 +++++ .../types/variant/ShreddingSchemaTest.java | 109 ++ .../VariantStatisticsCollectorTest.java | 250 ++++ .../fluss/types/variant/VariantTest.java | 364 ++++++ .../fluss/types/variant/VariantUtilTest.java | 398 ++++++ .../flink/sink/Flink22VariantTypeITCase.java | 248 ++++ .../LakeSnapshotAndLogSplitScanner.java | 3 +- .../fluss/flink/row/FlinkAsFlussArray.java | 48 + .../fluss/flink/row/FlinkAsFlussRow.java | 57 + .../fluss/flink/sink/FlussSinkBuilder.java | 1 + .../flink/source/FlussSourceBuilder.java | 2 + .../source/reader/FlinkSourceSplitReader.java | 1 + .../utils/FlinkRowToFlussRowConverter.java | 29 + .../flink/utils/FlinkTypeToFlussType.java | 6 + .../utils/FlussRowToFlinkRowConverter.java | 27 + .../flink/utils/FlussTypeToFlinkType.java | 26 + .../FlussDataTypeToIcebergDataType.java | 6 + .../source/IcebergArrayAsFlussArray.java | 6 + .../source/IcebergRecordAsFlussRow.java | 6 + .../utils/FlussDataTypeToPaimonDataType.java | 6 + .../paimon/utils/PaimonArrayAsFlussArray.java | 6 + .../paimon/utils/PaimonRowAsFlussRow.java | 6 + fluss-rpc/src/main/proto/FlussApi.proto | 9 + .../fluss/server/entity/FetchReqInfo.java | 24 +- .../apache/fluss/server/log/FetchParams.java | 27 +- .../fluss/server/replica/ReplicaManager.java | 1 + .../server/utils/ServerRpcMessageUtils.java | 17 +- .../server/replica/ReplicaManagerTest.java | 218 ++++ .../fluss/server/replica/ReplicaTestBase.java | 26 + .../fluss/spark/row/SparkAsFlussArray.scala | 6 + .../fluss/spark/row/SparkAsFlussRow.scala | 6 + .../spark/types/FlussToSparkTypeVisitor.scala | 4 + 98 files changed, 10907 insertions(+), 69 deletions(-) create mode 100644 docs/variant-shredding-design.md create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/write/VariantShreddingManager.java create mode 100644 fluss-client/src/test/java/org/apache/fluss/client/table/FlussVariantITCase.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ArrowVariantColumnVector.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ShreddedVariantColumnVector.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowShreddedVariantWriter.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowVariantWriter.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/row/columnar/VariantColumnVector.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/VariantType.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/FieldStatistics.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddedField.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddedVariant.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddingSchema.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddingSchemaInferrer.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/Variant.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/VariantBuilder.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/VariantStatisticsCollector.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/types/variant/VariantUtil.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/row/arrow/ShreddedVariantRoundTripTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/types/variant/JsonToVariantTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/types/variant/ShreddingSchemaInferrerTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/types/variant/ShreddingSchemaTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/types/variant/VariantStatisticsCollectorTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/types/variant/VariantTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/types/variant/VariantUtilTest.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22VariantTypeITCase.java diff --git a/docs/variant-shredding-design.md b/docs/variant-shredding-design.md new file mode 100644 index 0000000000..2416bed250 --- /dev/null +++ b/docs/variant-shredding-design.md @@ -0,0 +1,878 @@ +# Fluss Variant Shredding 设计方案 + +## 1. Parquet Variant Shredding 机制 + +### 1.1 Variant 是什么 + +Variant 是一种半结构化数据类型,将 JSON 等数据以二进制编码存储。每个 Variant 值由两部分组成: + +- **metadata**:字段名字典 +- **value**:自描述的二进制编码值(对象、数组、基础类型等) + +#### metadata 二进制格式 + +metadata 是**每行独立的**二进制字段,存储该行 Variant 对象中所有字段名的去重字典。二进制布局: + +``` ++----------+------------------+--------------------------------------+--------------------+ +| Header | Dictionary Size | Offsets Array | String Bytes | +| (1 byte) | (offset_size B) | ((dict_size + 1) * offset_size B) | (变长) | ++----------+------------------+--------------------------------------+--------------------+ +``` + +**Header 字节**: + +``` +位 7 6 5 4 3 2 1 0 + └─offset_size─┘ │ └─── version ───┘ + minus_1 sorted +``` + +| 位 | 含义 | 取值 | +|-----|------|------| +| 0-3 | 版本号 | 必须为 1 | +| 4 | sorted_strings 标志 | 1 = 字典按字母序排列(支持二分查找) | +| 5 | 保留 | 0 | +| 6-7 | offset_size_minus_1 | 0→1字节, 1→2, 2→3, 3→4字节 | + +Fluss 始终使用 4 字节偏移 + 已排序,因此 header 固定为 `0xD1`(version=1 | sorted=0x10 | offset_size_bits=0xC0)。 + +**字典内容示例**: + +对于 JSON `{"name":"alice", "age":30, "city":"beijing"}`,metadata 存储的是 3 个字段名按字母序排列的字典: + +``` +Header: 0xD1 +Dictionary Size: 3 (小端序 4字节: 03 00 00 00) +Offsets: [0, 3, 7, 11] ← 4 个偏移(dict_size+1)指向 string bytes 内的位置 +String Bytes: "age" + "city" + "name" = [61 67 65 | 63 69 74 79 | 6E 61 6D 65] + offset 0 offset 3 offset 7 offset 11 +字典索引: id=0(age) id=1(city) id=2(name) +``` + +value 中的对象字段通过 **字典索引 id** 引用字段名,而非存储字段名字符串本身。如 `"name"` 字段在 value 中用 id=2 表示。查找字段时,因为字典已排序,可以二分查找字段名对应的 id。 + +> **重要:metadata 是每行独立的**。同一列中,第 0 行的 metadata 可能含 `{age, name}`,第 1 行含 `{city, email, name}`,完全独立。这是 Variant 作为半结构化类型的核心 —— 每行的字段集合可以不同。 + +### 1.2 为什么需要 Shredding + +将 Variant 整块存为一个 Binary 列时,即使只查一个字段也必须完整读取和反序列化整个 Variant。Shredding(切碎)将高频访问的字段提取到独立的 **强类型列(typed_value)** 中,从而: + +- 利用列式压缩、编码(RLE、字典编码等) +- 支持列统计信息用于 Data Skipping +- 支持列裁剪(Column Pruning),只读需要的字段 + +### 1.3 typed_value 的存储结构 + +Parquet 中每个 Variant 字段用一个 **group** 表示,包含三个子列协作工作: + +``` +required group event (VARIANT) { + required binary metadata; -- 字段名字典(所有字段共用) + optional binary value; -- 兜底:Variant 二进制编码 + optional typed_value; -- 强类型列 / shredded 对象组 +} +``` + +#### value 与 typed_value 的语义矩阵 + +| value | typed_value | 含义 | +|----------|-------------|----------------------------------------------| +| null | null | 字段不存在(仅 object 的 shredded field 合法) | +| non-null | null | 值存在,类型不匹配预期,退回二进制编码 | +| null | non-null | 值存在,类型匹配,存在强类型列中 | +| non-null | non-null | 部分 shredded 对象:typed_value 存已拆字段,value 存残余 | + +这是 Parquet Variant Shredding 的核心设计理念:**每一层都有“正轨”(typed_value)和“兜底”(value)两条路**。类型匹配走正轨享受列式优化,不匹配走兜底保证不丢数据。 + +#### metadata、value、typed_value 的协作关系 + +- **metadata**:专门用来解码 `value` 二进制的字段名字典。只包含 residual 引用的字段名。 +- **value**:残余/兜底的 Variant 二进制编码,通过 metadata 字典 id 引用字段名。 +- **typed_value**:强类型列,字段名由 Struct 列名决定,**不依赖 metadata**。 + +三者的分工决定了读取时的数据流向: +- 查询已 shred 字段 → 只读 typed_value,不需要 metadata 和 value +- 查询未 shred 字段 → 只读 metadata + value,不需要 typed_value +- 完整还原 → 三者合并 + +> 具体的列存存储布局和多行读写示例参见 [3.1 核心思路](#31-核心思路)。 + +#### 对象的 Shredding + +假设要 shred `event_type`(string) 和 `event_ts`(timestamp): + +``` +optional group event (VARIANT) { + required binary metadata; + optional binary value; -- 存残余字段(如 email) + optional group typed_value { -- shredded 对象 + required group event_type { -- 每个字段也是 value + typed_value 对 + optional binary value; -- 类型不匹配时的兜底 + optional binary typed_value (STRING); -- 匹配时直接存 string + } + required group event_ts { + optional binary value; + optional int64 typed_value (TIMESTAMP); + } + } +} +``` + +实际存储示例: + +| 原始 JSON | value (残余) | event_type.typed_value | event_ts.typed_value | +|---|---|---|---| +| `{"event_type":"login", "event_ts":1729..., "email":"a@b.com"}` | `{"email":"a@b.com"}` | "login" | 1729... | +| `{"event_type":"login", "event_ts":1729...}` | null(完全 shred) | "login" | 1729... | +| `{"error_msg":"bad"}` | `{"error_msg":"bad"}` | null(无此字段) | null | +| `"not an object"` | `0x13 0x6E...`(string 二进制) | null | null | + +#### 数组的 Shredding + +数组用 Parquet 3-level LIST 表示,每个元素也有 value + typed_value 对: + +``` +optional group tags (VARIANT) { + required binary metadata; + optional binary value; + optional group typed_value (LIST) { + repeated group list { + required group element { + optional binary value; -- 元素类型不匹配时的兜底 + optional binary typed_value (STRING); -- 元素是 string 时走这里 + } + } + } +} +``` + +#### 递归嵌套 + +typed_value 结构支持任意层级的递归嵌套——typed_value 里的每个字段可以进一步包含自己的 value + typed_value 对,实现深层对象/数组的 shredding。 + +### 1.4 typed_value 的类型映射 + +| Variant Type | Parquet Physical Type | Parquet Logical Type | +|---|---|---| +| boolean | BOOLEAN | | +| int8 | INT32 | INT(8, signed=true) | +| int16 | INT32 | INT(16, signed=true) | +| int32 | INT32 | | +| int64 | INT64 | | +| float | FLOAT | | +| double | DOUBLE | | +| decimal4 | INT32 | DECIMAL(P, S) | +| decimal8 | INT64 | DECIMAL(P, S) | +| decimal16 | BYTE_ARRAY / FIXED_LEN_BYTE_ARRAY | DECIMAL(P, S) | +| date | INT32 | DATE | +| timestamp | INT64 | TIMESTAMP(true/false, MICROS/NANOS) | +| binary | BINARY | | +| string | BINARY | STRING | +| array | GROUP (LIST) | | +| object | GROUP (Struct) | | + +--- + +## 2. Arrow 对 Variant 的支持现状 + +### 2.1 Canonical Extension Type: `arrow.parquet.variant` + +Arrow 社区在 **v22.0.0(2025年10月)** 将 Variant 纳入 Canonical Extension Type,扩展名为 `arrow.parquet.variant`。 + +在 Arrow 内存中,Variant 用 Struct 表示: + +``` +// 非 shredded(基础形态) +Struct { + metadata: Binary/BinaryView, -- 字段名字典 + value: Binary/BinaryView -- Variant 二进制编码 +} + +// Shredded(带类型提取) +Struct { + metadata: Binary/BinaryView, + value: Binary/BinaryView, -- 残余/兜底 + typed_value: Struct { -- 与 Parquet shredding 规则对齐 + field_a: Struct { + value: Binary, + typed_value: Int64 + }, + field_b: Struct { + value: Binary, + typed_value: Utf8 + } + } +} +``` + +### 2.2 各语言实现进展 + +| 语言 | Variant 基础 | Shredding (typed_value) | 说明 | +|------|-------------|------------------------|------| +| **arrow-rs (Rust)** | ✅ 完整 | ✅ 最成熟 | `shred_variant` 函数、`VariantArrayBuilder`、typed_access | +| **parquet-java** | ✅ 完整 | ✅ 完整 | Parquet 层面 shredded read/write | +| **arrow-go** | ✅ 完整 | ✅ 完整 | VariantType Extension + shredded array builder | +| **arrow-java 19.0.0** | ✅ 基础 | ❌ 未实现 | 仅 metadata + value,无 typed_value | +| **arrow C++** | ⚠️ 仅类型定义 | ❌ 未实现 | decoding/encoding/shredding 均 Open | +| **DuckDB** | ✅ 完整 | ✅ 完整 | | + +> **来源**: [Parquet Implementation Status](https://parquet.apache.org/docs/file-format/implementationstatus/) + +### 2.3 Arrow Java 19.0.0 的 Variant 支持 + +Issue: [apache/arrow-java#946](https://github.com/apache/arrow-java/issues/946) +PR: [apache/arrow-java#947](https://github.com/apache/arrow-java/pull/947) — 已合并,发布于 2026年3月16日。 + +提供的能力: + +- **`arrow-vector` 模块**:`VariantType` Extension Type 定义 +- **`arrow-variant` 模块**:`VariantVector`(metadata/value pair)、`Variant` 解析类、Reader/Writer 支持 +- **不包含**:shredded variant(typed_value Struct)的构建和读取 + +Maven 坐标: + +```xml + + org.apache.arrow + arrow-vector + 19.0.0 + + + org.apache.arrow + arrow-variant + 19.0.0 + +``` + +--- + +## 3. Fluss Variant Shredding 实现方案 + +### 3.1 核心思路 + +基于 `arrow.parquet.variant` Extension Type 规范,将 shredded 列**内嵌到 Variant 列的 Struct 内部**。Shredding 对用户完全透明,表的对外 schema 始终不变: + +``` +(id INT, data VARIANT) -- 无论是否 shredding,用户看到的 schema 始终如此 +``` + +Variant 列内部结构: + +``` +StructVector("data") { -- arrow.parquet.variant Extension Type + metadata: VarBinaryVector -- 字段名字典 + value: VarBinaryVector -- 残余/兜底 Variant 二进制 + typed_value: StructVector { -- shredded 字段(新增) + "name": StructVector { + value: VarBinaryVector -- 类型不匹配时的兜底 + typed_value: VarCharVector -- String + } + "age": StructVector { + value: VarBinaryVector + typed_value: BigIntVector -- Int64 + } + } +} +``` + +#### value 与 typed_value 的行级存储示例 + +每个 shredded 字段的 `value` 和 `typed_value` 是**等长的两个 Vector**,按行索引一一对应。每一行只有其中一个非 null(或两个都 null 表示字段不存在)。 + +以 `"name"` 字段(期望类型 String)为例,假设一个 batch 有 5 行: + +``` +行号 原始 JSON name.value (VarBinary) name.typed_value (VarChar) +────────────────────────────────────────────────────────────────────────────────────────── + 0 {"name":"alice", "age":30} null "alice" ← 类型匹配,走 typed_value + 1 {"name":"bob"} null "bob" ← 类型匹配 + 2 {"name":12345} 0x05 0x30... (int 二进制) null ← 类型不匹配,走 value 兜底 + 3 {"age":25} null null ← name 不存在,两个都 null + 4 {"name":"eve", "age":20} null "eve" ← 类型匹配 +``` + +读取逻辑很简单 —— 对每一行,优先检查 `typed_value`,不为 null 则直接读;否则检查 `value` 反序列化;两者都为 null 则字段不存在: + +```java +Variant readShreddedField(int row, StructVector fieldStruct) { + VarCharVector typedValue = (VarCharVector) fieldStruct.getChild("typed_value"); + VarBinaryVector value = (VarBinaryVector) fieldStruct.getChild("value"); + + if (!typedValue.isNull(row)) { + // 快速路径:类型匹配,直接读强类型值 + return Variant.fromString(typedValue.get(row)); + } else if (!value.isNull(row)) { + // 兜底路径:类型不匹配,反序列化 Variant 二进制 + return Variant.fromBinary(value.get(row)); + } else { + // 该行没有这个字段 + return null; + } +} +``` + +> null 的行不占实际数据空间,只占 validity bitmap 中的 1 bit 标记。 + +#### 列存视角的完整读写示例 + +以两行数据为例,shredding schema 决定 shred `name`(STRING) 和 `age`(BIGINT): + +``` +Row 0: {"name":"alice", "age":30, "email":"a@b.com"} +Row 1: {"name":"bob", "age":"unknown", "city":"beijing"} + ↑ 类型不匹配 BIGINT +``` + +**写入后的 Arrow StructVector 布局(每列是一个 Vector,每行是一个 slot):** + +``` +StructVector("data") —— arrow.parquet.variant Extension Type +│ +├─ metadata: VarBinaryVector (2 rows) +│ Row 0: [0xD1, 01 00 00 00, ...] ← 字典 = ["email"],只含 residual 的字段名 +│ Row 1: [0xD1, 01 00 00 00, ...] ← 字典 = ["city"],每行 metadata 独立 +│ +├─ value: VarBinaryVector (2 rows) +│ Row 0: <{"email":"a@b.com"} 的 Variant 二进制> ← 通过 metadata 字典 id=0 引用 "email" +│ Row 1: <{"city":"beijing"} 的 Variant 二进制> ← 通过 metadata 字典 id=0 引用 "city" +│ +└─ typed_value: StructVector + │ + ├─ "name": StructVector + │ ├─ value: VarBinaryVector + │ │ Row 0: null + │ │ Row 1: null + │ └─ typed_value: VarCharVector + │ Row 0: "alice" ← STRING 匹配,走正轨 + │ Row 1: "bob" ← STRING 匹配,走正轨 + │ + └─ "age": StructVector + ├─ value: VarBinaryVector + │ Row 0: null + │ Row 1: <"unknown" 的 Variant 二进制> ← STRING 不匹配 BIGINT,走兜底 + └─ typed_value: BigIntVector + Row 0: 30 ← BIGINT 匹配,走正轨 + Row 1: null ← 类型不匹配,null +``` + +关键观察: +- **metadata VarBinaryVector**:每行是一个独立的二进制字典。Row 0 和 Row 1 的 metadata 完全不同,因为它们的 residual 字段不同 +- **value VarBinaryVector**:每行是 residual 对象的 Variant 二进制,内部通过该行 metadata 的字典 id 引用字段名 +- **typed_value 内的每个字段**:字段名由 StructVector 的 child name 决定,与 metadata 无关 +- **同一列内 Vector 长度严格相等**:VarBinaryVector、VarCharVector、BigIntVector 都是 2 行,通过 validity bitmap 标记 null + +**读取场景 1:只读 `name`(已 shred 字段)** + +``` +只需读 typed_value."name".typed_value 这一个 VarCharVector: + Row 0: VarCharVector.get(0) = "alice" + Row 1: VarCharVector.get(1) = "bob" +不需要触碰 metadata 和 value Vector,零反序列化。 +``` + +**读取场景 2:只读 `age`(已 shred,存在类型不匹配)** + +``` +只需读 typed_value."age" 的两个子 Vector,逐行判断: + Row 0: BigIntVector.isNull(0)=false → 走正轨,BigIntVector.get(0)=30 + Row 1: BigIntVector.isNull(1)=true → 检查 VarBinaryVector.get(1),解码得到 "unknown" +仍然不需要触碰顶层 metadata 和 value。 +``` + +**读取场景 3:只读 `email`(未 shred 字段)** + +``` +只需读顶层 metadata + value 两个 VarBinaryVector,不需要 typed_value: + Row 0: metadata=["email"], value= + → 用 metadata 字典解码 value,查找 "email" → id=0 → 提取得到 "a@b.com" + Row 1: metadata=["city"], value= + → 用 metadata 字典解码 value,查找 "email" → 字典中没有 → 返回 null +``` + +**读取场景 4:读全部字段(完整还原)** + +``` +三者都需要读,逐行合并: + +Row 0: + 1. 解码 residual: metadata VarBinaryVector.get(0) + value VarBinaryVector.get(0) + → 字典=["email"] + value 二进制 → {"email":"a@b.com"} + 2. 读 typed_value: name="alice" (VarCharVector), age=30 (BigIntVector) + 3. 合并: {"name":"alice", "age":30, "email":"a@b.com"} + +Row 1: + 1. 解码 residual: metadata VarBinaryVector.get(1) + value VarBinaryVector.get(1) + → 字典=["city"] + value 二进制 → {"city":"beijing"} + 2. 读 typed_value: name="bob" (VarCharVector), age 兜底解码 → "unknown" + 3. 合并: {"name":"bob", "age":"unknown", "city":"beijing"} +``` + +### 3.2 前提条件 + +| 前提 | 状态 | 说明 | +|------|------|------| +| Arrow StructVector 嵌套 | ✅ 已具备 | Arrow Java 15.0.0 就支持 | +| Variant 二进制编解码 | ✅ 已具备 | VariantUtil / VariantBuilder | +| Shredding 统计推断 | ✅ 已具备 | VariantStatisticsCollector / ShreddingSchemaInferrer | +| ShreddedVariant 合并读取 | ✅ 已具备 | ShreddedVariantColumnVector(需适配) | +| Arrow Java Variant Extension Type | ⚠️ 需升级 | Arrow Java 15.0.0 → 19.0.0 | +| typed_value Writer 拆分逻辑 | ❌ 需实现 | 将字段写入 Struct 内部子 Vector | +| typed_value Reader 逻辑 | ❌ 需适配 | 从 Struct 内部读取替代从顶层列读取 | +| Struct 内部 schema 管理 | ❌ 需实现 | typed_value 的子字段增减机制 | + +### 3.3 第一期能力边界(Scope Limitation) + +第一期实现聚焦于 **最常见、最有收益的场景**,有意限制 typed_value 的复杂度,避免引入过多边界情况。 + +#### 支持范围 + +| 维度 | 第一期支持 | 说明 | +|------|-----------|------| +| Shredding 深度 | **仅顶层字段** | 只分析 Variant 对象的第一层 key,不递归进入嵌套对象 | +| typed_value 类型 | **标量基础类型** | BOOLEAN, BIGINT, FLOAT, DOUBLE, STRING, DATE, TIMESTAMP | +| 整数类型处理 | **统一拓宽为 BIGINT** | INT8/INT16/INT32/INT64 全部映射为 BIGINT,避免类型不一致 | +| typed_value 结构 | **Struct { value: Binary, typed_value: }** | 每个 shredded 字段是固定的 value + typed_value 二元结构 | +| Variant 顶层类型 | **仅对象(Object)** | 顶层 Variant 是对象时才触发 shredding,非对象(如字符串、数组)直接走 value 兜底 | + +#### 明确不支持(后续版本) + +| 场景 | 原因 | 预计支持版本 | +|------|------|-------------| +| **嵌套对象 shredding** | typed_value 内部再嵌套 Struct,StructVector 层级加深,读写/合并逻辑复杂度指数增长 | v2+ | +| **数组 shredding** | 需要 ListVector 嵌套 Struct(Parquet 3-level LIST),Arrow 的 ListVector 投影、split/merge 逻辑复杂 | v2+ | +| **Map 类型 shredding** | Variant 规范中无原生 Map,且 Parquet Map shredding 缺乏社区共识 | 待定 | +| **DECIMAL / BINARY 类型** | 出现频率低,且 DECIMAL 的精度/标度推断规则复杂 | v2+ | +| **多层递归 typed_value** | 如 `typed_value.address.typed_value.city.typed_value`,Arrow StructVector 嵌套过深影响性能和可维护性 | v2+ | +| **Schema 降级(删除 shredded 字段)** | 第一期字段只增不减,已 shredded 的字段保持到数据过期。降级需处理存量 batch 兼容性 | v2+ | + +#### 第一期 StructVector 实际布局 + +基于以上限制,第一期的 StructVector 结构是扁平的、可预测的: + +``` +StructVector("data") { -- Variant Extension Type + metadata: VarBinaryVector -- 字段名字典 + value: VarBinaryVector -- 残余/兜底(去除已 shred 字段的对象) + typed_value: StructVector { -- 最多 N 个子字段(默认上限 20) + "name": StructVector { + value: VarBinaryVector -- 类型不匹配时的兜底 + typed_value: VarCharVector -- String(标量) + } + "age": StructVector { + value: VarBinaryVector + typed_value: BigIntVector -- Int64(标量) + } + "is_active": StructVector { + value: VarBinaryVector + typed_value: BitVector -- Boolean(标量) + } + -- 注意:不会出现 typed_value 内部再嵌套 StructVector 的子 typed_value + } +} +``` + +关键约束: +- **typed_value 内部每个子字段的 typed_value 一定是标量 Vector**(VarCharVector, BigIntVector, BitVector 等),不会是 StructVector 或 ListVector +- **typed_value 的最大子字段数**由 `maxShreddedFields` 控制(默认 20),防止字段爆炸 +- **统计采样阈值**:字段出现率 ≥ 0.5 且类型一致性 ≥ 0.9 才会被选为 shredded 字段 + +#### 第一期边界带来的简化 + +| 方面 | 简化效果 | +|------|----------| +| Writer | 只需处理「对象 → 提取标量字段 → 写入对应 Vector」,无需递归拆解 | +| Reader | 合并逻辑只有一层:residual + 标量 typed_value,无需递归合并 | +| 列裁剪 | 投影路径最多两层(`typed_value.name.typed_value`),无需多层嵌套投影 | +| Schema 管理 | typed_value 的 Struct children 只增不减,无需处理降级/迁移 | +| 测试覆盖 | 有限的类型组合(~7 种标量 × 兜底),可完整覆盖边界场景 | + +### 3.4 动态 Shredding:StructVector 如何实现运行时决策 + +#### 核心约束 + +Arrow Java 的 `VectorSchemaRoot` **schema 在创建时确定且不可变**。`StructVector` 的子 Vector 集合在分配时固定,不支持运行时动态添加子列。因此,typed_value 的子字段不能在写入过程中动态增减。 + +#### 解决方案:Per-Batch Schema + Writer 切换 + +关键思路:**不同的 Arrow Batch 可以有不同的 Struct 布局**。Shredding 决策不改变表级 schema,而是改变 ArrowWriter 内部的 VectorSchemaRoot 布局。 + +##### 阶段流转 + +``` +阶段 0 (无 shredding): + Variant 列 = Struct { metadata: Binary, value: Binary } + ↓ 客户端统计收集 (VariantStatisticsCollector) + ↓ 推断触发 (ShreddingSchemaInferrer, 默认采样 1000 行) + ↓ 获得 shredding schema(具体来源见 3.6 节) +阶段 1 (shredding 激活): + Variant 列 = Struct { metadata: Binary, value: Binary, + typed_value: Struct { name: Struct{value, typed_value}, ... } } +``` + +##### Writer 端实现 + +`ArrowWriterPool` 按 `tableId-schemaId-compressionInfo-shreddingSchemaHash` 缓存 writer。当 shredding schema 变化时(首次推断、增加字段),会创建一个**新的 ArrowWriter**,其 VectorSchemaRoot 使用新的 Struct 布局: + +```java +// 根据 shredding schema 构造 Variant 列的 Arrow Field +List variantChildren = new ArrayList<>(); +variantChildren.add(Field.nullable("metadata", ArrowType.Binary.INSTANCE)); +variantChildren.add(Field.nullable("value", ArrowType.Binary.INSTANCE)); + +if (shreddingSchema != null && !shreddingSchema.getFields().isEmpty()) { + List typedValueChildren = new ArrayList<>(); + for (ShreddedField sf : shreddingSchema.getFields()) { + typedValueChildren.add(new Field(sf.getFieldPath(), + FieldType.notNullable(ArrowType.Struct.INSTANCE), + List.of( + Field.nullable("value", ArrowType.Binary.INSTANCE), + Field.nullable("typed_value", toArrowType(sf.getType())) + ))); + } + variantChildren.add(new Field("typed_value", + FieldType.notNullable(ArrowType.Struct.INSTANCE), typedValueChildren)); +} + +Field variantField = new Field("data", + FieldType.nullable(ArrowType.Struct.INSTANCE), variantChildren); +``` + +##### Reader 端实现 + +Reader 从 Arrow IPC batch 的 **schema 自动发现** typed_value 结构,无需额外元数据: + +```java +StructVector variantVector = (StructVector) root.getVector("data"); +VarBinaryVector metadata = (VarBinaryVector) variantVector.getChild("metadata"); +VarBinaryVector value = (VarBinaryVector) variantVector.getChild("value"); + +// 检查是否有 typed_value(不同 batch 可能有/没有) +StructVector typedValue = (StructVector) variantVector.getChild("typed_value"); +if (typedValue != null) { + // 发现 shredded 字段,从 typed_value 子列读取 + for (FieldVector child : typedValue.getChildrenFromFields()) { + StructVector field = (StructVector) child; + String fieldName = field.getName(); + // field.getChild("typed_value") → 强类型值 + // field.getChild("value") → 兜底 Variant 二进制 + } +} else { + // 无 shredding,从 value 列反序列化完整 Variant +} +``` + +##### 新旧 Batch 共存 + +同一张表的 log 中,不同 batch 可以有不同的 typed_value 布局: + +| Batch | 写入时间 | typed_value 布局 | +|-------|---------|------------------| +| Batch 1-100 | 阶段 0 | 无 typed_value | +| Batch 101-200 | 阶段 1 | typed_value: {name, age} | +| Batch 201+ | 阶段 2 (新增字段) | typed_value: {name, age, city} | + +Reader 逐 batch 检查 Arrow schema,自适应处理。旧 batch 没有 typed_value → 完全从 value 反序列化。新 batch 有 typed_value → 从强类型列读取。 + +##### initFieldVector 的适配 + +`ArrowWriter.initFieldVector()` 需要支持 StructVector 的子列递归初始化: + +```java +private void initFieldVector(FieldVector fieldVector) { + fieldVector.setInitialCapacity(INITIAL_CAPACITY); + if (fieldVector instanceof BaseFixedWidthVector) { + ((BaseFixedWidthVector) fieldVector).allocateNew(INITIAL_CAPACITY); + } else if (fieldVector instanceof BaseVariableWidthVector) { + ((BaseVariableWidthVector) fieldVector).allocateNew(INITIAL_CAPACITY); + } else if (fieldVector instanceof ListVector) { + // ... 现有逻辑 + } else if (fieldVector instanceof StructVector) { + // 新增:递归初始化 StructVector 的所有子列 + StructVector structVector = (StructVector) fieldVector; + structVector.allocateNew(); + for (FieldVector child : structVector.getChildrenFromFields()) { + initFieldVector(child); + } + } else { + fieldVector.allocateNew(); + } +} +``` + +### 3.5 Arrow 升级说明 + +Fluss 需从 Arrow Java 15.0.0 升级到 19.0.0,跨 4 个大版本。19.0.0 包含多个 Breaking Changes: + +- `BitVectorHelper` API 变更 +- `ExtensionTypeWriterFactory` 重构(PR #891 → #892) +- `MetadataAdapter.getAll()` 不再返回 null + +升级后可直接使用 `arrow-variant` 模块的 `VariantType` 和 `VariantVector`,但 **shredding(typed_value)需要 Fluss 自行实现**,因为 Arrow Java 19.0.0 尚未支持。 + +### 3.6 Shredding Schema 的协调策略 + +Writer 需要知道「该 shred 哪些字段、每个字段什么类型」来构造 StructVector 布局。这个信息即 **shredding schema**。它不是表级 schema(对外 schema 始终不变),而是内部元数据,指导 Writer 如何拆分 Variant 字段。 + +有两种候选方案: + +#### 方案 A:Writer 独立决策(纯客户端) + +每个 Writer 独立采样、独立推断、独立决定自己 batch 的 typed_value 布局。不涉及任何 Server 端存储或分发。 + +``` +Writer A: Writer B: + 采样 1000 行 采样 1000 行 + → 推断: {name: STRING, age: BIGINT} → 推断: {name: STRING, city: STRING} + → 写入 batch: typed_value={name,age} → 写入 batch: typed_value={name,city} +``` + +优势: +- **实现最简单**:无 RPC、无持久化、无多 Writer 协调 +- **无单点**:Writer 完全自治,server 故障不影响 shredding 决策 +- **启动快**:新 Writer 无需等待 server 返回 shredding schema,采样完即可开始 + +劣势: +- **batch 布局不统一**:不同 Writer 可能 shred 不同字段,同一字段的 data skipping 统计被分散 +- **重复采样**:每个 Writer 重启后都需要重新采样推断,前 1000 行无 shredding +- **Tiering 复杂度**:合并不同布局的 batch 时需要重新组织 + +#### 方案 B:Server 协调(集中式) + +Writer 推断后上报 server,server 合并后统一分发给所有 Writer。Shredding schema 持久化在 server 端。 + +``` +Writer A: Server: Writer B: + 采样 → 推断 {name, age} merge({name,age}, {name,city}) 采样 → 推断 {name, city} + → 上报 server ───────→ → 统一 schema: {name,age,city} ←─────── 上报 server ← + ← 获取 {name,age,city} 持久化到 table metadata 获取 {name,age,city} → + → 所有 batch 统一布局 所有 batch 统一布局 ← +``` + +优势: +- **全局统一**:所有 Writer 产出的 batch 布局一致,利于列统计和 Tiering compaction +- **新 Writer 快速启动**:从 server 拉取已有 shredding schema,无需重新采样 +- **schema 稳定**:多个 Writer 的统计信息汇聚后推断更准确 + +劣势: +- **实现复杂度高**:需要 RPC 接口、server 端合并逻辑、持久化存储、变更通知 +- **依赖 server 可用性**:server 不可用时,新 Writer 无法获取 shredding schema(需回退无 shredding) +- **协调延迟**:推断完成到全局生效有一个 RPC 往返的窗口期 + +#### 对比总结 + +| 方面 | 方案 A:Writer 独立决策 | 方案 B:Server 协调 | +|------|----------------------|------------------| +| 实现复杂度 | 低(纯客户端) | 高(RPC + 持久化 + 通知) | +| batch 布局一致性 | 不一致(各 Writer 独立) | 全局一致 | +| Writer 重启 | 需重新采样(前 N 行无 shredding) | 直接拉取已有 schema | +| Tiering compaction | 需处理异构 batch | batch 布局统一,合并简单 | +| Server 依赖 | 无 | shredding 依赖 server 可用 | +| Reader | 两种方案相同:从 batch schema 自动发现 typed_value | 同左 | + +> **注意**:Reader 端两种方案完全相同 —— 都是从每个 batch 的 Arrow schema 自动发现 typed_value 结构,不依赖外部元数据。因此方案选择仅影响 Writer 端。 + +### 3.7 待确认的设计细节 + +以下问题在实现时需要明确: + +#### metadata 处理策略 + +当字段被 shred 出去后,residual 的 metadata(字段名字典)如何处理: +- **复用原始 metadata**:不改 metadata,多余字段名白占空间但实现简单 +- **重建 metadata**:只保留 residual 涉及的字段名,省空间但多一次编码 + +#### 完全 shredded 的边界 + +当一行的所有字段都被 shred(如 `{"name":"alice","age":30}` 且 name/age 都是 shredded 字段): +- 顶层 `value` 应为 **null**(而非空对象 `{}`),符合 Parquet 规范语义:value=null 且 typed_value=non-null 表示「值完全由 typed_value 表达」 + +#### Variant 列本身为 NULL + +整行的 Variant 列是 SQL NULL(不是 `{}`,而是无值)时,StructVector 的 validity bit 为 0,内部 metadata/value/typed_value 三个子 Vector 的对应行均无意义。Writer 端需正确设置 StructVector 的 null bit。 + +#### Residual 编码成本 + +Writer 对每行需要:解析原始 Variant → 提取 shredded 字段 → 重新编码剩余字段为 Variant 二进制。这个「拆一个小字段 + 重建整个对象」的逻行编码有 CPU 开销,需在实现时评估是否可通过零拷贝优化。 + +#### 下游引擎适配 + +Flink/Spark Connector 读 Variant 列时,现在拿到的是 StructVector 而非纯 Binary。需确认: +- Connector 层是否需要先把 StructVector 还原为完整 Variant 再传给引擎 +- 还是引擎侧的 `variant_get` 等 UDF 能透明地从 StructVector 中提取值 + +--- + +## 4. 列裁剪(Column Pruning)方案 + +Struct 方案下,Arrow 原生支持嵌套列裁剪。有三个层级: + +#### Level 1: Variant 列级裁剪 + +用户不选 `data` 列时,整个 StructVector 不读取(与当前一致)。 + +``` +SELECT id FROM table +→ 只读 id 列,data 的 Struct 完全跳过 +``` + +#### Level 2: Variant 内部子列裁剪(核心优势) + +如果用户只需要 `data.name`,可以利用 Arrow 的 Struct 嵌套投影: + +``` +SELECT id, variant_get(data, '$.name', 'string') FROM table +→ 读 id + data.metadata + data.typed_value.name.typed_value +→ 跳过 data.value(残余)和其他 shredded 字段 +``` + +这正是 Parquet shredding 的核心价值:**只读需要的子列,无需反序列化完整 Variant**。 + +实现方式: + +```java +// 构造 Struct 投影:只读 metadata + typed_value.name +StructVector dataVector = (StructVector) root.getVector("data"); +VarBinaryVector metadata = (VarBinaryVector) dataVector.getChild("metadata"); +StructVector typedValue = (StructVector) dataVector.getChild("typed_value"); +StructVector nameField = (StructVector) typedValue.getChild("name"); +VarCharVector nameTypedValue = (VarCharVector) nameField.getChild("typed_value"); + +// 直接从强类型列读取,零反序列化 +String name = new String(nameTypedValue.get(rowIndex), StandardCharsets.UTF_8); +``` + +#### Level 3: 兜底路径 + +当查询的字段没有被 shredded,或者类型不匹配时,回退到读取 `value`(残余 Variant 二进制): + +``` +SELECT variant_get(data, '$.email', 'string') FROM table +-- email 未被 shred +→ 读 data.metadata + data.value +→ 反序列化 value 提取 email 字段 +``` + +### 4.1 Projection Pushdown 实现思路 + +``` +用户查询: SELECT id, variant_get(data, '$.name') FROM table + +1. 查询优化器识别 variant_get 引用的字段路径: $.name +2. 查找 data 列的 shredding schema: {name: STRING, age: BIGINT} +3. name 在 shredded 字段中 → 构造 Struct 子列投影: + [metadata, typed_value.name.typed_value] +4. 发送投影到 server → server 只返回指定子列 +5. Reader 直接从 typed_value.name.typed_value 读取 + +如果查询的字段不在 shredded 列中: +1. 查询引用 $.email → 不在 shredding schema 中 +2. 构造投影: [metadata, value] +3. Reader 从 value 反序列化提取 email +``` + +--- + +## 5. 总结 + +### 5.1 实现路径 + +``` +Task 1: 升级 Arrow Java 15.0.0 → 19.0.0 + - 适配 Breaking Changes + - 引入 arrow-variant 模块 + +Task 2: 实现 Struct 内嵌 typed_value(仅标量,仅顶层字段) + - VariantType → 底层改为 StructVector 存储 + - Writer: 将字段拆分写入 Struct 内部子 Vector(仅标量类型) + - Reader: 从 Struct 内部子 Vector 合并读取(单层合并) + - 能力边界: 不支持嵌套对象/数组 shredding + +Task 3: 实现 Struct 级别的列裁剪 + - Server 端支持 Struct 子列投影下推 + - 查询优化器生成 variant field → Struct 子列的映射 + +Task 4(后续): 扩展 typed_value 能力 + - 支持嵌套对象递归 shredding + - 支持数组 shredding(ListVector 嵌套) + - 支持 DECIMAL / BINARY 等扩展类型 + - 支持 shredded 字段降级(移除不再高频的字段) +``` + +### 5.2 收益 + +| 收益 | 说明 | +|------|------| +| 对外 schema 不变 | 始终 `(id INT, data VARIANT)`,shredding 完全透明 | +| Shredding 内部完成 | typed_value 的子字段增减在 Struct 内部,不影响表级 schema | +| 细粒度列裁剪 | 只读需要的 shredded 子列,跳过其他 | +| 社区标准对齐 | 符合 `arrow.parquet.variant` 规范,利于与 Parquet 生态互通 | +| Arrow 原生投影 | 复用 Arrow Struct 嵌套投影机制,无需额外投影扩展逻辑 | + +### 5.3 Flink Connector 高效对接 + +#### 问题 + +Flink 读 shredded Variant 的路径存在三重编解码浪费: + +``` +ShreddedVariantColumnVector.getVariant(i) + → mergeVariant(): 把 typed_value 编码回 Variant 二进制 (O(fields)) + → FlussRowToFlinkRowConverter: new BinaryVariant(value, metadata) (byte[] 拷贝) + → Flink variant_get('$.name'): 重新解析二进制找字段 (O(fields)) +``` + +#### ShreddedVariant 惰性合并 + +`ShreddedVariant` 继承 `Variant`,携带 per-field typed 数据(Java 对象),仅在调用 `metadata()`/`value()` 时才执行合并: + +- `getVariant(i)` 从 Arrow 向量提取字段数据,构造 `ShreddedVariant` +- `hasShreddedField(name)` / `getTypedFieldValue(name)` — O(1) 直接访问 +- `ensureMerged()` — 惰性触发,将 shredded 数据编码回完整 Variant 二进制 + +#### FlussVariant 实现 Flink Variant 接口 + +`FlussVariant`(在 fluss-flink-2.2 模块)实现 `org.apache.flink.types.variant.Variant` 接口: + +``` +getField("name") + → ShreddedVariant.hasShreddedField("name")? + → YES: getTypedFieldValue → toPrimitiveVariant() [O(1), 无 merge] + → NO: ensureBinaryVariant().getField("name") [回退路径] +``` + +- 快速路径:shredded 字段直接从 Java 对象创建 `BinaryVariant`,跳过 `ensureMerged()` +- 慢速路径:非 shredded 字段或完整访问(`toJson()`、嵌套对象)时触发合并 + +`FlussVariantFactory` 通过反射桥接 `fluss-flink-common`(避免编译期依赖 Flink 2.2 API)。 + +#### Converter 集成 + +`FlussRowToFlinkRowConverter` 的 VARIANT 分支优先使用 `FlussVariant`: + +1. 尝试反射加载 `FlussVariantFactory`(Flink 2.2+ 可用) +2. 回退到 `BinaryVariant`(Flink 2.1+) + +#### Variant 子列投影 API + +Scan API 新增 `variantFieldProjection(Map>)` 方法,指定 Variant 列的子字段投影: + +```java +table.newScan() + .project(projectedFields) + .variantFieldProjection(columnToFields) // columnIndex → field names + .createLogScanner(); +``` + +数据流路径:`Scan` → `TableScan` → `LogScannerImpl` → `LogFetcher` → `PbFetchLogReqForTable`(含 `PbVariantFieldProjection`)→ Server `FetchParams` → `FileLogProjection` + +初始版本 `variantFieldHints` 为 null(不做子列投影),后续由 Flink planner rule 或 table properties 激活。 + +#### 优化前后数据路径对比 + +| | 优化前 | 优化后 | +|---|---|---| +| `getVariant(i)` | 立即 merge → `Variant(metadata, value)` | 构造 `ShreddedVariant`(惰性) | +| Converter | `new BinaryVariant(value, metadata)` | `new FlussVariant(shreddedVariant)` | +| Flink `variant_get('$.name')` | 解析二进制 O(fields) | `hasShreddedField` → O(1) 直接读取 | +| 编解码次数 | 3 次 | 0 次(shredded 字段) | diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java index 233d2d358d..f0e6f5b3c4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; /** * Used to configure and create a scanner to scan data for a table. @@ -72,6 +73,19 @@ public interface Scan { */ Scan filter(@Nullable Predicate predicate); + /** + * Returns a new scan from this with variant sub-field projection hints. When a Variant column + * has been shredded, specifying which sub-fields are needed allows the server to send only the + * relevant typed_value children, reducing network transfer. + * + *

The map key is the column index (in the projected or full schema), and the value is the + * list of field names to project within that Variant column. + * + * @param columnToFields mapping from Variant column index to desired field names, or null to + * disable sub-field projection + */ + Scan variantFieldProjection(@Nullable Map> columnToFields); + /** * Creates a {@link LogScanner} to continuously read log data for this scan. * diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java index 6502b7946f..595eec0c67 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java @@ -42,6 +42,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -56,12 +57,14 @@ public class TableScan implements Scan { /** The limited row number to read. No limit if is null. */ @Nullable private final Integer limit; + /** Variant sub-field projection hints. Null means no sub-field projection. */ + @Nullable private final Map> variantFieldProjection; /** The record batch filter to apply. No filter if is null. */ @Nullable private final Predicate recordBatchFilter; public TableScan(FlussConnection conn, TableInfo tableInfo, SchemaGetter schemaGetter) { - this(conn, tableInfo, schemaGetter, null, null, null); + this(conn, tableInfo, schemaGetter, null, null, null, null); } private TableScan( @@ -70,19 +73,27 @@ private TableScan( SchemaGetter schemaGetter, @Nullable int[] projectedColumns, @Nullable Integer limit, - @Nullable Predicate recordBatchFilter) { + @Nullable Predicate recordBatchFilter, + @Nullable Map> variantFieldProjection) { this.conn = conn; this.tableInfo = tableInfo; this.projectedColumns = projectedColumns; this.limit = limit; this.schemaGetter = schemaGetter; this.recordBatchFilter = recordBatchFilter; + this.variantFieldProjection = variantFieldProjection; } @Override public Scan project(@Nullable int[] projectedColumns) { return new TableScan( - conn, tableInfo, schemaGetter, projectedColumns, limit, recordBatchFilter); + conn, + tableInfo, + schemaGetter, + projectedColumns, + limit, + recordBatchFilter, + variantFieldProjection); } @Override @@ -102,18 +113,49 @@ public Scan project(List projectedColumnNames) { columnIndexes[i] = index; } return new TableScan( - conn, tableInfo, schemaGetter, columnIndexes, limit, recordBatchFilter); + conn, + tableInfo, + schemaGetter, + columnIndexes, + limit, + recordBatchFilter, + variantFieldProjection); } @Override public Scan limit(int rowNumber) { return new TableScan( - conn, tableInfo, schemaGetter, projectedColumns, rowNumber, recordBatchFilter); + conn, + tableInfo, + schemaGetter, + projectedColumns, + rowNumber, + recordBatchFilter, + variantFieldProjection); } @Override public Scan filter(@Nullable Predicate predicate) { - return new TableScan(conn, tableInfo, schemaGetter, projectedColumns, limit, predicate); + return new TableScan( + conn, + tableInfo, + schemaGetter, + projectedColumns, + limit, + predicate, + variantFieldProjection); + } + + @Override + public Scan variantFieldProjection(@Nullable Map> columnToFields) { + return new TableScan( + conn, + tableInfo, + schemaGetter, + projectedColumns, + limit, + recordBatchFilter, + columnToFields); } @Override @@ -142,7 +184,8 @@ public LogScanner createLogScanner() { conn.getOrCreateRemoteFileDownloader(), projectedColumns, schemaGetter, - recordBatchFilter); + recordBatchFilter, + variantFieldProjection); } @Override diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java index 505b82c127..358429e63c 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java @@ -50,6 +50,7 @@ import org.apache.fluss.rpc.messages.PbFetchLogReqForTable; import org.apache.fluss.rpc.messages.PbFetchLogRespForBucket; import org.apache.fluss.rpc.messages.PbFetchLogRespForTable; +import org.apache.fluss.rpc.messages.PbVariantFieldProjection; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.rpc.protocol.Errors; import org.apache.fluss.rpc.util.PredicateMessageUtils; @@ -97,6 +98,12 @@ public class LogFetcher implements Closeable { @Nullable private final Projection projection; @Nullable private final org.apache.fluss.rpc.messages.PbPredicate cachedPbPredicate; private final int filterSchemaId; + /** + * Variant sub-field projection hints. Maps Variant column index to desired sub-field names. + * Null means no sub-field projection (initial version). + */ + @Nullable private final Map> variantFieldProjection; + private final int maxFetchBytes; private final int maxBucketFetchBytes; private final int minFetchBytes; @@ -120,6 +127,7 @@ public LogFetcher( TableInfo tableInfo, @Nullable Projection projection, @Nullable Predicate recordBatchFilter, + @Nullable Map> variantFieldProjection, LogScannerStatus logScannerStatus, Configuration conf, MetadataUpdater metadataUpdater, @@ -139,6 +147,7 @@ public LogFetcher( recordBatchFilter, tableInfo.getRowType()) : null; this.filterSchemaId = tableInfo.getSchemaId(); + this.variantFieldProjection = variantFieldProjection; this.logScannerStatus = logScannerStatus; this.maxFetchBytes = (int) conf.get(ConfigOptions.CLIENT_SCANNER_LOG_FETCH_MAX_BYTES).getBytes(); @@ -555,9 +564,10 @@ Map prepareFetchLogRequests() { new PbFetchLogReqForTable().setTableId(finalTableId); if (readContext.isProjectionPushDowned()) { assert projection != null; + int[] projectedFields = projection.getProjectionInOrder(); reqForTable .setProjectionPushdownEnabled(true) - .setProjectedFields(projection.getProjectionInOrder()); + .setProjectedFields(projectedFields); } else { reqForTable.setProjectionPushdownEnabled(false); } @@ -565,6 +575,16 @@ Map prepareFetchLogRequests() { reqForTable.setFilterPredicate(cachedPbPredicate); reqForTable.setFilterSchemaId(filterSchemaId); } + // Serialize variant sub-field projection hints if present + if (variantFieldProjection != null && !variantFieldProjection.isEmpty()) { + variantFieldProjection.forEach( + (colIdx, fieldNames) -> { + PbVariantFieldProjection vfp = + reqForTable.addVariantFieldProjection(); + vfp.setColumnIndex(colIdx); + vfp.addAllFieldNames(fieldNames); + }); + } reqForTable.addAllBucketsReqs(reqForBuckets); fetchLogRequest.addAllTablesReqs(Collections.singletonList(reqForTable)); fetchLogRequests.put(leaderId, fetchLogRequest); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScan.java index 0b8a8db0f8..b932c4af02 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScan.java @@ -21,6 +21,9 @@ import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + /** * Used to describe the operation to scan log data by {@link LogScanner} to a table. * @@ -32,12 +35,21 @@ public class LogScan { /** The projected fields to do projection. No projection if is null. */ @Nullable private final int[] projectedFields; + /** + * Variant sub-field projection hints. Maps Variant column index to the list of sub-field names + * that should be projected. Null means no sub-field projection. + */ + @Nullable private final Map> variantFieldProjection; + public LogScan() { - this(null); + this(null, null); } - private LogScan(@Nullable int[] projectedFields) { + private LogScan( + @Nullable int[] projectedFields, + @Nullable Map> variantFieldProjection) { this.projectedFields = projectedFields; + this.variantFieldProjection = variantFieldProjection; } /** @@ -46,11 +58,26 @@ private LogScan(@Nullable int[] projectedFields) { * @param projectedFields the projection fields */ public LogScan withProjectedFields(int[] projectedFields) { - return new LogScan(projectedFields); + return new LogScan(projectedFields, variantFieldProjection); + } + + /** + * Returns a new instance of LogScan description with variant sub-field projection hints. + * + * @param variantFieldProjection mapping from Variant column index to desired field names + */ + public LogScan withVariantFieldProjection( + @Nullable Map> variantFieldProjection) { + return new LogScan(projectedFields, variantFieldProjection); } @Nullable public int[] getProjectedFields() { return projectedFields; } + + @Nullable + public Map> getVariantFieldProjection() { + return variantFieldProjection; + } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index 9a2dbf0b4c..ebf33304ea 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -41,6 +41,8 @@ import java.time.Duration; import java.util.Collections; import java.util.ConcurrentModificationException; +import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -83,7 +85,8 @@ public LogScannerImpl( RemoteFileDownloader remoteFileDownloader, @Nullable int[] projectedFields, SchemaGetter schemaGetter, - @Nullable Predicate recordBatchFilter) { + @Nullable Predicate recordBatchFilter, + @Nullable Map> variantFieldProjection) { this.tablePath = tableInfo.getTablePath(); this.tableId = tableInfo.getTableId(); this.isPartitionedTable = tableInfo.isPartitioned(); @@ -98,6 +101,7 @@ public LogScannerImpl( tableInfo, projection, recordBatchFilter, + variantFieldProjection, logScannerStatus, conf, metadataUpdater, @@ -112,6 +116,8 @@ public LogScannerImpl( */ @Nullable private Projection sanityProjection(@Nullable int[] projectedFields, TableInfo tableInfo) { + // Validate against the user-visible row type (excludes internal shredded columns like $v.x) + // so that projection indices from callers (e.g. Flink) stay within user-visible bounds. RowType tableRowType = tableInfo.getRowType(); if (projectedFields != null) { for (int projectedField : projectedFields) { diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java b/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java index 70988d0e25..ff350f8d8b 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java @@ -33,10 +33,13 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metrics.MetricNames; import org.apache.fluss.record.LogRecordBatchStatisticsCollector; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.arrow.ArrowWriter; import org.apache.fluss.row.arrow.ArrowWriterPool; import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.RootAllocator; +import org.apache.fluss.types.variant.ShreddingSchema; +import org.apache.fluss.types.variant.ShreddingSchemaInferrer; import org.apache.fluss.utils.CopyOnWriteMap; import org.apache.fluss.utils.MathUtils; import org.apache.fluss.utils.clock.Clock; @@ -113,6 +116,13 @@ public final class RecordAccumulator { private final Clock clock; private final DynamicWriteBatchSizeEstimator batchSizeEstimator; + /** + * Per-table {@link VariantShreddingManager}s. Created lazily on first append to an ARROW_LOG + * table with Variant shredding enabled and at least one Variant column. + */ + private final ConcurrentMap shreddingManagers = + new CopyOnWriteMap<>(); + // TODO add retryBackoffMs to retry the produce request upon receiving an error. // TODO add deliveryTimeoutMs to report success or failure on record delivery. // TODO add nextBatchExpiryTimeMs @@ -158,6 +168,71 @@ private void registerMetrics(WriterMetricGroup writerMetricGroup) { MetricNames.WRITER_BUFFER_WAITING_THREADS, writerBufferPool::queued); } + /** + * Collects Variant statistics for the row being appended, and — once enough samples have been + * observed — stores the inferred shredding schema locally in the per-table manager. + * + *

This method is a no-op when: + * + *

    + *
  • the write format is not {@link WriteFormat#ARROW_LOG} + *
  • the table has no Variant columns + *
  • Variant shredding is disabled in the table's configuration + *
+ */ + private void maybeCollectVariantStats( + PhysicalTablePath physicalTablePath, + TableInfo tableInfo, + WriteFormat writeFormat, + InternalRow row) { + if (writeFormat != WriteFormat.ARROW_LOG) { + return; + } + if (!tableInfo.isVariantShreddingEnabled()) { + return; + } + int[] variantIndices = tableInfo.getVariantColumnIndices(); + if (variantIndices.length == 0) { + return; + } + + VariantShreddingManager manager = + shreddingManagers.computeIfAbsent( + physicalTablePath, + path -> { + String[] colNames = new String[variantIndices.length]; + for (int i = 0; i < variantIndices.length; i++) { + colNames[i] = + tableInfo + .getRowType() + .getFields() + .get(variantIndices[i]) + .getName(); + } + ShreddingSchemaInferrer inferrer = + new ShreddingSchemaInferrer() + .setPresenceThreshold( + tableInfo + .getTableConfig() + .getVariantShreddingPresenceThreshold()) + .setTypeConsistencyThreshold( + tableInfo + .getTableConfig() + .getVariantShreddingTypeConsistencyThreshold()) + .setMaxShreddedFields( + tableInfo + .getTableConfig() + .getVariantShreddingMaxFields()) + .setMinSampleSize( + tableInfo + .getTableConfig() + .getVariantShreddingMinSampleSize()); + return new VariantShreddingManager( + path.getTablePath(), variantIndices, colNames, inferrer); + }); + manager.collectRow(row); + } + /** * Add a record to the accumulator, return to append result. * @@ -195,6 +270,12 @@ public RecordAppendResult append( synchronized (dq) { RecordAppendResult appendResult = tryAppend(writeRecord, callback, dq); if (appendResult != null) { + // Row was appended to an existing batch; collect Variant statistics. + maybeCollectVariantStats( + physicalTablePath, + tableInfo, + writeRecord.getWriteFormat(), + writeRecord.getRow()); return appendResult; } } @@ -212,6 +293,12 @@ public RecordAppendResult append( writeRecord, callback, bucketId, tableInfo, dq, memorySegments); if (appendResult.newBatchCreated) { memorySegments = Collections.emptyList(); + // Row was appended to the new batch; collect Variant statistics. + maybeCollectVariantStats( + physicalTablePath, + tableInfo, + writeRecord.getWriteFormat(), + writeRecord.getRow()); } return appendResult; } @@ -622,13 +709,23 @@ private WriteBatch createWriteBatch( clock.milliseconds()); case ARROW_LOG: + // Get shredding schemas from the local manager (Writer-independent decision) + Map shreddingSchemas = null; + VariantShreddingManager mgr = shreddingManagers.get(physicalTablePath); + if (mgr != null) { + Map inferred = mgr.getShreddingSchemas(); + if (!inferred.isEmpty()) { + shreddingSchemas = inferred; + } + } ArrowWriter arrowWriter = arrowWriterPool.getOrCreateWriter( tableInfo.getTableId(), schemaId, outputView.getPreAllocatedSize(), tableInfo.getRowType(), - tableInfo.getTableConfig().getArrowCompressionInfo()); + tableInfo.getTableConfig().getArrowCompressionInfo(), + shreddingSchemas); LogRecordBatchStatisticsCollector statisticsCollector = null; if (tableInfo.isStatisticsEnabled()) { statisticsCollector = diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/VariantShreddingManager.java b/fluss-client/src/main/java/org/apache/fluss/client/write/VariantShreddingManager.java new file mode 100644 index 0000000000..e21743040f --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/VariantShreddingManager.java @@ -0,0 +1,170 @@ +/* + * 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.fluss.client.write; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.variant.ShreddingSchema; +import org.apache.fluss.types.variant.ShreddingSchemaInferrer; +import org.apache.fluss.types.variant.Variant; +import org.apache.fluss.types.variant.VariantStatisticsCollector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * Manages automatic Variant shredding inference for a single table on the client write path. + * + *

Uses the Writer-independent-decision approach (Plan A): each Writer independently samples, + * infers, and decides the typed_value layout for its own batches. No server-side coordination or + * RPC is involved. + * + *

Each time a row is appended to a write batch ({@link #collectRow(InternalRow)}), this manager + * extracts the Variant values from the row's Variant-typed columns and feeds them into per-column + * {@link VariantStatisticsCollector}s. Once the minimum sample threshold is met and a non-empty + * {@link ShreddingSchema} is inferred, the result is stored locally and made available via {@link + * #getShreddingSchemas()}. + * + *

Inference is triggered at most once per manager instance. Once schemas are inferred, + * subsequent calls to {@link #collectRow} are no-ops. + * + *

Thread safety: {@link #collectRow} is called from the writer thread and is guarded by the + * deque lock in {@link RecordAccumulator}. + */ +@Internal +public class VariantShreddingManager { + + private static final Logger LOG = LoggerFactory.getLogger(VariantShreddingManager.class); + + private final TablePath tablePath; + + /** + * Column indices (into the row's schema) of all Variant-typed columns. Each index maps to the + * corresponding {@link VariantStatisticsCollector} in {@link #collectors} at the same array + * position. + */ + private final int[] variantColumnIndices; + + /** + * Names of the Variant columns, used to construct the column-name-based {@link + * ShreddingSchema}. + */ + private final String[] variantColumnNames; + + /** One statistics collector per Variant column. */ + private final VariantStatisticsCollector[] collectors; + + /** Inferrer, configured from the table's shredding options. */ + private final ShreddingSchemaInferrer inferrer; + + /** Whether inference has already been completed. */ + private volatile boolean inferenceCompleted = false; + + /** + * Locally inferred shredding schemas. Key is the variant column name. Empty until inference + * completes with non-empty results. + */ + private volatile Map inferredSchemas = Collections.emptyMap(); + + public VariantShreddingManager( + TablePath tablePath, + int[] variantColumnIndices, + String[] variantColumnNames, + ShreddingSchemaInferrer inferrer) { + this.tablePath = tablePath; + this.variantColumnIndices = variantColumnIndices; + this.variantColumnNames = variantColumnNames; + this.inferrer = inferrer; + + this.collectors = new VariantStatisticsCollector[variantColumnIndices.length]; + for (int i = 0; i < variantColumnIndices.length; i++) { + this.collectors[i] = new VariantStatisticsCollector(); + } + } + + /** + * Collects statistics from one row that is about to be (or has just been) written. + * + *

This method extracts the Variant value at each variant-column index from {@code row} and + * feeds it into the corresponding {@link VariantStatisticsCollector}. Once enough samples are + * collected and the inferrer produces a non-empty schema, the result is stored locally. + * + * @param row the row being written + */ + public void collectRow(InternalRow row) { + if (inferenceCompleted) { + return; + } + + for (int c = 0; c < variantColumnIndices.length; c++) { + int colIdx = variantColumnIndices[c]; + Variant variant = row.isNullAt(colIdx) ? null : row.getVariant(colIdx); + collectors[c].collect(variant); + } + + maybeInfer(); + } + + /** + * Returns the locally inferred shredding schemas. Returns an empty map if inference has not yet + * completed or produced no results. + * + * @return unmodifiable map of variant column name to its ShreddingSchema + */ + public Map getShreddingSchemas() { + return inferredSchemas; + } + + // -------------------------------------------------------------------------------------------- + // Internal helpers + // -------------------------------------------------------------------------------------------- + + private void maybeInfer() { + Map schemas = new HashMap<>(); + + for (int c = 0; c < variantColumnIndices.length; c++) { + VariantStatisticsCollector collector = collectors[c]; + long totalRecords = collector.getTotalRecords(); + + // Skip inference until we have enough samples to be statistically meaningful. + if (totalRecords < inferrer.getMinSampleSize()) { + return; + } + + ShreddingSchema schema = + inferrer.infer(variantColumnNames[c], collector.getStatistics(), totalRecords); + if (!schema.getFields().isEmpty()) { + schemas.put(variantColumnNames[c], schema); + } + } + + if (!schemas.isEmpty()) { + inferredSchemas = Collections.unmodifiableMap(schemas); + inferenceCompleted = true; + LOG.info( + "Inferred Variant shredding schemas for table {} (Writer-local): {}", + tablePath, + inferredSchemas); + } + } +} diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussVariantITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussVariantITCase.java new file mode 100644 index 0000000000..28f616e43b --- /dev/null +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussVariantITCase.java @@ -0,0 +1,577 @@ +/* + * 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.fluss.client.table; + +import org.apache.fluss.client.admin.ClientToServerITCaseBase; +import org.apache.fluss.client.lookup.Lookuper; +import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.client.table.scanner.log.LogScanner; +import org.apache.fluss.client.table.scanner.log.ScanRecords; +import org.apache.fluss.client.table.writer.AppendWriter; +import org.apache.fluss.client.table.writer.UpsertWriter; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.record.ChangeType; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.variant.Variant; + +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT case for Variant type end-to-end: client write to Fluss server and read back. */ +class FlussVariantITCase extends ClientToServerITCaseBase { + + // -------------------------------------------------------------------------------------------- + // Append (log table) with Variant + // -------------------------------------------------------------------------------------------- + + @Test + void testAppendAndScanVariant() throws Exception { + TablePath tablePath = TablePath.of("test_db_1", "test_variant_append"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("data", DataTypes.VARIANT()) + .build(); + TableDescriptor tableDescriptor = + TableDescriptor.builder().schema(schema).distributedBy(1).build(); + createTable(tablePath, tableDescriptor, false); + + Variant v1 = Variant.fromJson("{\"name\":\"Alice\",\"age\":30}"); + Variant v2 = Variant.fromJson("[1,2,3]"); + Variant v3 = Variant.fromJson("\"hello\""); + Variant v4 = Variant.fromJson("42"); + Variant v5 = Variant.ofNull(); + + try (Table table = conn.getTable(tablePath)) { + AppendWriter appendWriter = table.newAppend().createWriter(); + appendWriter.append(row(1, v1)); + appendWriter.append(row(2, v2)); + appendWriter.append(row(3, v3)); + appendWriter.append(row(4, v4)); + appendWriter.append(row(5, v5)); + appendWriter.flush(); + + // Read back via log scanner + LogScanner logScanner = createLogScanner(table); + subscribeFromBeginning(logScanner, table); + + List results = new ArrayList<>(); + while (results.size() < 5) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord record : scanRecords) { + assertThat(record.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); + results.add(record.getRow()); + } + } + logScanner.close(); + + // Verify variant data + assertThat(results).hasSize(5); + + // Row 1: JSON object + Variant readV1 = results.get(0).getVariant(1); + assertThat(readV1.isObject()).isTrue(); + assertThat(readV1.getFieldByName("name").getString()).isEqualTo("Alice"); + assertThat(readV1.getFieldByName("age").getByte()).isEqualTo((byte) 30); + + // Row 2: JSON array + Variant readV2 = results.get(1).getVariant(1); + assertThat(readV2.isArray()).isTrue(); + assertThat(readV2.arraySize()).isEqualTo(3); + + // Row 3: string + Variant readV3 = results.get(2).getVariant(1); + assertThat(readV3.getString()).isEqualTo("hello"); + + // Row 4: number + Variant readV4 = results.get(3).getVariant(1); + assertThat(readV4.getByte()).isEqualTo((byte) 42); + + // Row 5: variant null + Variant readV5 = results.get(4).getVariant(1); + assertThat(readV5.isNull()).isTrue(); + } + } + + // -------------------------------------------------------------------------------------------- + // Upsert and Lookup (PK table) with Variant + // -------------------------------------------------------------------------------------------- + + @Test + void testUpsertAndLookupVariant() throws Exception { + TablePath tablePath = TablePath.of("test_db_1", "test_variant_upsert"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("data", DataTypes.VARIANT()) + .primaryKey("id") + .build(); + TableDescriptor tableDescriptor = + TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); + createTable(tablePath, tableDescriptor, false); + + Variant v1 = Variant.fromJson("{\"event\":\"click\",\"count\":10}"); + Variant v2 = Variant.fromJson("[\"tag1\",\"tag2\"]"); + + try (Table table = conn.getTable(tablePath)) { + UpsertWriter upsertWriter = table.newUpsert().createWriter(); + upsertWriter.upsert(row(1, v1)); + upsertWriter.upsert(row(2, v2)); + upsertWriter.flush(); + + // Lookup by primary key + Lookuper lookuper = table.newLookup().createLookuper(); + + InternalRow result1 = lookupRow(lookuper, row(1)); + assertThat(result1).isNotNull(); + Variant readV1 = result1.getVariant(1); + assertThat(readV1.isObject()).isTrue(); + assertThat(readV1.getFieldByName("event").getString()).isEqualTo("click"); + assertThat(readV1.getFieldByName("count").getByte()).isEqualTo((byte) 10); + + InternalRow result2 = lookupRow(lookuper, row(2)); + assertThat(result2).isNotNull(); + Variant readV2 = result2.getVariant(1); + assertThat(readV2.isArray()).isTrue(); + assertThat(readV2.arraySize()).isEqualTo(2); + assertThat(readV2.getElementAt(0).getString()).isEqualTo("tag1"); + + // Update existing key with new variant value + Variant v1Updated = Variant.fromJson("{\"event\":\"click\",\"count\":20}"); + upsertWriter.upsert(row(1, v1Updated)); + upsertWriter.flush(); + + InternalRow result1Updated = lookupRow(lookuper, row(1)); + assertThat(result1Updated).isNotNull(); + Variant readV1Updated = result1Updated.getVariant(1); + assertThat(readV1Updated.getFieldByName("count").getByte()).isEqualTo((byte) 20); + } + } + + // -------------------------------------------------------------------------------------------- + // Shredding + projection read + // -------------------------------------------------------------------------------------------- + + @Test + void testShreddingWithProjectionRead() throws Exception { + TablePath tablePath = TablePath.of("test_db_1", "test_variant_shredding_projection"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("data", DataTypes.VARIANT()) + .build(); + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(1) + .property("table.variant.shredding.enabled", "true") + .property("table.variant.shredding.min-sample-size", "5") + .build(); + createTable(tablePath, tableDescriptor, false); + + int totalRecords = 15; + + // Write records with consistent STRING fields to trigger local shredding inference. + // With Writer-independent approach, the Writer infers the shredding schema locally + // and uses it for subsequent batches. No server-side schema evolution occurs. + try (Table table = conn.getTable(tablePath)) { + AppendWriter writer = table.newAppend().createWriter(); + for (int i = 0; i < totalRecords; i++) { + writer.append( + row( + i, + Variant.fromJson( + String.format( + "{\"name\":\"user_%d\",\"city\":\"city_%d\"}", + i, i)))); + } + writer.flush(); + } + + // Read back with projection — only the variant column [1] + try (Table table = conn.getTable(tablePath)) { + LogScanner logScanner = createLogScanner(table, new int[] {1}); + subscribeFromBeginning(logScanner, table); + + List results = new ArrayList<>(); + while (results.size() < totalRecords) { + ScanRecords records = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord record : records) { + assertThat(record.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); + results.add(record.getRow()); + } + } + logScanner.close(); + + // Verify all variant values are correctly reconstructed + assertThat(results).hasSize(totalRecords); + for (int i = 0; i < totalRecords; i++) { + Variant v = results.get(i).getVariant(0); // projected index 0 = "data" + assertThat(v.isObject()).isTrue(); + assertThat(v.getFieldByName("name").getString()).isEqualTo("user_" + i); + assertThat(v.getFieldByName("city").getString()).isEqualTo("city_" + i); + } + } + } + + // -------------------------------------------------------------------------------------------- + // Design-doc example: type mismatch + non-shredded fields + projection pushdown + // -------------------------------------------------------------------------------------------- + + /** + * Shredding schema inferred from warm-up data: name(STRING), age(BIGINT). Two test rows: + * + *

+     *   Row 0: {"name":"alice", "age":30,        "email":"a@b.com"}
+     *   Row 1: {"name":"bob",   "age":"unknown", "city":"beijing"}
+     *                              ↑ type mismatch BIGINT
+     * 
+ * + *

Verifies 4 read scenarios from the design doc: + * + *

    + *
  1. Read "name" — shredded, type match → zero deserialization path + *
  2. Read "age" — shredded, type mismatch in Row 1 → per-field value fallback + *
  3. Read "email"/"city" — non-shredded → residual decode path + *
  4. Full reconstruction — merge shredded + residual + *
+ * + *

Also validates column-level projection pushdown ({@code [1]} = variant-only). + */ + @Test + void testShreddingDesignDocExampleWithProjection() throws Exception { + TablePath tablePath = + TablePath.of("test_db_1", "test_variant_shredding_design_doc_example"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("data", DataTypes.VARIANT()) + .build(); + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(1) + .property("table.variant.shredding.enabled", "true") + .property("table.variant.shredding.min-sample-size", "5") + .build(); + createTable(tablePath, tableDescriptor, false); + + int warmUpCount = 10; + int totalRecords = warmUpCount + 2; + + // Phase 1: write warm-up rows to trigger shredding inference for name(STRING), age(BIGINT) + // Phase 2: write the 2 design-doc example rows through the shredded writer path + try (Table table = conn.getTable(tablePath)) { + AppendWriter writer = table.newAppend().createWriter(); + for (int i = 0; i < warmUpCount; i++) { + writer.append( + row( + i, + Variant.fromJson( + String.format( + "{\"name\":\"user_%d\",\"age\":%d}", i, i * 10)))); + } + writer.flush(); + + // These 2 rows exercise shredding with type mismatch and non-shredded fields + writer.append( + row( + 100, + Variant.fromJson( + "{\"name\":\"alice\",\"age\":30,\"email\":\"a@b.com\"}"))); + writer.append( + row( + 101, + Variant.fromJson( + "{\"name\":\"bob\",\"age\":\"unknown\",\"city\":\"beijing\"}"))); + writer.flush(); + } + + // ---- Read scenario A: full scan (no column projection) ---- + try (Table table = conn.getTable(tablePath)) { + LogScanner logScanner = createLogScanner(table); + subscribeFromBeginning(logScanner, table); + + List results = new ArrayList<>(); + while (results.size() < totalRecords) { + ScanRecords records = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord record : records) { + results.add(record.getRow()); + } + } + logScanner.close(); + + assertThat(results).hasSize(totalRecords); + // variantIndex = 1 (schema: [id, data]) + verifyDesignDocRow0(results.get(warmUpCount).getVariant(1)); + verifyDesignDocRow1(results.get(warmUpCount + 1).getVariant(1)); + } + + // ---- Read scenario B: projection — only variant column [1] ---- + try (Table table = conn.getTable(tablePath)) { + LogScanner logScanner = createLogScanner(table, new int[] {1}); + subscribeFromBeginning(logScanner, table); + + List results = new ArrayList<>(); + while (results.size() < totalRecords) { + ScanRecords records = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord record : records) { + results.add(record.getRow()); + } + } + logScanner.close(); + + assertThat(results).hasSize(totalRecords); + // projected index 0 = "data" + verifyDesignDocRow0(results.get(warmUpCount).getVariant(0)); + verifyDesignDocRow1(results.get(warmUpCount + 1).getVariant(0)); + } + + // ---- Read scenario C: projection — only id column [0] ---- + try (Table table = conn.getTable(tablePath)) { + LogScanner logScanner = createLogScanner(table, new int[] {0}); + subscribeFromBeginning(logScanner, table); + + List results = new ArrayList<>(); + while (results.size() < totalRecords) { + ScanRecords records = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord record : records) { + results.add(record.getRow()); + } + } + logScanner.close(); + + assertThat(results).hasSize(totalRecords); + assertThat(results.get(warmUpCount).getInt(0)).isEqualTo(100); + assertThat(results.get(warmUpCount + 1).getInt(0)).isEqualTo(101); + } + } + + /** + * Verify design-doc Row 0: {"name":"alice", "age":30, "email":"a@b.com"}. + * + *

    + *
  • "name" — shredded STRING, type match + *
  • "age" — shredded BIGINT, type match (encoded as INT8 or INT64 depending on path) + *
  • "email" — non-shredded, from residual + *
  • "city" — absent + *
+ */ + private void verifyDesignDocRow0(Variant v) { + assertThat(v.isObject()).isTrue(); + assertThat(v.getFieldByName("name").getString()).isEqualTo("alice"); + // age=30 may be INT8 (un-shredded) or INT64 (shredded), verify via toJson() + assertThat(v.getFieldByName("age").toJson()).isEqualTo("30"); + assertThat(v.getFieldByName("email").getString()).isEqualTo("a@b.com"); + assertThat(v.getFieldByName("city")).isNull(); + } + + /** + * Verify design-doc Row 1: {"name":"bob", "age":"unknown", "city":"beijing"}. + * + *
    + *
  • "name" — shredded STRING, type match + *
  • "age" — shredded BIGINT but value is STRING → type mismatch, per-field value fallback + *
  • "email" — absent + *
  • "city" — non-shredded, from residual + *
+ */ + private void verifyDesignDocRow1(Variant v) { + assertThat(v.isObject()).isTrue(); + assertThat(v.getFieldByName("name").getString()).isEqualTo("bob"); + assertThat(v.getFieldByName("age").getString()).isEqualTo("unknown"); + assertThat(v.getFieldByName("email")).isNull(); + assertThat(v.getFieldByName("city").getString()).isEqualTo("beijing"); + } + + // -------------------------------------------------------------------------------------------- + // Variant sub-field projection API + // -------------------------------------------------------------------------------------------- + + /** + * Verifies that the {@code variantFieldProjection} API works end-to-end. + * + *

The API passes sub-field hints to the server so that only the relevant {@code typed_value} + * children are transferred when a Variant column has been shredded. The hint is a transfer + * optimisation; the full variant value is still returned. + * + *

Two read scenarios: + * + *

    + *
  1. Column projection {@code [1]} + sub-field hint {@code "name"} + *
  2. Full schema (no column projection) + sub-field hint {@code "name","city"} + *
+ */ + @Test + void testVariantSubFieldProjection() throws Exception { + TablePath tablePath = TablePath.of("test_db_1", "test_variant_sub_field_projection"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("data", DataTypes.VARIANT()) + .build(); + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(1) + .property("table.variant.shredding.enabled", "true") + .property("table.variant.shredding.min-sample-size", "5") + .build(); + createTable(tablePath, tableDescriptor, false); + + int warmUpCount = 10; + int testCount = 5; + int totalRecords = warmUpCount + testCount; + + // Phase 1: warm-up to trigger shredding inference for name(STRING), city(STRING) + // Phase 2: test rows through the shredded writer path + try (Table table = conn.getTable(tablePath)) { + AppendWriter writer = table.newAppend().createWriter(); + for (int i = 0; i < warmUpCount; i++) { + writer.append( + row( + i, + Variant.fromJson( + String.format( + "{\"name\":\"user_%d\",\"city\":\"city_%d\",\"score\":%d}", + i, i, i * 10)))); + } + writer.flush(); + + for (int i = 0; i < testCount; i++) { + int idx = warmUpCount + i; + writer.append( + row( + idx, + Variant.fromJson( + String.format( + "{\"name\":\"user_%d\",\"city\":\"city_%d\",\"score\":%d}", + idx, idx, idx * 10)))); + } + writer.flush(); + } + + // ---- Scenario A: column projection [1] + sub-field hint {1 -> ["name"]} ---- + try (Table table = conn.getTable(tablePath)) { + Map> variantHints = new HashMap<>(); + variantHints.put(1, Arrays.asList("name")); + + LogScanner logScanner = + table.newScan() + .project(new int[] {1}) + .variantFieldProjection(variantHints) + .createLogScanner(); + subscribeFromBeginning(logScanner, table); + + List results = new ArrayList<>(); + while (results.size() < totalRecords) { + ScanRecords records = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord record : records) { + assertThat(record.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); + results.add(record.getRow()); + } + } + logScanner.close(); + + assertThat(results).hasSize(totalRecords); + // Sub-field projection is a server-side transfer hint; + // the full variant value is still returned. + for (int i = 0; i < totalRecords; i++) { + Variant v = results.get(i).getVariant(0); // projected index 0 = "data" + assertThat(v.isObject()).isTrue(); + assertThat(v.getFieldByName("name").getString()).isEqualTo("user_" + i); + } + } + + // ---- Scenario B: full schema + sub-field hint {1 -> ["name","city"]} ---- + try (Table table = conn.getTable(tablePath)) { + Map> variantHints = new HashMap<>(); + variantHints.put(1, Arrays.asList("name", "city")); + + LogScanner logScanner = + table.newScan().variantFieldProjection(variantHints).createLogScanner(); + subscribeFromBeginning(logScanner, table); + + List results = new ArrayList<>(); + while (results.size() < totalRecords) { + ScanRecords records = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord record : records) { + results.add(record.getRow()); + } + } + logScanner.close(); + + assertThat(results).hasSize(totalRecords); + for (int i = 0; i < totalRecords; i++) { + Variant v = results.get(i).getVariant(1); // full schema index 1 = "data" + assertThat(v.isObject()).isTrue(); + assertThat(v.getFieldByName("name").getString()).isEqualTo("user_" + i); + assertThat(v.getFieldByName("city").getString()).isEqualTo("city_" + i); + } + } + } + + // -------------------------------------------------------------------------------------------- + // Null Variant column value (SQL NULL, not JSON null) + // -------------------------------------------------------------------------------------------- + + @Test + void testNullVariantColumn() throws Exception { + TablePath tablePath = TablePath.of("test_db_1", "test_variant_null_column"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("data", DataTypes.VARIANT()) + .primaryKey("id") + .build(); + TableDescriptor tableDescriptor = + TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); + createTable(tablePath, tableDescriptor, false); + + try (Table table = conn.getTable(tablePath)) { + UpsertWriter upsertWriter = table.newUpsert().createWriter(); + // null column value (SQL NULL, not JSON null) + upsertWriter.upsert(row(1, null)); + upsertWriter.upsert(row(2, Variant.fromJson("{\"key\":\"value\"}"))); + upsertWriter.flush(); + + Lookuper lookuper = table.newLookup().createLookuper(); + + InternalRow result1 = lookupRow(lookuper, row(1)); + assertThat(result1).isNotNull(); + assertThat(result1.isNullAt(1)).isTrue(); + + InternalRow result2 = lookupRow(lookuper, row(2)); + assertThat(result2).isNotNull(); + assertThat(result2.isNullAt(1)).isFalse(); + assertThat(result2.getVariant(1).isObject()).isTrue(); + } + } +} diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterITCase.java index ac64fbdbdb..e28575c84f 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterITCase.java @@ -112,6 +112,7 @@ protected void setup() throws Exception { DATA1_TABLE_INFO, null, // projection recordBatchFilter, // recordBatchFilter + null, // variantFieldProjection logScannerStatus, clientConf, metadataUpdater, @@ -434,6 +435,7 @@ private LogFetcher createFetcherWithBuckets(Map scanBuckets) DATA1_TABLE_INFO, null, recordBatchFilter, + null, scannerStatus, clientConf, metadataUpdater, diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java index 50addfcfe0..f320a4546a 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java @@ -104,6 +104,7 @@ protected void setup() throws Exception { DATA1_TABLE_INFO, null, null, + null, logScannerStatus, clientConf, metadataUpdater, @@ -179,6 +180,7 @@ void testFetchWithSchemaChange() throws Exception { DATA1_TABLE_INFO.getModifiedTime()), null, null, + null, logScannerStatus, clientConf, metadataUpdater, @@ -278,6 +280,7 @@ void testFetchWhenDestinationIsNullInMetadata() throws Exception { DATA1_TABLE_INFO, null, null, + null, logScannerStatus, clientConf, metadataUpdater, @@ -319,6 +322,7 @@ void testFetchWithInvalidTableOrPartitions() throws Exception { DATA1_TABLE_INFO, null, null, + null, logScannerStatus, clientConf, metadataUpdater1, diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java index 576eab5dae..610854d7ac 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java @@ -80,6 +80,7 @@ public void setup() { DATA1_TABLE_INFO, null, null, + null, logScannerStatus, new Configuration(), metadataUpdater, diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index 8da2246b0a..3dfb8c9109 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -1595,6 +1595,57 @@ public class ConfigOptions { + "Downstream consumers must be upgraded to Fluss v1.0+ before enabling this option, " + "as older versions cannot parse the extended batch format."); + // ------------------------------------------------------------------------ + // ConfigOptions for Variant Shredding + // ------------------------------------------------------------------------ + + public static final ConfigOption TABLE_VARIANT_SHREDDING_ENABLED = + key("table.variant.shredding.enabled") + .booleanType() + .defaultValue(true) + .withDescription( + "Whether to enable automatic shredding for Variant columns. " + + "When enabled, the system will analyze Variant data during writes " + + "and automatically extract frequently occurring fields with consistent types " + + "into independent typed columns for efficient columnar access."); + + public static final ConfigOption TABLE_VARIANT_SHREDDING_PRESENCE_THRESHOLD = + key("table.variant.shredding.presence-threshold") + .doubleType() + .defaultValue(0.5) + .withDescription( + "The minimum presence ratio required for a Variant field to be considered for shredding. " + + "A field must appear in at least this fraction of total records " + + "to be eligible. Value range: 0.0 to 1.0. Default is 0.5 (50%)."); + + public static final ConfigOption TABLE_VARIANT_SHREDDING_TYPE_CONSISTENCY_THRESHOLD = + key("table.variant.shredding.type-consistency-threshold") + .doubleType() + .defaultValue(0.9) + .withDescription( + "The minimum type consistency ratio required for a Variant field to be considered for shredding. " + + "At least this fraction of the field's occurrences must have the same type. " + + "Value range: 0.0 to 1.0. Default is 0.9 (90%)."); + + public static final ConfigOption TABLE_VARIANT_SHREDDING_MAX_FIELDS = + key("table.variant.shredding.max-fields") + .intType() + .defaultValue(100) + .withDescription( + "The maximum number of fields that can be shredded from a single Variant column. " + + "Fields are ranked by presence ratio * type consistency, " + + "and only the top-N are selected. Default is 100."); + + public static final ConfigOption TABLE_VARIANT_SHREDDING_MIN_SAMPLE_SIZE = + key("table.variant.shredding.min-sample-size") + .intType() + .defaultValue(1000) + .withDescription( + "The minimum number of records that must be sampled before " + + "the shredding inference algorithm can run. " + + "This ensures reliable statistics before making schema changes. " + + "Default is 1000."); + // ------------------------------------------------------------------------ // ConfigOptions for Kv // ------------------------------------------------------------------------ diff --git a/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java b/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java index 13cb49e9ed..06aec526f0 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java @@ -184,4 +184,49 @@ public StatisticsColumnsConfig getStatisticsColumns() { .collect(Collectors.toList()); return StatisticsColumnsConfig.of(columns); } + + /** + * Returns whether automatic Variant shredding is enabled for this table. + * + *

When enabled, the write path will collect field statistics for Variant columns and + * automatically trigger schema evolution to extract frequently-occurring, type-consistent + * fields into independent typed columns. + */ + public boolean isVariantShreddingEnabled() { + return config.get(ConfigOptions.TABLE_VARIANT_SHREDDING_ENABLED); + } + + /** + * Returns the minimum presence ratio threshold for Variant field shredding. + * + *

A Variant field must appear in at least this fraction of total records to be eligible for + * shredding. Range: 0.0 to 1.0. + */ + public float getVariantShreddingPresenceThreshold() { + return config.get(ConfigOptions.TABLE_VARIANT_SHREDDING_PRESENCE_THRESHOLD).floatValue(); + } + + /** + * Returns the minimum type-consistency ratio threshold for Variant field shredding. + * + *

At least this fraction of a field's occurrences must share the same type. Range: 0.0 to + * 1.0. + */ + public float getVariantShreddingTypeConsistencyThreshold() { + return config.get(ConfigOptions.TABLE_VARIANT_SHREDDING_TYPE_CONSISTENCY_THRESHOLD) + .floatValue(); + } + + /** Returns the maximum number of fields that can be shredded from a single Variant column. */ + public int getVariantShreddingMaxFields() { + return config.get(ConfigOptions.TABLE_VARIANT_SHREDDING_MAX_FIELDS); + } + + /** + * Returns the minimum number of records that must be observed before the shredding inferrer + * produces a schema. + */ + public int getVariantShreddingMinSampleSize() { + return config.get(ConfigOptions.TABLE_VARIANT_SHREDDING_MIN_SAMPLE_SIZE); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java b/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java index 00f58b81f0..a83367673c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java @@ -23,6 +23,7 @@ import org.apache.fluss.config.TableConfig; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DataTypeChecks; +import org.apache.fluss.types.DataTypeRoot; import org.apache.fluss.types.RowType; import javax.annotation.Nullable; @@ -69,7 +70,10 @@ public final class TableInfo { private final long createdTime; private final long modifiedTime; - private int[] cachedStatsIndexMapping = null; + private volatile int[] cachedStatsIndexMapping = null; + + /** Cached indices (into {@link #rowType}) of Variant-type columns. Computed lazily. */ + private volatile int[] cachedVariantColumnIndices = null; public TableInfo( TablePath tablePath, @@ -317,6 +321,38 @@ public int[] getStatsIndexMapping() { return mapping; } + /** + * Returns whether automatic Variant shredding is enabled for this table. + * + *

Shredding is only meaningful for ARROW_LOG tables that contain at least one Variant + * column. Callers should also check {@link #getVariantColumnIndices()} to confirm there are + * Variant columns before setting up shredding infrastructure. + */ + public boolean isVariantShreddingEnabled() { + return tableConfig.isVariantShreddingEnabled(); + } + + /** + * Returns the column indices (into {@link #getRowType()}) of all Variant-type columns. + * + *

The result is cached after the first call. + * + * @return an array of zero-based column indices; empty if there are no Variant columns. + */ + public int[] getVariantColumnIndices() { + if (cachedVariantColumnIndices != null) { + return cachedVariantColumnIndices; + } + List indices = new ArrayList<>(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + if (rowType.getTypeAt(i).getTypeRoot() == DataTypeRoot.VARIANT) { + indices.add(i); + } + } + cachedVariantColumnIndices = indices.stream().mapToInt(Integer::intValue).toArray(); + return cachedVariantColumnIndices; + } + /** * Get the partition keys of the table. This will be an empty set if the table is not * partitioned. diff --git a/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatch.java b/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatch.java index 288c4b9060..bcbb5867a9 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatch.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatch.java @@ -439,6 +439,7 @@ public void close() { // Clear old buffers before the next batch load to avoid temporary // duplication of buffers (old + new) during loadFieldBuffers. root.clear(); + reader.close(); } } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatchStatistics.java b/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatchStatistics.java index b242545eaf..d777362a9d 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatchStatistics.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatchStatistics.java @@ -27,6 +27,7 @@ import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.row.aligned.AlignedRow; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.Variant; import java.util.Arrays; @@ -461,5 +462,11 @@ public InternalRow getRow(int pos, int numFields) { ensureColumnExists(pos); return internalRow.getRow(reversedStatsIndexMapping[pos], numFields); } + + @Override + public Variant getVariant(int pos) { + ensureColumnExists(pos); + return internalRow.getVariant(reversedStatsIndexMapping[pos]); + } } } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java b/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java index bedaac4753..14b2b9a121 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java @@ -28,6 +28,7 @@ import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.Buffer; import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.FieldNode; import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.Message; +import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.MessageHeader; import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.RecordBatch; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.TypeLayout; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.compression.CompressionUtil; @@ -43,6 +44,8 @@ import org.apache.fluss.utils.ArrowUtils; import org.apache.fluss.utils.types.Tuple2; +import javax.annotation.Nullable; + import java.io.ByteArrayOutputStream; import java.io.EOFException; import java.io.IOException; @@ -54,6 +57,7 @@ import java.util.Arrays; import java.util.BitSet; import java.util.List; +import java.util.Map; import static org.apache.fluss.record.DefaultLogRecordBatch.APPEND_ONLY_FLAG_MASK; import static org.apache.fluss.record.LogRecordBatchFormat.LENGTH_OFFSET; @@ -105,6 +109,12 @@ public class FileLogProjection { private long tableId; private ArrowCompressionInfo compressionInfo; private int[] selectedFieldPositions; + /** + * Variant sub-field projection hints. Maps Variant column index to the list of sub-field names + * to project within that Variant's typed_value StructVector. Null means project all sub-fields + * (no filtering). + */ + @Nullable private Map> variantFieldProjection; public FileLogProjection(ProjectionPushdownCache projectionsCache) { this.projectionsCache = projectionsCache; @@ -121,10 +131,27 @@ public void setCurrentProjection( SchemaGetter schemaGetter, ArrowCompressionInfo compressionInfo, int[] selectedFieldPositions) { + setCurrentProjection(tableId, schemaGetter, compressionInfo, selectedFieldPositions, null); + } + + public void setCurrentProjection( + long tableId, + SchemaGetter schemaGetter, + ArrowCompressionInfo compressionInfo, + int[] selectedFieldPositions, + @Nullable Map> variantFieldProjection) { + // Validate projection against the latest schema to catch genuinely invalid + // projections (out-of-bound, non-ascending, duplicated) early. + // Per-batch schema evolution filtering is handled in createProjectionInfo. + int latestFieldCount = + schemaGetter.getLatestSchemaInfo().getSchema().getRowType().getFieldCount(); + toBitSet(latestFieldCount, selectedFieldPositions); + this.tableId = tableId; this.schemaGetter = schemaGetter; this.compressionInfo = compressionInfo; this.selectedFieldPositions = selectedFieldPositions; + this.variantFieldProjection = variantFieldProjection; } /** @@ -290,18 +317,68 @@ private int projectSingleBatch( arrowMetadataBuffer.rewind(); Message metadata = Message.getRootAsMessage(arrowMetadataBuffer); + + // Handle embedded Schema IPC prefix (present when variant shredding + // is active). The writer serializes Schema + RecordBatch for shredded + // batches. We must: + // (1) compute projection from the actual (shredded) schema, + // (2) skip past the Schema to the RecordBatch, + // (3) optionally include a projected Schema prefix in the output. + byte[] projectedSchemaPrefix = null; + ProjectionInfo effectiveProjection = currentProjection; + long recordBatchBodyOffset; + + if (metadata.headerType() == MessageHeader.Schema) { + Schema actualSchema = MessageSerializer.deserializeSchema(metadata); + effectiveProjection = createProjectionInfoFromActualSchema(actualSchema); + + // Include Schema prefix only when projected fields contain + // Variant columns with shredded typed_value children. + if (projectedFieldsHaveShredding( + actualSchema, effectiveProjection.selectedFieldPositions)) { + projectedSchemaPrefix = serializeProjectedSchemaPrefix(actualSchema); + } + + // Advance past the Schema IPC message to the RecordBatch + int paddedMetadataSize = (arrowMetadataSize + 7) & ~7; + long rbOffset = arrowHeaderOffset + ARROW_HEADER_SIZE + paddedMetadataSize; + + // Re-read for the RecordBatch IPC message + arrowHeaderBuffer.rewind(); + readFullyOrFail(channel, arrowHeaderBuffer, rbOffset, "record batch arrow header"); + arrowHeaderBuffer.position(ARROW_IPC_METADATA_SIZE_OFFSET); + arrowMetadataSize = arrowHeaderBuffer.getInt(); + + resizeArrowMetadataBuffer(arrowMetadataSize); + arrowMetadataBuffer.rewind(); + readFullyOrFail( + channel, + arrowMetadataBuffer, + rbOffset + ARROW_HEADER_SIZE, + "record batch metadata"); + arrowMetadataBuffer.rewind(); + metadata = Message.getRootAsMessage(arrowMetadataBuffer); + + recordBatchBodyOffset = rbOffset + ARROW_HEADER_SIZE + arrowMetadataSize; + } else { + recordBatchBodyOffset = arrowHeaderOffset + ARROW_HEADER_SIZE + arrowMetadataSize; + } + + // Project the RecordBatch ProjectedArrowBatch projectedArrowBatch = projectArrowBatch( metadata, - currentProjection.nodesProjection, - currentProjection.buffersProjection, - currentProjection.bufferCount); + effectiveProjection.nodesProjection, + effectiveProjection.buffersProjection, + effectiveProjection.bufferCount); long arrowBodyLength = projectedArrowBatch.bodyLength(); + int schemaPrefixLen = projectedSchemaPrefix != null ? projectedSchemaPrefix.length : 0; int newBatchSizeInBytes = recordBatchHeaderSize + changeTypeBytes - + currentProjection.arrowMetadataLength + + schemaPrefixLen + + effectiveProjection.arrowMetadataLength + (int) arrowBodyLength; if (newBatchSizeInBytes > maxBytes) { @@ -311,9 +388,9 @@ private int projectSingleBatch( // create new arrow batch metadata which already projected byte[] headerMetadata = serializeArrowRecordBatchMetadata( - projectedArrowBatch, arrowBodyLength, currentProjection.bodyCompression); + projectedArrowBatch, arrowBodyLength, effectiveProjection.bodyCompression); checkState( - headerMetadata.length == currentProjection.arrowMetadataLength, + headerMetadata.length == effectiveProjection.arrowMetadataLength, "Invalid metadata length"); // update and copy log batch header @@ -332,10 +409,13 @@ private int projectSingleBatch( if (!isAppendOnly) { builder.addBytes(channel, position + recordsStartOffset, changeTypeBytes); } + if (projectedSchemaPrefix != null) { + builder.addBytes(projectedSchemaPrefix); + } builder.addBytes(headerMetadata); - final long bufferOffset = arrowHeaderOffset + ARROW_HEADER_SIZE + arrowMetadataSize; + final long bodyBaseOffset = recordBatchBodyOffset; projectedArrowBatch.buffers.forEach( - b -> builder.addBytes(channel, bufferOffset + b.getOffset(), (int) b.getSize())); + b -> builder.addBytes(channel, bodyBaseOffset + b.getOffset(), (int) b.getSize())); return newBatchSizeInBytes; } @@ -506,7 +586,21 @@ private ProjectionInfo createProjectionInfo(short schemaId, int[] selectedFieldP // initialize the projection util information Schema arrowSchema = ArrowUtils.toArrowSchema(rowType); - BitSet selection = toBitSet(arrowSchema.getFields().size(), selectedFieldPositions); + int schemaFieldCount = arrowSchema.getFields().size(); + + // Filter projection indices to valid range for this schema version. + // This handles schema evolution: data written before shredding doesn't have + // shredded columns, so those indices are simply skipped. + int[] effectivePositions = selectedFieldPositions; + if (selectedFieldPositions.length > 0 + && selectedFieldPositions[selectedFieldPositions.length - 1] >= schemaFieldCount) { + effectivePositions = + Arrays.stream(selectedFieldPositions) + .filter(i -> i < schemaFieldCount) + .toArray(); + } + + BitSet selection = toBitSet(schemaFieldCount, effectivePositions); List> flattenedFields = new ArrayList<>(); flattenFields(arrowSchema.getFields(), selection, flattenedFields); int totalFieldNodes = flattenedFields.size(); @@ -529,8 +623,7 @@ private ProjectionInfo createProjectionInfo(short schemaId, int[] selectedFieldP bufferIndex += bufferLayoutCount[i]; } - Schema projectedArrowSchema = - ArrowUtils.toArrowSchema(rowType.project(selectedFieldPositions)); + Schema projectedArrowSchema = ArrowUtils.toArrowSchema(rowType.project(effectivePositions)); ArrowBodyCompression bodyCompression = CompressionUtil.createBodyCompression(compressionInfo.createCompressionCodec()); int metadataLength = @@ -541,7 +634,101 @@ private ProjectionInfo createProjectionInfo(short schemaId, int[] selectedFieldP bufferIndex, metadataLength, bodyCompression, - selectedFieldPositions); + effectivePositions); + } + + /** + * Compute a per-batch {@link ProjectionInfo} from the actual Arrow Schema embedded in a + * shredded batch. Unlike {@link #createProjectionInfo} (which derives the schema from RowType), + * this accounts for typed_value children in Variant columns. + */ + private ProjectionInfo createProjectionInfoFromActualSchema(Schema actualSchema) { + int schemaFieldCount = actualSchema.getFields().size(); + + int[] effectivePositions = selectedFieldPositions; + if (selectedFieldPositions.length > 0 + && selectedFieldPositions[selectedFieldPositions.length - 1] >= schemaFieldCount) { + effectivePositions = + Arrays.stream(selectedFieldPositions) + .filter(i -> i < schemaFieldCount) + .toArray(); + } + + BitSet selection = toBitSet(schemaFieldCount, effectivePositions); + List> flattenedFields = new ArrayList<>(); + flattenFields(actualSchema.getFields(), selection, flattenedFields); + int totalFieldNodes = flattenedFields.size(); + int[] bufferLayoutCount = new int[totalFieldNodes]; + BitSet nodesProjection = new BitSet(totalFieldNodes); + int totalBuffers = 0; + for (int i = 0; i < totalFieldNodes; i++) { + Field fieldNode = flattenedFields.get(i).f0; + boolean selected = flattenedFields.get(i).f1; + nodesProjection.set(i, selected); + bufferLayoutCount[i] = TypeLayout.getTypeBufferCount(fieldNode.getType()); + totalBuffers += bufferLayoutCount[i]; + } + BitSet buffersProjection = new BitSet(totalBuffers); + int bufferIndex = 0; + for (int i = 0; i < totalFieldNodes; i++) { + if (nodesProjection.get(i)) { + buffersProjection.set(bufferIndex, bufferIndex + bufferLayoutCount[i]); + } + bufferIndex += bufferLayoutCount[i]; + } + + List projectedFields = new ArrayList<>(); + for (int pos : effectivePositions) { + projectedFields.add(actualSchema.getFields().get(pos)); + } + Schema projectedSchema = new Schema(projectedFields); + ArrowBodyCompression bodyCompression = + CompressionUtil.createBodyCompression(compressionInfo.createCompressionCodec()); + int metadataLength = + ArrowUtils.estimateArrowMetadataLength(projectedSchema, bodyCompression); + return new ProjectionInfo( + nodesProjection, + buffersProjection, + bufferIndex, + metadataLength, + bodyCompression, + effectivePositions); + } + + /** + * Check whether any of the projected fields has a "typed_value" child, indicating that the + * projected output needs a Schema IPC prefix for correct deserialization. + */ + private static boolean projectedFieldsHaveShredding( + Schema actualSchema, int[] selectedPositions) { + for (int pos : selectedPositions) { + if (pos < actualSchema.getFields().size()) { + for (Field child : actualSchema.getFields().get(pos).getChildren()) { + if ("typed_value".equals(child.getName())) { + return true; + } + } + } + } + return false; + } + + /** + * Serialize the projected actual Schema as an Arrow IPC Schema message. This prefix is included + * in the projected output so the reader can create a VectorSchemaRoot with the correct + * structure (including typed_value children). + */ + private byte[] serializeProjectedSchemaPrefix(Schema actualSchema) throws IOException { + List projectedFields = new ArrayList<>(); + for (int pos : selectedFieldPositions) { + if (pos < actualSchema.getFields().size()) { + projectedFields.add(actualSchema.getFields().get(pos)); + } + } + Schema projectedSchema = new Schema(projectedFields); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + MessageSerializer.serialize(new WriteChannel(Channels.newChannel(baos)), projectedSchema); + return baos.toByteArray(); } /** Projection pushdown information for a specific schema and selected fields. */ diff --git a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordReadContext.java b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordReadContext.java index 71536c8122..5e2fc17beb 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordReadContext.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordReadContext.java @@ -49,7 +49,7 @@ public class LogRecordReadContext implements LogRecordBatch.ReadContext, AutoClo // the log format of the table private final LogFormat logFormat; - // the schema of the date read form server or remote. (which is projected in the server side) + // the schema of the data read from server or remote private final RowType dataRowType; // the static schemaId of the table, should support dynamic schema evolution in the future private final int targetSchemaId; @@ -74,7 +74,6 @@ public static LogRecordReadContext createReadContext( boolean projectionPushDowned = projection != null && logFormat == LogFormat.ARROW; int schemaId = tableInfo.getSchemaId(); if (projection == null) { - // set a default dummy projection to simplify code projection = Projection.of(IntStream.range(0, rowType.getFieldCount()).toArray()); } @@ -252,10 +251,13 @@ public LogFormat getLogFormat() { @Override public RowType getRowType(int schemaId) { - if (isSameRowType(schemaId)) { + if (targetSchemaId == schemaId) { + return dataRowType; + } + if (projectionPushDowned) { + // Non-shredding pushdown: all schemas have the same projected shape return dataRowType; } - Schema schema = schemaGetter.getSchema(schemaId); return schema.getRowType(); } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/BinaryArray.java b/fluss-common/src/main/java/org/apache/fluss/row/BinaryArray.java index f33300287c..c79ddd375a 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/BinaryArray.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/BinaryArray.java @@ -21,6 +21,7 @@ import org.apache.fluss.memory.MemorySegment; import org.apache.fluss.row.array.PrimitiveBinaryArray; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.variant.Variant; import java.lang.reflect.Array; @@ -90,6 +91,7 @@ public static int calculateFixLengthPartSize(DataType type) { case ARRAY: case MAP: case ROW: + case VARIANT: // long and double are 8 bytes; // otherwise it stores the length and offset of the variable-length part for types // such as is string, map, etc. @@ -233,6 +235,11 @@ public byte[] getBytes(int pos) { return BinarySegmentUtils.readBinary(segments, offset, fieldOffset, offsetAndSize); } + @Override + public Variant getVariant(int pos) { + return Variant.fromBytes(getBytes(pos)); + } + @Override public TimestampNtz getTimestampNtz(int pos, int precision) { assertIndexIsValid(pos); diff --git a/fluss-common/src/main/java/org/apache/fluss/row/BinaryWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/BinaryWriter.java index 1064750c1b..40ad7aea59 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/BinaryWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/BinaryWriter.java @@ -26,6 +26,7 @@ import org.apache.fluss.types.DataType; import org.apache.fluss.types.MapType; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.Variant; import javax.annotation.Nullable; @@ -184,6 +185,8 @@ static BinaryWriter.ValueWriter createNotNullValueWriter( rowType.getFieldTypes().toArray(new DataType[0]), rowFormat); return (writer, pos, value) -> writer.writeRow(pos, (InternalRow) value, rowSerializer); + case VARIANT: + return (writer, pos, value) -> writer.writeBytes(pos, ((Variant) value).toBytes()); default: String msg = String.format( diff --git a/fluss-common/src/main/java/org/apache/fluss/row/DataGetters.java b/fluss-common/src/main/java/org/apache/fluss/row/DataGetters.java index 5af9dcdcd4..aca9a03f0d 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/DataGetters.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/DataGetters.java @@ -19,6 +19,7 @@ package org.apache.fluss.row; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.types.variant.Variant; /** * Getters to get data. @@ -100,4 +101,7 @@ public interface DataGetters { /** Returns the row value at the given position. */ InternalRow getRow(int pos, int numFields); + + /** Returns the variant value at the given position. */ + Variant getVariant(int pos); } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/GenericArray.java b/fluss-common/src/main/java/org/apache/fluss/row/GenericArray.java index 08c5a90271..fb0e14787b 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/GenericArray.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/GenericArray.java @@ -20,6 +20,7 @@ import org.apache.fluss.annotation.PublicEvolving; import org.apache.fluss.types.ArrayType; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.ArrayUtils; import java.io.Serializable; @@ -228,6 +229,11 @@ public InternalRow getRow(int pos, int numFields) { return (InternalRow) getObject(pos); } + @Override + public Variant getVariant(int pos) { + return (Variant) getObject(pos); + } + private Object getObject(int pos) { return ((Object[]) array)[pos]; } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/GenericRow.java b/fluss-common/src/main/java/org/apache/fluss/row/GenericRow.java index d67eba1892..d0812a0023 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/GenericRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/GenericRow.java @@ -19,6 +19,7 @@ import org.apache.fluss.annotation.PublicEvolving; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.StringUtils; import java.io.Serializable; @@ -181,6 +182,11 @@ public InternalRow getRow(int pos, int numFields) { return (InternalRow) this.fields[pos]; } + @Override + public Variant getVariant(int pos) { + return (Variant) this.fields[pos]; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/InternalArray.java b/fluss-common/src/main/java/org/apache/fluss/row/InternalArray.java index 6575de1b39..907e7d885c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/InternalArray.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/InternalArray.java @@ -143,6 +143,9 @@ static ElementGetter createElementGetter(DataType fieldType) { final int rowFieldCount = ((RowType) fieldType).getFieldCount(); elementGetter = (array, pos) -> array.getRow(pos, rowFieldCount); break; + case VARIANT: + elementGetter = InternalArray::getVariant; + break; default: String msg = String.format( diff --git a/fluss-common/src/main/java/org/apache/fluss/row/InternalRow.java b/fluss-common/src/main/java/org/apache/fluss/row/InternalRow.java index e19f42b2e0..cd96e32c01 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/InternalRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/InternalRow.java @@ -25,6 +25,7 @@ import org.apache.fluss.types.DataType; import org.apache.fluss.types.MapType; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.Variant; import javax.annotation.Nullable; @@ -135,6 +136,8 @@ static Class getDataClass(DataType type) { return InternalMap.class; case ROW: return InternalRow.class; + case VARIANT: + return Variant.class; default: throw new IllegalArgumentException("Illegal type: " + type); } @@ -224,6 +227,9 @@ static FieldGetter createFieldGetter(DataType fieldType, int fieldPos) { final int numFields = ((RowType) fieldType).getFieldCount(); fieldGetter = row -> row.getRow(fieldPos, numFields); break; + case VARIANT: + fieldGetter = row -> row.getVariant(fieldPos); + break; default: throw new IllegalArgumentException("Illegal type: " + fieldType); } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/PaddingRow.java b/fluss-common/src/main/java/org/apache/fluss/row/PaddingRow.java index df202ec7e2..48df4277f5 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/PaddingRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/PaddingRow.java @@ -17,6 +17,8 @@ package org.apache.fluss.row; +import org.apache.fluss.types.variant.Variant; + /** * An {@link InternalRow} that pads another {@link InternalRow} with nulls up to a target field * count. @@ -140,4 +142,9 @@ public InternalMap getMap(int pos) { public InternalRow getRow(int pos, int numFields) { return row.getRow(pos, numFields); } + + @Override + public Variant getVariant(int pos) { + return row.getVariant(pos); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/ProjectedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/ProjectedRow.java index 25656204d2..13c1daa206 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/ProjectedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/ProjectedRow.java @@ -19,6 +19,7 @@ import org.apache.fluss.annotation.PublicEvolving; import org.apache.fluss.metadata.Schema; +import org.apache.fluss.types.variant.Variant; import java.util.Arrays; @@ -155,6 +156,11 @@ public InternalRow getRow(int pos, int numFields) { return row.getRow(indexMapping[pos], numFields); } + @Override + public Variant getVariant(int pos) { + return row.getVariant(indexMapping[pos]); + } + @Override public boolean equals(Object o) { // only can compare when the index mapping and row are equal. diff --git a/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java index f8a399a20f..c2f322a0f4 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java @@ -38,6 +38,7 @@ import org.apache.fluss.types.LocalZonedTimestampType; import org.apache.fluss.types.RowType; import org.apache.fluss.types.TimestampType; +import org.apache.fluss.types.variant.Variant; import javax.annotation.Nullable; @@ -414,6 +415,15 @@ public InternalRow getRow(int pos, int numFields) { return BinarySegmentUtils.readAlignedRow(segments, offset, offsetAndSize, numFields); } + @Override + public Variant getVariant(int pos) { + assertIndexIsValid(pos); + int fieldOffset = getFieldOffset(pos); + final long offsetAndLen = segments[0].getLong(fieldOffset); + byte[] bytes = BinarySegmentUtils.readBinary(segments, offset, fieldOffset, offsetAndLen); + return Variant.fromBytes(bytes); + } + /** The bit is 1 when the field is null. Default is 0. */ public boolean anyNull() { // Skip the header. @@ -560,6 +570,9 @@ public static AlignedRow from(RowType rowType, @Nullable InternalRow row) { row.getTimestampLtz(i, localZonedTimestampType.getPrecision()), localZonedTimestampType.getPrecision()); break; + case VARIANT: + writer.writeBytes(i, row.getVariant(i).toBytes()); + break; default: throw new UnsupportedOperationException( "Statistics collection is not supported for type: " diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowReader.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowReader.java index 36a37b44d5..3cebd094fc 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowReader.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowReader.java @@ -22,6 +22,9 @@ import org.apache.fluss.row.columnar.ColumnVector; import org.apache.fluss.row.columnar.ColumnarRow; import org.apache.fluss.row.columnar.VectorizedColumnBatch; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VectorSchemaRoot; + +import javax.annotation.Nullable; import static org.apache.fluss.utils.Preconditions.checkNotNull; @@ -36,9 +39,21 @@ public class ArrowReader { private final int rowCount; + /** + * Optional VectorSchemaRoot owned by this reader (created for shredded batches). Will be closed + * when {@link #close()} is called. + */ + @Nullable private final VectorSchemaRoot ownedRoot; + public ArrowReader(ColumnVector[] columnVectors, int rowCount) { + this(columnVectors, rowCount, null); + } + + public ArrowReader( + ColumnVector[] columnVectors, int rowCount, @Nullable VectorSchemaRoot ownedRoot) { this.columnVectors = checkNotNull(columnVectors); this.rowCount = rowCount; + this.ownedRoot = ownedRoot; } public int getRowCount() { @@ -49,4 +64,11 @@ public int getRowCount() { public ColumnarRow read(int rowId) { return new ColumnarRow(new VectorizedColumnBatch(columnVectors), rowId); } + + /** Releases any owned resources (e.g., VectorSchemaRoot created for shredded batches). */ + public void close() { + if (ownedRoot != null) { + ownedRoot.close(); + } + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriter.java index a07329a0b1..b40a58cfa4 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriter.java @@ -31,17 +31,26 @@ import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VectorSchemaRoot; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VectorUnloader; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.ListVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.compression.CompressionCodec; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.compression.CompressionUtil; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.ipc.WriteChannel; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.ipc.message.ArrowBlock; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.ipc.message.ArrowRecordBatch; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.ipc.message.MessageSerializer; +import org.apache.fluss.types.DataType; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.VariantType; +import org.apache.fluss.types.variant.ShreddingSchema; import org.apache.fluss.utils.ArrowUtils; import org.apache.fluss.utils.PagedMemorySegmentWritableChannel; +import javax.annotation.Nullable; + +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.nio.channels.Channels; +import java.util.Map; import static org.apache.fluss.utils.Preconditions.checkNotNull; import static org.apache.fluss.utils.Preconditions.checkState; @@ -90,10 +99,17 @@ public class ArrowWriter implements AutoCloseable { /** * The metadata length of each serialized {@link ArrowRecordBatch} generated by this root. The - * metadata length should be consistent if the arrow schema is not changed. + * metadata length should be consistent if the arrow schema is not changed. When shredding is + * active, this includes the Schema message size. */ private final int metadataLength; + /** Whether the writer uses variant shredding (typed_value children in the Arrow schema). */ + private final boolean hasShredding; + + /** The serialized size of the Arrow Schema IPC message (0 when no shredding). */ + private final int schemaMessageSize; + private final CompressionCodec compressionCodec; private final ArrowCompressionRatioEstimator compressionRatioEstimator; @@ -116,26 +132,87 @@ public class ArrowWriter implements AutoCloseable { ArrowWriterProvider provider, ArrowCompressionInfo compressionInfo, ArrowCompressionRatioEstimator compressionRatioEstimator) { + this( + writerKey, + bufferSizeInBytes, + schema, + allocator, + provider, + compressionInfo, + compressionRatioEstimator, + null); + } + + ArrowWriter( + String writerKey, + int bufferSizeInBytes, + RowType schema, + BufferAllocator allocator, + ArrowWriterProvider provider, + ArrowCompressionInfo compressionInfo, + ArrowCompressionRatioEstimator compressionRatioEstimator, + @Nullable Map shreddingSchemas) { this.writerKey = writerKey; this.schema = schema; - this.root = VectorSchemaRoot.create(ArrowUtils.toArrowSchema(schema), allocator); + this.root = + shreddingSchemas != null && !shreddingSchemas.isEmpty() + ? VectorSchemaRoot.create( + ArrowUtils.toArrowSchemaWithShredding(schema, shreddingSchemas), + allocator) + : VectorSchemaRoot.create(ArrowUtils.toArrowSchema(schema), allocator); this.provider = checkNotNull(provider); this.compressionCodec = compressionInfo.createCompressionCodec(); this.compressionRatioEstimator = compressionRatioEstimator; this.estimatedCompressionRatio = compressionRatioEstimator.estimation(); + this.hasShredding = shreddingSchemas != null && !shreddingSchemas.isEmpty(); + if (hasShredding) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + try { + MessageSerializer.serialize( + new WriteChannel(Channels.newChannel(baos)), root.getSchema()); + } catch (IOException e) { + throw new RuntimeException("Failed to compute schema message size", e); + } + this.schemaMessageSize = baos.size(); + } else { + this.schemaMessageSize = 0; + } + this.metadataLength = - ArrowUtils.estimateArrowMetadataLength( - root.getSchema(), CompressionUtil.createBodyCompression(compressionCodec)); + schemaMessageSize + + ArrowUtils.estimateArrowMetadataLength( + root.getSchema(), + CompressionUtil.createBodyCompression(compressionCodec)); this.writeLimitInBytes = (int) (bufferSizeInBytes * BUFFER_USAGE_RATIO); this.estimatedMaxRecordsCount = -1; this.recordsCount = 0; this.epoch = 0; - this.fieldWriters = new ArrowFieldWriter[schema.getFieldCount()]; - for (int i = 0; i < fieldWriters.length; i++) { - FieldVector fieldVector = root.getVector(i); + + // Initialize ALL vectors first (including shredded columns) + for (FieldVector fieldVector : root.getFieldVectors()) { initFieldVector(fieldVector); - fieldWriters[i] = ArrowUtils.createArrowFieldWriter(fieldVector, schema.getTypeAt(i)); + } + + // Determine user-visible field count (no more $ prefix columns to skip) + int userFieldCount = schema.getFieldCount(); + + this.fieldWriters = new ArrowFieldWriter[userFieldCount]; + for (int i = 0; i < schema.getFieldCount(); i++) { + String fieldName = schema.getFields().get(i).getName(); + FieldVector fieldVector = root.getVector(i); + DataType dataType = schema.getTypeAt(i); + + if (dataType instanceof VariantType + && shreddingSchemas != null + && shreddingSchemas.containsKey(fieldName)) { + // Variant column with shredding - StructVector contains typed_value + fieldWriters[i] = + ArrowUtils.createArrowShreddedVariantWriter( + (StructVector) fieldVector, shreddingSchemas.get(fieldName)); + } else { + fieldWriters[i] = ArrowUtils.createArrowFieldWriter(fieldVector, dataType); + } } } @@ -187,8 +264,8 @@ public void reset(int bufferSizeInBytes) { estimatedMaxRecordsCount = -1; } writeLimitInBytes = newWriteLimit; - for (int i = 0; i < fieldWriters.length; i++) { - FieldVector fieldVector = root.getVector(i); + // Initialize ALL vectors including shredded ones + for (FieldVector fieldVector : root.getFieldVectors()) { initFieldVector(fieldVector); } // Reset field writers to clear their offset counters (for ArrayWriter) @@ -274,7 +351,18 @@ public int serializeToOutputView(AbstractPagedOutputView outputView) throws IOEx new VectorUnloader(root, true, compressionCodec, true).getRecordBatch()) { PagedMemorySegmentWritableChannel channel = new PagedMemorySegmentWritableChannel(outputView); - ArrowBlock block = MessageSerializer.serialize(new WriteChannel(channel), arrowBatch); + WriteChannel writeChannel = new WriteChannel(channel); + + int totalSize = 0; + + // Serialize Arrow Schema message when shredding is active. + // This allows readers to discover the actual schema structure + // (including typed_value children) from the IPC stream. + if (hasShredding) { + totalSize += (int) MessageSerializer.serialize(writeChannel, root.getSchema()); + } + + ArrowBlock block = MessageSerializer.serialize(writeChannel, arrowBatch); checkState( uncompressedBodySizeInBytes > 0, @@ -283,7 +371,8 @@ public int serializeToOutputView(AbstractPagedOutputView outputView) throws IOEx (float) block.getBodyLength() / uncompressedBodySizeInBytes; compressionRatioEstimator.updateEstimation(actualCompressionRatio); - return (int) (block.getMetadataLength() + block.getBodyLength()); + totalSize += (int) (block.getMetadataLength() + block.getBodyLength()); + return totalSize; } } @@ -335,6 +424,12 @@ private void initFieldVector(FieldVector fieldVector) { if (dataVector != null) { initFieldVector(dataVector); } + } else if (fieldVector instanceof StructVector) { + StructVector structVector = (StructVector) fieldVector; + structVector.allocateNew(); + for (FieldVector child : structVector.getChildrenFromFields()) { + initFieldVector(child); + } } else { fieldVector.allocateNew(); } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriterPool.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriterPool.java index 6a384849c5..a077307cc0 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriterPool.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriterPool.java @@ -25,7 +25,9 @@ import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VectorSchemaRoot; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.ShreddingSchema; +import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; @@ -89,7 +91,25 @@ public ArrowWriter getOrCreateWriter( int bufferSizeInBytes, RowType schema, ArrowCompressionInfo compressionInfo) { - final String writerKey = tableId + "-" + schemaId + "-" + compressionInfo.toString(); + return getOrCreateWriter( + tableId, schemaId, bufferSizeInBytes, schema, compressionInfo, null); + } + + @Override + public ArrowWriter getOrCreateWriter( + long tableId, + int schemaId, + int bufferSizeInBytes, + RowType schema, + ArrowCompressionInfo compressionInfo, + @Nullable Map shreddingSchemas) { + final String writerKey = + tableId + + "-" + + schemaId + + "-" + + compressionInfo.toString() + + (shreddingSchemas != null ? "-" + shreddingSchemas.hashCode() : ""); return inLock( lock, () -> { @@ -112,7 +132,8 @@ public ArrowWriter getOrCreateWriter( allocator, this, compressionInfo, - compressionRatioEstimator), + compressionRatioEstimator, + shreddingSchemas), bufferSizeInBytes); } }); diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriterProvider.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriterProvider.java index 58ae7d72de..13a4f03516 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriterProvider.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriterProvider.java @@ -20,6 +20,11 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.compression.ArrowCompressionInfo; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.ShreddingSchema; + +import javax.annotation.Nullable; + +import java.util.Map; /** The provider used for requesting and releasing {@link ArrowWriter}. */ @Internal @@ -31,5 +36,21 @@ ArrowWriter getOrCreateWriter( RowType schema, ArrowCompressionInfo compressionInfo); + /** + * Gets or creates an {@link ArrowWriter} with shredding support for Variant columns. + * + * @param shreddingSchemas map from variant column name to its shredding schema, or null if no + * shredding + */ + default ArrowWriter getOrCreateWriter( + long tableId, + int schemaId, + int bufferSizeInBytes, + RowType schema, + ArrowCompressionInfo compressionInfo, + @Nullable Map shreddingSchemas) { + return getOrCreateWriter(tableId, schemaId, bufferSizeInBytes, schema, compressionInfo); + } + void recycleWriter(ArrowWriter arrowWriter); } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ArrowVariantColumnVector.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ArrowVariantColumnVector.java new file mode 100644 index 0000000000..5e33692524 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ArrowVariantColumnVector.java @@ -0,0 +1,58 @@ +/* + * 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.fluss.row.arrow.vectors; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.row.columnar.VariantColumnVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VarBinaryVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; +import org.apache.fluss.types.variant.Variant; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Arrow column vector for Variant type. Reads {@link Variant} values from a {@link StructVector} + * with two children: metadata (VarBinaryVector) and value (VarBinaryVector). + */ +@Internal +public class ArrowVariantColumnVector implements VariantColumnVector { + + /** The StructVector that contains metadata and value sub-vectors. */ + private final StructVector structVector; + + private final VarBinaryVector metadataVector; + private final VarBinaryVector valueVector; + + public ArrowVariantColumnVector(StructVector structVector) { + this.structVector = checkNotNull(structVector); + this.metadataVector = (VarBinaryVector) structVector.getChild("metadata"); + this.valueVector = (VarBinaryVector) structVector.getChild("value"); + } + + @Override + public Variant getVariant(int i) { + byte[] metadata = metadataVector.get(i); + byte[] value = valueVector.get(i); + return new Variant(metadata, value); + } + + @Override + public boolean isNullAt(int i) { + return structVector.isNull(i); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ShreddedVariantColumnVector.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ShreddedVariantColumnVector.java new file mode 100644 index 0000000000..cb03edc7ec --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ShreddedVariantColumnVector.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.fluss.row.arrow.vectors; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.row.columnar.VariantColumnVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.BigIntVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.BitVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.DateDayVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.Float4Vector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.Float8Vector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.IntVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.SmallIntVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.TinyIntVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.ValueVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VarBinaryVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VarCharVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeRoot; +import org.apache.fluss.types.variant.ShreddedField; +import org.apache.fluss.types.variant.ShreddedVariant; +import org.apache.fluss.types.variant.ShreddingSchema; +import org.apache.fluss.types.variant.Variant; +import org.apache.fluss.types.variant.VariantUtil; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Arrow column vector for shredded Variant columns. Reads from a StructVector with the layout: + * + *

+ * StructVector (Variant column)
+ *   ├── VarBinaryVector "metadata"
+ *   ├── VarBinaryVector "value"           (residual)
+ *   └── StructVector "typed_value"
+ *         ├── StructVector "fieldName"
+ *         │     ├── VarBinaryVector "value"       (per-field fallback binary)
+ *         │     └── <TypedVector> "typed_value"   (typed value)
+ *         └── ...
+ * 
+ * + *

For each shredded field: + * + *

    + *
  • typed_value non-null → use typed value directly + *
  • typed_value null, per-field value non-null → decode from binary using metadata + *
  • both null → field not present + *
+ * + *

The complete Variant is reconstructed by merging typed_value fields with the residual. + */ +@Internal +public class ShreddedVariantColumnVector implements VariantColumnVector { + + private final StructVector variantStructVector; + private final VarBinaryVector metadataVector; + private final VarBinaryVector valueVector; + private final StructVector typedValueVector; + + // Per-field vectors + private final StructVector[] fieldStructVectors; + private final VarBinaryVector[] fieldValueVectors; + private final ValueVector[] fieldTypedValueVectors; + + private final ShreddingSchema shreddingSchema; + private final DataType[] shreddedTypes; + private final String[] fieldPaths; + + public ShreddedVariantColumnVector( + StructVector variantStructVector, ShreddingSchema shreddingSchema) { + this.variantStructVector = variantStructVector; + this.shreddingSchema = shreddingSchema; + + this.metadataVector = (VarBinaryVector) variantStructVector.getChild("metadata"); + this.valueVector = (VarBinaryVector) variantStructVector.getChild("value"); + this.typedValueVector = (StructVector) variantStructVector.getChild("typed_value"); + + List fields = shreddingSchema.getFields(); + int numFields = fields.size(); + this.fieldStructVectors = new StructVector[numFields]; + this.fieldValueVectors = new VarBinaryVector[numFields]; + this.fieldTypedValueVectors = new ValueVector[numFields]; + this.shreddedTypes = new DataType[numFields]; + this.fieldPaths = new String[numFields]; + + for (int i = 0; i < numFields; i++) { + ShreddedField sf = fields.get(i); + fieldPaths[i] = sf.getFieldPath(); + shreddedTypes[i] = sf.getShreddedType(); + fieldStructVectors[i] = (StructVector) typedValueVector.getChild(sf.getFieldPath()); + fieldValueVectors[i] = (VarBinaryVector) fieldStructVectors[i].getChild("value"); + fieldTypedValueVectors[i] = (ValueVector) fieldStructVectors[i].getChild("typed_value"); + } + } + + @Override + public Variant getVariant(int i) { + byte[] metadata = metadataVector.isNull(i) ? null : metadataVector.get(i); + + // Collect shredded field values + // Field structs are notNullable (required); field presence is determined by + // checking children: both value and typed_value null = "field not present". + boolean hasShreddedValues = false; + for (int si = 0; si < fieldPaths.length; si++) { + if (!fieldValueVectors[si].isNull(i) || !fieldTypedValueVectors[si].isNull(i)) { + hasShreddedValues = true; + break; + } + } + + boolean hasResidual = !valueVector.isNull(i); + + if (!hasResidual && !hasShreddedValues) { + // No data at all + if (metadata == null) { + return Variant.ofNull(); + } + // Metadata exists but no value - shouldn't happen, but handle gracefully + return Variant.ofNull(); + } + + if (!hasShreddedValues) { + // Only residual - return as Variant(metadata, value) + return new Variant(metadata, valueVector.get(i)); + } + + // Build ShreddedVariant: extract per-field typed objects and fallback values + byte[] residualValue = hasResidual ? valueVector.get(i) : null; + + Object[] typedValues = new Object[fieldPaths.length]; + byte[][] fallbackValues = new byte[fieldPaths.length][]; + + for (int si = 0; si < fieldPaths.length; si++) { + if (!fieldTypedValueVectors[si].isNull(i)) { + typedValues[si] = extractTypedObject(si, i); + } else if (!fieldValueVectors[si].isNull(i)) { + fallbackValues[si] = fieldValueVectors[si].get(i); + } + } + + return new ShreddedVariant( + metadata, residualValue, fieldPaths, shreddedTypes, typedValues, fallbackValues); + } + + @Override + public boolean isNullAt(int i) { + return variantStructVector.isNull(i); + } + + /** + * Extracts a typed value from an Arrow vector as a Java object. This avoids encoding to Variant + * binary, enabling downstream consumers (e.g., Flink's FlussVariant) to directly access the + * typed value without decoding overhead. + * + * @return the typed Java object: Boolean, Byte, Short, Integer, Long, Float, Double, String, + * etc. + */ + private Object extractTypedObject(int fieldIndex, int rowIndex) { + ValueVector vector = fieldTypedValueVectors[fieldIndex]; + DataTypeRoot root = shreddedTypes[fieldIndex].getTypeRoot(); + switch (root) { + case BOOLEAN: + return ((BitVector) vector).get(rowIndex) != 0; + case TINYINT: + return ((TinyIntVector) vector).get(rowIndex); + case SMALLINT: + return ((SmallIntVector) vector).get(rowIndex); + case INTEGER: + return ((IntVector) vector).get(rowIndex); + case BIGINT: + return ((BigIntVector) vector).get(rowIndex); + case FLOAT: + return ((Float4Vector) vector).get(rowIndex); + case DOUBLE: + return ((Float8Vector) vector).get(rowIndex); + case STRING: + case CHAR: + { + byte[] utf8 = ((VarCharVector) vector).get(rowIndex); + return new String(utf8, StandardCharsets.UTF_8); + } + case DATE: + return ((DateDayVector) vector).get(rowIndex); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + case TIMESTAMP_WITHOUT_TIME_ZONE: + return ((TimeStampMicroVector) vector).get(rowIndex); + default: + throw new UnsupportedOperationException( + "Unsupported shredded type for extracting: " + shreddedTypes[fieldIndex]); + } + } + + /** + * Builds a Variant metadata dictionary from a list of field names. The names are sorted for + * binary search support. + */ + static byte[] buildMetadata(List fieldNames) { + List sorted = new ArrayList<>(fieldNames); + Collections.sort(sorted); + + byte[][] nameBytes = new byte[sorted.size()][]; + int totalStringBytes = 0; + for (int i = 0; i < sorted.size(); i++) { + nameBytes[i] = sorted.get(i).getBytes(StandardCharsets.UTF_8); + totalStringBytes += nameBytes[i].length; + } + + int dictSize = sorted.size(); + int totalSize = 1 + 4 + (dictSize + 1) * 4 + totalStringBytes; + byte[] metadata = new byte[totalSize]; + + metadata[0] = + (byte) + (VariantUtil.METADATA_VERSION + | VariantUtil.METADATA_SORTED_STRINGS_BIT + | VariantUtil.METADATA_OFFSET_SIZE_BITS); + + VariantUtil.writeIntLE(metadata, 1, dictSize); + + int offsetPos = 5; + int strOffset = 0; + int strBytesStart = 5 + (dictSize + 1) * 4; + for (int i = 0; i < dictSize; i++) { + VariantUtil.writeIntLE(metadata, offsetPos, strOffset); + offsetPos += 4; + System.arraycopy( + nameBytes[i], 0, metadata, strBytesStart + strOffset, nameBytes[i].length); + strOffset += nameBytes[i].length; + } + VariantUtil.writeIntLE(metadata, offsetPos, strOffset); + + return metadata; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowShreddedVariantWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowShreddedVariantWriter.java new file mode 100644 index 0000000000..4235291477 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowShreddedVariantWriter.java @@ -0,0 +1,507 @@ +/* + * 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.fluss.row.arrow.writers; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.row.DataGetters; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.BigIntVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.BitVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.DateDayVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.FieldVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.Float4Vector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.Float8Vector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.IntVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.SmallIntVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.TinyIntVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VarBinaryVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VarCharVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeRoot; +import org.apache.fluss.types.variant.ShreddedField; +import org.apache.fluss.types.variant.ShreddingSchema; +import org.apache.fluss.types.variant.Variant; +import org.apache.fluss.types.variant.VariantUtil; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * {@link ArrowFieldWriter} for shredded Variant columns. Writes a Variant value into a StructVector + * with the following layout: + * + *

+ * StructVector (Variant column)
+ *   ├── VarBinaryVector "metadata"       (Variant metadata dictionary)
+ *   ├── VarBinaryVector "value"          (residual Variant value binary)
+ *   └── StructVector "typed_value"
+ *         ├── StructVector "fieldName1"
+ *         │     ├── VarBinaryVector "value"       (per-field fallback binary)
+ *         │     └── <TypedVector> "typed_value"   (typed value when type matches)
+ *         └── StructVector "fieldName2"
+ *               ├── VarBinaryVector "value"
+ *               └── <TypedVector> "typed_value"
+ * 
+ * + *

For each shredded field, if the Variant value type matches the target type, the value is + * written to the typed_value vector; if there's a type mismatch, the raw binary is written to the + * per-field value vector as a fallback; if the field is absent, both are null. + * + *

The residual (metadata + value) contains the remaining non-shredded fields. + */ +@Internal +public class ArrowShreddedVariantWriter extends ArrowFieldWriter { + + private final VarBinaryVector metadataVector; + private final VarBinaryVector valueVector; + private final StructVector typedValueVector; + + // Per-field vectors extracted from typed_value children + private final StructVector[] fieldStructVectors; + private final VarBinaryVector[] fieldValueVectors; + private final FieldVector[] fieldTypedValueVectors; + + private final ShreddingSchema shreddingSchema; + private final DataType[] shreddedTypes; + private final String[] fieldPaths; + + public ArrowShreddedVariantWriter( + StructVector variantStructVector, ShreddingSchema shreddingSchema) { + super(variantStructVector); + this.shreddingSchema = shreddingSchema; + + // Extract top-level children + this.metadataVector = (VarBinaryVector) variantStructVector.getChild("metadata"); + this.valueVector = (VarBinaryVector) variantStructVector.getChild("value"); + this.typedValueVector = (StructVector) variantStructVector.getChild("typed_value"); + + // Extract per-field vectors from typed_value + List fields = shreddingSchema.getFields(); + int numFields = fields.size(); + this.fieldStructVectors = new StructVector[numFields]; + this.fieldValueVectors = new VarBinaryVector[numFields]; + this.fieldTypedValueVectors = new FieldVector[numFields]; + this.shreddedTypes = new DataType[numFields]; + this.fieldPaths = new String[numFields]; + + for (int i = 0; i < numFields; i++) { + ShreddedField sf = fields.get(i); + fieldPaths[i] = sf.getFieldPath(); + shreddedTypes[i] = sf.getShreddedType(); + fieldStructVectors[i] = (StructVector) typedValueVector.getChild(sf.getFieldPath()); + fieldValueVectors[i] = (VarBinaryVector) fieldStructVectors[i].getChild("value"); + fieldTypedValueVectors[i] = (FieldVector) fieldStructVectors[i].getChild("typed_value"); + } + } + + @Override + public void write(int rowIndex, DataGetters getters, int ordinal, boolean handleSafe) { + StructVector variantStruct = (StructVector) fieldVector; + if (getters.isNullAt(ordinal)) { + // Set null for the entire Variant struct + variantStruct.setNull(rowIndex); + metadataVector.setNull(rowIndex); + valueVector.setNull(rowIndex); + typedValueVector.setNull(rowIndex); + for (int i = 0; i < fieldStructVectors.length; i++) { + fieldStructVectors[i].setNull(rowIndex); + fieldValueVectors[i].setNull(rowIndex); + fieldTypedValueVectors[i].setNull(rowIndex); + } + } else { + doWrite(rowIndex, getters, ordinal, handleSafe); + } + } + + @Override + public void doWrite(int rowIndex, DataGetters row, int ordinal, boolean handleSafe) { + StructVector variantStruct = (StructVector) fieldVector; + Variant variant = row.getVariant(ordinal); + + // Always mark the top-level Variant struct as defined + variantStruct.setIndexDefined(rowIndex); + + if (!variant.isObject()) { + // Not an object - write entire variant as residual, typed_value all "not present" + metadataVector.setSafe(rowIndex, variant.metadata()); + valueVector.setSafe(rowIndex, variant.value()); + typedValueVector.setIndexDefined(rowIndex); + for (int i = 0; i < fieldStructVectors.length; i++) { + fieldStructVectors[i].setIndexDefined(rowIndex); + fieldValueVectors[i].setNull(rowIndex); + fieldTypedValueVectors[i].setNull(rowIndex); + } + return; + } + + byte[] metadata = variant.metadata(); + byte[] value = variant.value(); + int numObjFields = VariantUtil.objectSize(value, 0); + + // Track which original fields are shredded successfully + boolean[] fieldShredded = new boolean[numObjFields]; + + // Mark typed_value struct as defined + typedValueVector.setIndexDefined(rowIndex); + + // For each shredded field, try to extract and write + for (int si = 0; si < fieldPaths.length; si++) { + String fieldPath = fieldPaths[si]; + int fieldId = VariantUtil.findFieldId(metadata, fieldPath); + + // Field struct is notNullable (required), always mark as defined + fieldStructVectors[si].setIndexDefined(rowIndex); + + if (fieldId < 0) { + // Field not present: both children null = "field absent" per semantic matrix + fieldValueVectors[si].setNull(rowIndex); + fieldTypedValueVectors[si].setNull(rowIndex); + continue; + } + + int valueOffset = VariantUtil.findFieldValueOffset(value, 0, fieldId); + if (valueOffset < 0) { + fieldValueVectors[si].setNull(rowIndex); + fieldTypedValueVectors[si].setNull(rowIndex); + continue; + } + + // Check for Variant null values - kept in residual only. + // Both children null = "field not present" in the shredding layer, + // while the actual null value is preserved in residual. + int basicType = VariantUtil.basicType(value, valueOffset); + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE + && VariantUtil.primitiveTypeId(value, valueOffset) + == VariantUtil.PRIMITIVE_TYPE_NULL) { + fieldValueVectors[si].setNull(rowIndex); + fieldTypedValueVectors[si].setNull(rowIndex); + continue; + } + + // Try to write the value to the typed_value column + boolean written = + writeTypedValue( + fieldTypedValueVectors[si], + rowIndex, + value, + valueOffset, + shreddedTypes[si]); + + if (written) { + // Type matched: typed_value is set, per-field value is null + fieldValueVectors[si].setNull(rowIndex); + markFieldShredded(value, fieldId, fieldShredded); + } else { + // Type mismatch: write raw binary to per-field value, typed_value is null + int fieldValueSize = VariantUtil.valueSize(value, valueOffset); + byte[] rawBytes = + Arrays.copyOfRange(value, valueOffset, valueOffset + fieldValueSize); + fieldValueVectors[si].setSafe(rowIndex, rawBytes); + fieldTypedValueVectors[si].setNull(rowIndex); + markFieldShredded(value, fieldId, fieldShredded); + } + } + + // Build residual variant (original fields minus shredded ones) + buildAndWriteResidual(rowIndex, metadata, value, numObjFields, fieldShredded); + } + + private void markFieldShredded(byte[] value, int fieldId, boolean[] fieldShredded) { + int numFields = VariantUtil.objectSize(value, 0); + for (int i = 0; i < numFields; i++) { + if (VariantUtil.objectFieldId(value, 0, i) == fieldId) { + fieldShredded[i] = true; + return; + } + } + } + + /** + * Attempts to write a variant field value to a typed Arrow vector. + * + * @return true if the value was successfully written (type compatible), false otherwise + */ + private boolean writeTypedValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, DataType targetType) { + int basicType = VariantUtil.basicType(value, valueOffset); + + DataTypeRoot root = targetType.getTypeRoot(); + try { + switch (root) { + case BOOLEAN: + return writeBooleanValue(vector, rowIndex, value, valueOffset, basicType); + case TINYINT: + return writeTinyIntValue(vector, rowIndex, value, valueOffset, basicType); + case SMALLINT: + return writeSmallIntValue(vector, rowIndex, value, valueOffset, basicType); + case INTEGER: + return writeIntValue(vector, rowIndex, value, valueOffset, basicType); + case BIGINT: + return writeBigIntValue(vector, rowIndex, value, valueOffset, basicType); + case FLOAT: + return writeFloatValue(vector, rowIndex, value, valueOffset, basicType); + case DOUBLE: + return writeDoubleValue(vector, rowIndex, value, valueOffset, basicType); + case STRING: + case CHAR: + return writeStringValue(vector, rowIndex, value, valueOffset, basicType); + case DATE: + return writeDateValue(vector, rowIndex, value, valueOffset, basicType); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return writeTimestampValue( + vector, + rowIndex, + value, + valueOffset, + basicType, + VariantUtil.PRIMITIVE_TYPE_TIMESTAMP); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return writeTimestampValue( + vector, + rowIndex, + value, + valueOffset, + basicType, + VariantUtil.PRIMITIVE_TYPE_TIMESTAMP_NTZ); + default: + return false; + } + } catch (Exception e) { + return false; + } + } + + private boolean writeBooleanValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE) { + int tid = VariantUtil.primitiveTypeId(value, valueOffset); + if (tid == VariantUtil.PRIMITIVE_TYPE_TRUE) { + ((BitVector) vector).setSafe(rowIndex, 1); + return true; + } else if (tid == VariantUtil.PRIMITIVE_TYPE_FALSE) { + ((BitVector) vector).setSafe(rowIndex, 0); + return true; + } + } + return false; + } + + private boolean writeTinyIntValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE + && VariantUtil.primitiveTypeId(value, valueOffset) + == VariantUtil.PRIMITIVE_TYPE_INT8) { + ((TinyIntVector) vector).setSafe(rowIndex, VariantUtil.getByte(value, valueOffset)); + return true; + } + return false; + } + + private boolean writeSmallIntValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE) { + int tid = VariantUtil.primitiveTypeId(value, valueOffset); + if (tid == VariantUtil.PRIMITIVE_TYPE_INT16) { + ((SmallIntVector) vector) + .setSafe(rowIndex, VariantUtil.getShort(value, valueOffset)); + return true; + } else if (tid == VariantUtil.PRIMITIVE_TYPE_INT8) { + ((SmallIntVector) vector) + .setSafe(rowIndex, VariantUtil.getByte(value, valueOffset)); + return true; + } + } + return false; + } + + private boolean writeIntValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE) { + int tid = VariantUtil.primitiveTypeId(value, valueOffset); + if (tid == VariantUtil.PRIMITIVE_TYPE_INT32) { + ((IntVector) vector).setSafe(rowIndex, VariantUtil.getInt(value, valueOffset)); + return true; + } else if (tid == VariantUtil.PRIMITIVE_TYPE_INT16) { + ((IntVector) vector).setSafe(rowIndex, VariantUtil.getShort(value, valueOffset)); + return true; + } else if (tid == VariantUtil.PRIMITIVE_TYPE_INT8) { + ((IntVector) vector).setSafe(rowIndex, VariantUtil.getByte(value, valueOffset)); + return true; + } + } + return false; + } + + private boolean writeBigIntValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE) { + int tid = VariantUtil.primitiveTypeId(value, valueOffset); + if (tid == VariantUtil.PRIMITIVE_TYPE_INT64) { + ((BigIntVector) vector).setSafe(rowIndex, VariantUtil.getLong(value, valueOffset)); + return true; + } else if (tid == VariantUtil.PRIMITIVE_TYPE_INT32) { + ((BigIntVector) vector).setSafe(rowIndex, VariantUtil.getInt(value, valueOffset)); + return true; + } else if (tid == VariantUtil.PRIMITIVE_TYPE_INT16) { + ((BigIntVector) vector).setSafe(rowIndex, VariantUtil.getShort(value, valueOffset)); + return true; + } else if (tid == VariantUtil.PRIMITIVE_TYPE_INT8) { + ((BigIntVector) vector).setSafe(rowIndex, VariantUtil.getByte(value, valueOffset)); + return true; + } + } + return false; + } + + private boolean writeFloatValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE + && VariantUtil.primitiveTypeId(value, valueOffset) + == VariantUtil.PRIMITIVE_TYPE_FLOAT) { + ((Float4Vector) vector).setSafe(rowIndex, VariantUtil.getFloat(value, valueOffset)); + return true; + } + return false; + } + + private boolean writeDoubleValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE + && VariantUtil.primitiveTypeId(value, valueOffset) + == VariantUtil.PRIMITIVE_TYPE_DOUBLE) { + ((Float8Vector) vector).setSafe(rowIndex, VariantUtil.getDouble(value, valueOffset)); + return true; + } + return false; + } + + private boolean writeStringValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_SHORT_STRING + || (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE + && VariantUtil.primitiveTypeId(value, valueOffset) + == VariantUtil.PRIMITIVE_TYPE_STRING)) { + String str = VariantUtil.getString(value, valueOffset); + byte[] utf8 = str.getBytes(StandardCharsets.UTF_8); + ((VarCharVector) vector).setSafe(rowIndex, utf8); + return true; + } + return false; + } + + private boolean writeDateValue( + FieldVector vector, int rowIndex, byte[] value, int valueOffset, int basicType) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE + && VariantUtil.primitiveTypeId(value, valueOffset) + == VariantUtil.PRIMITIVE_TYPE_DATE) { + ((DateDayVector) vector).setSafe(rowIndex, VariantUtil.getDate(value, valueOffset)); + return true; + } + return false; + } + + private boolean writeTimestampValue( + FieldVector vector, + int rowIndex, + byte[] value, + int valueOffset, + int basicType, + int expectedPrimitiveTypeId) { + if (basicType == VariantUtil.BASIC_TYPE_PRIMITIVE + && VariantUtil.primitiveTypeId(value, valueOffset) == expectedPrimitiveTypeId) { + long micros = + expectedPrimitiveTypeId == VariantUtil.PRIMITIVE_TYPE_TIMESTAMP + ? VariantUtil.getTimestamp(value, valueOffset) + : VariantUtil.getTimestampNtz(value, valueOffset); + ((TimeStampMicroVector) vector).setSafe(rowIndex, micros); + return true; + } + return false; + } + + /** + * Builds and writes the residual Variant (original object minus successfully shredded fields). + * The original metadata dictionary is preserved to keep nested object field IDs valid. + */ + private void buildAndWriteResidual( + int rowIndex, byte[] metadata, byte[] value, int numFields, boolean[] fieldShredded) { + // Always write metadata - it's needed for decoding both residual and per-field value + metadataVector.setSafe(rowIndex, metadata); + + // Special-case: empty object {} has numFields == 0 + if (numFields == 0) { + byte[] emptyObjValue = + VariantUtil.encodeObject( + java.util.Collections.emptyList(), java.util.Collections.emptyList()); + valueVector.setSafe(rowIndex, emptyObjValue); + return; + } + + // Check if all fields are shredded + boolean allShredded = true; + for (boolean s : fieldShredded) { + if (!s) { + allShredded = false; + break; + } + } + + if (allShredded) { + // All fields are in typed_value - no residual value needed + valueVector.setNull(rowIndex); + return; + } + + // Collect non-shredded fields + List residualFieldIds = new ArrayList<>(); + List residualFieldValues = new ArrayList<>(); + + for (int i = 0; i < numFields; i++) { + if (!fieldShredded[i]) { + int fieldId = VariantUtil.objectFieldId(value, 0, i); + int fieldValueOffset = VariantUtil.objectFieldValueOffset(value, 0, i); + int fieldValueSize = VariantUtil.valueSize(value, fieldValueOffset); + byte[] fieldValueBytes = + Arrays.copyOfRange( + value, fieldValueOffset, fieldValueOffset + fieldValueSize); + residualFieldIds.add(fieldId); + residualFieldValues.add(fieldValueBytes); + } + } + + // Re-encode residual object with original metadata's field IDs + byte[] residualValue = VariantUtil.encodeObject(residualFieldIds, residualFieldValues); + valueVector.setSafe(rowIndex, residualValue); + } + + @Override + public void reset() { + super.reset(); + metadataVector.reset(); + valueVector.reset(); + typedValueVector.reset(); + for (int i = 0; i < fieldStructVectors.length; i++) { + fieldStructVectors[i].reset(); + fieldValueVectors[i].reset(); + fieldTypedValueVectors[i].reset(); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowVariantWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowVariantWriter.java new file mode 100644 index 0000000000..897f74d93a --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowVariantWriter.java @@ -0,0 +1,50 @@ +/* + * 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.fluss.row.arrow.writers; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.row.DataGetters; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VarBinaryVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; +import org.apache.fluss.types.variant.Variant; + +/** + * {@link ArrowFieldWriter} for Variant type. Serializes {@link Variant} values into a {@link + * StructVector} with two children: metadata (VarBinaryVector) and value (VarBinaryVector). + */ +@Internal +public class ArrowVariantWriter extends ArrowFieldWriter { + + private final VarBinaryVector metadataVector; + private final VarBinaryVector valueVector; + + public ArrowVariantWriter(StructVector structVector) { + super(structVector); + this.metadataVector = (VarBinaryVector) structVector.getChild("metadata"); + this.valueVector = (VarBinaryVector) structVector.getChild("value"); + } + + @Override + public void doWrite(int rowIndex, DataGetters row, int ordinal, boolean handleSafe) { + StructVector vector = (StructVector) fieldVector; + Variant variant = row.getVariant(ordinal); + vector.setIndexDefined(rowIndex); + metadataVector.setSafe(rowIndex, variant.metadata()); + valueVector.setSafe(rowIndex, variant.value()); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarArray.java b/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarArray.java index 943c203080..fb28fc5d35 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarArray.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarArray.java @@ -25,6 +25,7 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.variant.Variant; import java.io.Serializable; import java.util.Arrays; @@ -148,6 +149,11 @@ public InternalRow getRow(int pos, int numFields) { return ((RowColumnVector) data).getRow(offset + pos); } + @Override + public Variant getVariant(int pos) { + return ((VariantColumnVector) data).getVariant(offset + pos); + } + @Override public boolean[] toBooleanArray() { boolean[] res = new boolean[numElements]; diff --git a/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarRow.java b/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarRow.java index 40c935104a..a6c3c2a6ad 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarRow.java @@ -25,6 +25,7 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.variant.Variant; /** * Columnar row to support access to vector column data. It is a row view in {@link @@ -148,6 +149,11 @@ public InternalRow getRow(int pos, int numFields) { return vectorizedColumnBatch.getRow(rowId, pos); } + @Override + public Variant getVariant(int pos) { + return vectorizedColumnBatch.getVariant(rowId, pos); + } + @Override public int getFieldCount() { return vectorizedColumnBatch.getFieldCount(); diff --git a/fluss-common/src/main/java/org/apache/fluss/row/columnar/VariantColumnVector.java b/fluss-common/src/main/java/org/apache/fluss/row/columnar/VariantColumnVector.java new file mode 100644 index 0000000000..b2929f90aa --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/columnar/VariantColumnVector.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.fluss.row.columnar; + +import org.apache.fluss.types.variant.Variant; + +/** A column vector for reading {@link Variant} values. */ +public interface VariantColumnVector extends ColumnVector { + Variant getVariant(int i); +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/columnar/VectorizedColumnBatch.java b/fluss-common/src/main/java/org/apache/fluss/row/columnar/VectorizedColumnBatch.java index 0d4975d6c2..ee36147188 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/columnar/VectorizedColumnBatch.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/columnar/VectorizedColumnBatch.java @@ -24,6 +24,7 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.variant.Variant; import java.nio.charset.StandardCharsets; @@ -117,4 +118,8 @@ public InternalMap getMap(int rowId, int colId) { public InternalRow getRow(int rowId, int colId) { return ((RowColumnVector) columns[colId]).getRow(rowId); } + + public Variant getVariant(int rowId, int colId) { + return ((VariantColumnVector) columns[colId]).getVariant(rowId); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java index b56355622f..836ee5033b 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java @@ -31,6 +31,7 @@ import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.row.indexed.IndexedRow; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.MurmurHashUtils; import static org.apache.fluss.utils.Preconditions.checkArgument; @@ -266,6 +267,11 @@ public InternalRow getRow(int pos, int numFields) { return decodedRow().getRow(pos, numFields); } + @Override + public Variant getVariant(int pos) { + return decodedRow().getVariant(pos); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowReader.java b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowReader.java index e51f2ad609..1a726a7537 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowReader.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowReader.java @@ -337,6 +337,13 @@ static FieldReader createFieldReader(DataType fieldType) { ((RowType) fieldType).getFieldTypes().toArray(new DataType[0]); fieldReader = (reader, pos) -> reader.readRow(nestedFieldTypes); break; + case VARIANT: + fieldReader = + (reader, pos) -> { + byte[] bytes = reader.readBytes(); + return org.apache.fluss.types.variant.Variant.fromBytes(bytes); + }; + break; default: throw new IllegalArgumentException( "Unsupported type for CompatedRow: " + fieldType); diff --git a/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRow.java index 9a7b18f772..c47110a75d 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRow.java @@ -42,6 +42,7 @@ import org.apache.fluss.types.MapType; import org.apache.fluss.types.RowType; import org.apache.fluss.types.StringType; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.MurmurHashUtils; import java.util.Arrays; @@ -438,6 +439,12 @@ public InternalRow getRow(int pos, int numFields) { } } + @Override + public Variant getVariant(int pos) { + byte[] bytes = getBytes(pos); + return Variant.fromBytes(bytes); + } + private void assertIndexIsValid(int index) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < arity : "index (" + index + ") should < " + arity; @@ -516,6 +523,7 @@ public static boolean isFixedLength(DataType dataType) { case ARRAY: case MAP: case ROW: + case VARIANT: return false; case DECIMAL: return Decimal.isCompact(((DecimalType) dataType).getPrecision()); diff --git a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeDefaultVisitor.java b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeDefaultVisitor.java index eb58c5a6c4..c2036e3084 100644 --- a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeDefaultVisitor.java +++ b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeDefaultVisitor.java @@ -122,5 +122,10 @@ public R visit(RowType rowType) { return defaultMethod(rowType); } + @Override + public R visit(VariantType variantType) { + return defaultMethod(variantType); + } + protected abstract R defaultMethod(DataType dataType); } diff --git a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeParser.java b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeParser.java index 5068e02596..62d3b0c56d 100644 --- a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeParser.java +++ b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeParser.java @@ -281,6 +281,7 @@ private enum Keyword { ROW, NOT, NULL, + VARIANT, } private enum UnsupportedKeyword { @@ -290,7 +291,6 @@ private enum UnsupportedKeyword { MULTISET, RAW, LEGACY, - VARIANT, BITMAP } @@ -514,6 +514,8 @@ private DataType parseTypeByKeyword() { return parseMapType(); case ROW: return parseRowType(); + case VARIANT: + return new VariantType(); default: throw parsingError("Unsupported type: " + token().value); } diff --git a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeRoot.java b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeRoot.java index f092e94661..07287f1bc3 100644 --- a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeRoot.java +++ b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeRoot.java @@ -103,7 +103,9 @@ public enum DataTypeRoot { MAP(DataTypeFamily.CONSTRUCTED, DataTypeFamily.EXTENSION), - ROW(DataTypeFamily.CONSTRUCTED); + ROW(DataTypeFamily.CONSTRUCTED), + + VARIANT(DataTypeFamily.PREDEFINED, DataTypeFamily.EXTENSION); private final Set families; diff --git a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeVisitor.java b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeVisitor.java index b3df5a12ba..b13a59e034 100644 --- a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeVisitor.java +++ b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeVisitor.java @@ -66,4 +66,6 @@ public interface DataTypeVisitor { R visit(MapType mapType); R visit(RowType rowType); + + R visit(VariantType variantType); } diff --git a/fluss-common/src/main/java/org/apache/fluss/types/DataTypes.java b/fluss-common/src/main/java/org/apache/fluss/types/DataTypes.java index 758b717e19..7fa9e49ae6 100644 --- a/fluss-common/src/main/java/org/apache/fluss/types/DataTypes.java +++ b/fluss-common/src/main/java/org/apache/fluss/types/DataTypes.java @@ -311,6 +311,16 @@ public static MapType MAP(DataType keyType, DataType valueType) { return new MapType(keyType, valueType); } + /** + * Data type of a semi-structured variant value. A variant can hold any data type including + * primitives, strings, arrays, and objects (similar to JSON). + * + * @see VariantType + */ + public static VariantType VARIANT() { + return new VariantType(); + } + public static DataField FIELD(String name, DataType type) { return new DataField(name, type); } diff --git a/fluss-common/src/main/java/org/apache/fluss/types/VariantType.java b/fluss-common/src/main/java/org/apache/fluss/types/VariantType.java new file mode 100644 index 0000000000..56cb515487 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/VariantType.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.fluss.types; + +import org.apache.fluss.annotation.PublicStable; + +import java.util.Collections; +import java.util.List; + +/** + * Data type of a semi-structured variant value. A variant can hold any data type including + * primitives, strings, arrays, and objects (similar to JSON). Internally, variant values are stored + * using the Parquet Variant Binary Encoding format (metadata dictionary + self-describing value). + * + * @since 0.7 + */ +@PublicStable +public class VariantType extends DataType { + private static final long serialVersionUID = 1L; + + private static final String FORMAT = "VARIANT"; + + public VariantType(boolean isNullable) { + super(isNullable, DataTypeRoot.VARIANT); + } + + public VariantType() { + this(true); + } + + @Override + public DataType copy(boolean isNullable) { + return new VariantType(isNullable); + } + + @Override + public String asSerializableString() { + return withNullability(FORMAT); + } + + @Override + public List getChildren() { + return Collections.emptyList(); + } + + @Override + public R accept(DataTypeVisitor visitor) { + return visitor.visit(this); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/FieldStatistics.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/FieldStatistics.java new file mode 100644 index 0000000000..7da308a1fe --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/FieldStatistics.java @@ -0,0 +1,178 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.types.DataTypeRoot; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** + * Statistics for a single field within Variant records. Tracks how often the field appears and the + * distribution of data types observed for that field. + * + *

Used by {@link ShreddingSchemaInferrer} to determine whether a field should be shredded and + * what type the shredded column should have. + * + * @since 0.7 + */ +@Internal +public final class FieldStatistics implements Serializable { + + private static final long serialVersionUID = 1L; + + /** The field path (e.g., "age", "address.city"). */ + private final String fieldPath; + + /** Number of records where this field is present. */ + private long presenceCount; + + /** Count of each observed data type for this field. */ + private final Map typeCounts; + + public FieldStatistics(String fieldPath) { + this.fieldPath = fieldPath; + this.presenceCount = 0; + this.typeCounts = new HashMap<>(); + } + + /** Creates a copy of this statistics for merging. */ + public FieldStatistics(FieldStatistics other) { + this.fieldPath = other.fieldPath; + this.presenceCount = other.presenceCount; + this.typeCounts = new HashMap<>(other.typeCounts); + } + + /** Returns the field path. */ + public String getFieldPath() { + return fieldPath; + } + + /** Returns the number of records where this field is present. */ + public long getPresenceCount() { + return presenceCount; + } + + /** Returns the type counts map. */ + public Map getTypeCounts() { + return Collections.unmodifiableMap(typeCounts); + } + + /** Records an observation of this field with the given type. */ + public void record(DataTypeRoot typeRoot) { + presenceCount++; + typeCounts.merge(typeRoot, 1L, Long::sum); + } + + /** + * Returns the presence ratio of this field relative to total records. + * + * @param totalRecords the total number of records + * @return the presence ratio (0.0 to 1.0) + */ + public float presenceRatio(long totalRecords) { + if (totalRecords == 0) { + return 0.0f; + } + return (float) presenceCount / totalRecords; + } + + /** + * Returns the dominant type (the type with the highest count). + * + * @return the most frequently observed type, or null if no observations + */ + public DataTypeRoot dominantType() { + DataTypeRoot dominant = null; + long maxCount = 0; + for (Map.Entry entry : typeCounts.entrySet()) { + if (entry.getValue() > maxCount) { + maxCount = entry.getValue(); + dominant = entry.getKey(); + } + } + return dominant; + } + + /** + * Returns the type consistency ratio, i.e., how consistently this field appears with the same + * type. + * + * @return the ratio of the dominant type count to the total presence count (0.0 to 1.0) + */ + public float typeConsistency() { + if (presenceCount == 0) { + return 0.0f; + } + long maxCount = 0; + for (Long count : typeCounts.values()) { + if (count > maxCount) { + maxCount = count; + } + } + return (float) maxCount / presenceCount; + } + + /** + * Merges another FieldStatistics into this one. + * + * @param other the other statistics to merge + */ + public void merge(FieldStatistics other) { + if (!this.fieldPath.equals(other.fieldPath)) { + throw new IllegalArgumentException( + String.format( + "Cannot merge statistics for different fields: '%s' vs '%s'", + this.fieldPath, other.fieldPath)); + } + this.presenceCount += other.presenceCount; + for (Map.Entry entry : other.typeCounts.entrySet()) { + this.typeCounts.merge(entry.getKey(), entry.getValue(), Long::sum); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof FieldStatistics)) { + return false; + } + FieldStatistics that = (FieldStatistics) o; + return presenceCount == that.presenceCount + && Objects.equals(fieldPath, that.fieldPath) + && Objects.equals(typeCounts, that.typeCounts); + } + + @Override + public int hashCode() { + return Objects.hash(fieldPath, presenceCount, typeCounts); + } + + @Override + public String toString() { + return String.format( + "FieldStatistics{path='%s', presence=%d, types=%s}", + fieldPath, presenceCount, typeCounts); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddedField.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddedField.java new file mode 100644 index 0000000000..80c53b1b99 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddedField.java @@ -0,0 +1,87 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.types.DataType; + +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Represents a single field that has been extracted (shredded) from a Variant column into a + * typed_value sub-vector within the Variant's StructVector for efficient columnar access. + * + *

For example, if a Variant column "data" frequently contains an integer field "age", it will be + * shredded into a typed_value child: {@code StructVector{value: VarBinary, typed_value: BigInt}}. + * + * @since 0.7 + */ +@PublicEvolving +public final class ShreddedField implements Serializable { + + private static final long serialVersionUID = 1L; + + /** The field path within the Variant object (e.g., "age", "address.city"). */ + private final String fieldPath; + + /** The data type of the shredded column. */ + private final DataType shreddedType; + + public ShreddedField(String fieldPath, DataType shreddedType) { + this.fieldPath = checkNotNull(fieldPath, "fieldPath must not be null"); + this.shreddedType = checkNotNull(shreddedType, "shreddedType must not be null"); + } + + /** Returns the field path within the Variant object. */ + public String getFieldPath() { + return fieldPath; + } + + /** Returns the data type of the shredded column. */ + public DataType getShreddedType() { + return shreddedType; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ShreddedField)) { + return false; + } + ShreddedField that = (ShreddedField) o; + return Objects.equals(fieldPath, that.fieldPath) + && Objects.equals(shreddedType, that.shreddedType); + } + + @Override + public int hashCode() { + return Objects.hash(fieldPath, shreddedType); + } + + @Override + public String toString() { + return String.format( + "ShreddedField{fieldPath='%s', type=%s}", + fieldPath, shreddedType.asSummaryString()); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddedVariant.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddedVariant.java new file mode 100644 index 0000000000..9e4d7d66ea --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddedVariant.java @@ -0,0 +1,457 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeRoot; + +import javax.annotation.Nullable; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A lazily-merged Variant that carries shredded field data alongside residual binary data. Instead + * of immediately merging typed_value columns back into Variant binary encoding (which is then + * re-parsed by downstream consumers), this class defers the merge until {@link #metadata()} or + * {@link #value()} is actually called. + * + *

This enables downstream consumers (e.g., Flink's FlussVariant) to directly access shredded + * field values as Java objects (String, Long, Boolean, etc.) without any binary encoding/decoding + * overhead when querying individual fields. + * + *

Data layout: + * + *

    + *
  • residualMetadata: metadata dictionary from the residual Variant (may be null if no + * residual) + *
  • residualValue: residual value bytes (may be null if all fields are shredded) + *
  • fieldNames: parallel array of shredded field names + *
  • fieldTypes: parallel array of shredded field data types + *
  • typedValues: parallel array of Java objects (non-null if typed_value matched) + *
  • fallbackValues: parallel array of Variant binary (non-null if typed_value didn't + * match) + *
+ * + * @since 0.7 + */ +@Internal +public class ShreddedVariant extends Variant { + + private static final long serialVersionUID = 1L; + + // Residual data (from the non-shredded portion) + @Nullable private final byte[] residualMetadata; + @Nullable private final byte[] residualValue; + + // Shredded fields: parallel arrays + private final String[] fieldNames; + private final DataType[] fieldTypes; + private final Object[] typedValues; // Java objects: String, Long, Boolean, etc. + private final byte[][] fallbackValues; // Variant binary when type didn't match + + // Field name → index lookup (lazily built) + private transient Map fieldIndexMap; + + // Lazily merged full Variant binary + private transient byte[] mergedMetadata; + private transient byte[] mergedValue; + private transient boolean merged; + + /** + * Creates a ShreddedVariant with residual data and shredded field data. + * + * @param residualMetadata metadata bytes from the residual (may be null) + * @param residualValue residual value bytes (may be null if all fields are shredded) + * @param fieldNames parallel array of shredded field names + * @param fieldTypes parallel array of shredded field data types + * @param typedValues parallel array of typed Java objects (null entry = not typed-matched) + * @param fallbackValues parallel array of fallback Variant binary (null entry = not present) + */ + public ShreddedVariant( + @Nullable byte[] residualMetadata, + @Nullable byte[] residualValue, + String[] fieldNames, + DataType[] fieldTypes, + Object[] typedValues, + byte[][] fallbackValues) { + super(); // protected no-arg constructor + this.residualMetadata = residualMetadata; + this.residualValue = residualValue; + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + this.typedValues = typedValues; + this.fallbackValues = fallbackValues; + this.merged = false; + } + + // -------------------------------------------------------------------------------------------- + // Lazy merge: override metadata() and value() to trigger merge on demand + // -------------------------------------------------------------------------------------------- + + @Override + public byte[] metadata() { + ensureMerged(); + return mergedMetadata; + } + + @Override + public byte[] value() { + ensureMerged(); + return mergedValue; + } + + // -------------------------------------------------------------------------------------------- + // Direct field access API (for FlussVariant fast-path) + // -------------------------------------------------------------------------------------------- + + /** Returns the shredded field names. */ + public String[] getShreddedFieldNames() { + return fieldNames; + } + + /** Returns the shredded field types. */ + public DataType[] getShreddedFieldTypes() { + return fieldTypes; + } + + /** Returns whether the given field name exists as a shredded field. */ + public boolean hasShreddedField(String name) { + return getFieldIndex(name) >= 0; + } + + /** + * Returns the typed Java object for a shredded field, or null if the field is not present or + * uses fallback binary. The returned object type depends on the field's DataType: + * + *
    + *
  • BOOLEAN → Boolean + *
  • TINYINT → Byte + *
  • SMALLINT → Short + *
  • INTEGER → Integer + *
  • BIGINT → Long + *
  • FLOAT → Float + *
  • DOUBLE → Double + *
  • STRING/CHAR → String + *
  • DATE → Integer (days since epoch) + *
  • TIMESTAMP_* → Long (microseconds since epoch) + *
  • DECIMAL → BigDecimal + *
+ */ + @Nullable + public Object getTypedFieldValue(String name) { + int idx = getFieldIndex(name); + if (idx < 0) { + return null; + } + return typedValues[idx]; + } + + /** + * Returns the fallback Variant binary for a shredded field (when the actual value didn't match + * the expected typed_value type), or null if the field is not present or uses a typed value. + */ + @Nullable + public byte[] getFallbackFieldValue(String name) { + int idx = getFieldIndex(name); + if (idx < 0) { + return null; + } + return fallbackValues[idx]; + } + + /** Returns the DataType of a shredded field, or null if the field name is not shredded. */ + @Nullable + public DataType getShreddedFieldType(String name) { + int idx = getFieldIndex(name); + if (idx < 0) { + return null; + } + return fieldTypes[idx]; + } + + /** Returns the residual metadata bytes, or null if no residual exists. */ + @Nullable + public byte[] getResidualMetadata() { + return residualMetadata; + } + + /** Returns the residual value bytes, or null if no residual exists. */ + @Nullable + public byte[] getResidualValue() { + return residualValue; + } + + // -------------------------------------------------------------------------------------------- + // Internal: lazy merge logic + // -------------------------------------------------------------------------------------------- + + private void ensureMerged() { + if (merged) { + return; + } + doMerge(); + merged = true; + } + + private void doMerge() { + boolean hasResidual = residualValue != null; + boolean hasShreddedValues = false; + for (int i = 0; i < fieldNames.length; i++) { + if (typedValues[i] != null || fallbackValues[i] != null) { + hasShreddedValues = true; + break; + } + } + + if (!hasResidual && !hasShreddedValues) { + // No data at all, produce null variant + mergedMetadata = Variant.emptyMetadata(); + mergedValue = VariantUtil.encodeNull(); + return; + } + + if (!hasShreddedValues) { + // Only residual + mergedMetadata = residualMetadata; + mergedValue = residualValue; + return; + } + + if (!hasResidual) { + // Only shredded values + buildFromShreddedOnly(); + return; + } + + // Both residual and shredded - merge + mergeWithResidual(); + } + + private void buildFromShreddedOnly() { + List presentNames = new ArrayList<>(); + List presentValues = new ArrayList<>(); + + for (int i = 0; i < fieldNames.length; i++) { + byte[] encoded = encodeFieldValue(i); + if (encoded != null) { + presentNames.add(fieldNames[i]); + presentValues.add(encoded); + } + } + + if (presentNames.isEmpty()) { + mergedMetadata = Variant.emptyMetadata(); + mergedValue = VariantUtil.encodeNull(); + return; + } + + mergedMetadata = + (residualMetadata != null) ? residualMetadata : buildMetadata(presentNames); + + List fieldIds = new ArrayList<>(); + for (String name : presentNames) { + fieldIds.add(VariantUtil.findFieldId(mergedMetadata, name)); + } + mergedValue = VariantUtil.encodeObject(fieldIds, presentValues); + } + + private void mergeWithResidual() { + // Check if residual is an object + int basicType = VariantUtil.basicType(residualValue, 0); + if (basicType != VariantUtil.BASIC_TYPE_OBJECT) { + // Residual is not an object - return as-is + mergedMetadata = residualMetadata; + mergedValue = residualValue; + return; + } + + // Collect fields from residual + int numResidualFields = VariantUtil.objectSize(residualValue, 0); + Set allFieldNames = new LinkedHashSet<>(); + Map fieldValueMap = new LinkedHashMap<>(); + + for (int i = 0; i < numResidualFields; i++) { + int fieldId = VariantUtil.objectFieldId(residualValue, 0, i); + String fieldName = VariantUtil.metadataFieldName(residualMetadata, fieldId); + int fieldValueOffset = VariantUtil.objectFieldValueOffset(residualValue, 0, i); + int fieldValueSize = VariantUtil.valueSize(residualValue, fieldValueOffset); + byte[] fieldValueBytes = + Arrays.copyOfRange( + residualValue, fieldValueOffset, fieldValueOffset + fieldValueSize); + allFieldNames.add(fieldName); + fieldValueMap.put(fieldName, fieldValueBytes); + } + + // Add shredded fields (overwrite residual if same name) + boolean needNewMetadata = false; + for (int i = 0; i < fieldNames.length; i++) { + byte[] encoded = encodeFieldValue(i); + if (encoded == null) { + continue; + } + String name = fieldNames[i]; + if (!allFieldNames.contains(name)) { + if (VariantUtil.findFieldId(residualMetadata, name) < 0) { + needNewMetadata = true; + } + } + allFieldNames.add(name); + fieldValueMap.put(name, encoded); + } + + // Build metadata + if (needNewMetadata) { + Set allMetadataNames = new LinkedHashSet<>(); + int origDictSize = VariantUtil.metadataDictSize(residualMetadata); + for (int d = 0; d < origDictSize; d++) { + allMetadataNames.add(VariantUtil.metadataFieldName(residualMetadata, d)); + } + allMetadataNames.addAll(allFieldNames); + mergedMetadata = buildMetadata(new ArrayList<>(allMetadataNames)); + } else { + mergedMetadata = residualMetadata; + } + + // Build merged object value + List fieldIds = new ArrayList<>(); + List fieldValuesList = new ArrayList<>(); + for (Map.Entry entry : fieldValueMap.entrySet()) { + int fieldId = VariantUtil.findFieldId(mergedMetadata, entry.getKey()); + fieldIds.add(fieldId); + fieldValuesList.add(entry.getValue()); + } + mergedValue = VariantUtil.encodeObject(fieldIds, fieldValuesList); + } + + // -------------------------------------------------------------------------------------------- + // Encode a shredded field's typed value or fallback to Variant binary + // -------------------------------------------------------------------------------------------- + + @Nullable + private byte[] encodeFieldValue(int fieldIndex) { + Object typed = typedValues[fieldIndex]; + if (typed != null) { + return encodeTypedObject(typed, fieldTypes[fieldIndex]); + } + byte[] fallback = fallbackValues[fieldIndex]; + if (fallback != null) { + return fallback; + } + return null; // field not present + } + + /** Encodes a typed Java object to Variant value binary. */ + public static byte[] encodeTypedObject(Object value, DataType type) { + DataTypeRoot root = type.getTypeRoot(); + switch (root) { + case BOOLEAN: + return VariantUtil.encodeBoolean((Boolean) value); + case TINYINT: + return VariantUtil.encodeByte((Byte) value); + case SMALLINT: + return VariantUtil.encodeShort((Short) value); + case INTEGER: + return VariantUtil.encodeInt((Integer) value); + case BIGINT: + return VariantUtil.encodeLong((Long) value); + case FLOAT: + return VariantUtil.encodeFloat((Float) value); + case DOUBLE: + return VariantUtil.encodeDouble((Double) value); + case STRING: + case CHAR: + return VariantUtil.encodeString(((String) value).getBytes(StandardCharsets.UTF_8)); + case DATE: + return VariantUtil.encodeDate((Integer) value); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return VariantUtil.encodeTimestamp((Long) value); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return VariantUtil.encodeTimestampNtz((Long) value); + default: + throw new UnsupportedOperationException( + "Unsupported shredded type for encoding: " + type); + } + } + + // -------------------------------------------------------------------------------------------- + // Metadata builder (same logic as ShreddedVariantColumnVector.buildMetadata) + // -------------------------------------------------------------------------------------------- + + static byte[] buildMetadata(List fieldNames) { + List sorted = new ArrayList<>(fieldNames); + java.util.Collections.sort(sorted); + + byte[][] nameBytes = new byte[sorted.size()][]; + int totalStringBytes = 0; + for (int i = 0; i < sorted.size(); i++) { + nameBytes[i] = sorted.get(i).getBytes(StandardCharsets.UTF_8); + totalStringBytes += nameBytes[i].length; + } + + int dictSize = sorted.size(); + int totalSize = 1 + 4 + (dictSize + 1) * 4 + totalStringBytes; + byte[] metadata = new byte[totalSize]; + + metadata[0] = + (byte) + (VariantUtil.METADATA_VERSION + | VariantUtil.METADATA_SORTED_STRINGS_BIT + | VariantUtil.METADATA_OFFSET_SIZE_BITS); + + VariantUtil.writeIntLE(metadata, 1, dictSize); + + int offsetPos = 5; + int strOffset = 0; + int strBytesStart = 5 + (dictSize + 1) * 4; + for (int i = 0; i < dictSize; i++) { + VariantUtil.writeIntLE(metadata, offsetPos, strOffset); + offsetPos += 4; + System.arraycopy( + nameBytes[i], 0, metadata, strBytesStart + strOffset, nameBytes[i].length); + strOffset += nameBytes[i].length; + } + VariantUtil.writeIntLE(metadata, offsetPos, strOffset); + + return metadata; + } + + // -------------------------------------------------------------------------------------------- + // Field index lookup + // -------------------------------------------------------------------------------------------- + + private int getFieldIndex(String name) { + if (fieldIndexMap == null) { + Map map = new HashMap<>(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + map.put(fieldNames[i], i); + } + fieldIndexMap = map; + } + Integer idx = fieldIndexMap.get(name); + return idx != null ? idx : -1; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddingSchema.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddingSchema.java new file mode 100644 index 0000000000..b986fd4b14 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddingSchema.java @@ -0,0 +1,364 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeParser; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Describes the shredding schema for a Variant column. A shredding schema defines which fields + * within a Variant value are extracted into typed_value sub-vectors within the Variant's + * StructVector for efficient columnar access. + * + *

With the Writer-independent-decision approach (Plan A), each Writer independently infers the + * shredding schema based on collected field statistics and uses it to construct the per-batch + * StructVector layout. No server-side coordination or storage is involved. + * + * @since 0.7 + */ +@PublicEvolving +public final class ShreddingSchema implements Serializable { + + private static final long serialVersionUID = 1L; + + /** The name of the Variant column this shredding schema belongs to. */ + private final String variantColumnName; + + /** The list of fields that have been shredded from the Variant column. */ + private final List fields; + + public ShreddingSchema(String variantColumnName, List fields) { + this.variantColumnName = + checkNotNull(variantColumnName, "variantColumnName must not be null"); + this.fields = Collections.unmodifiableList(new ArrayList<>(checkNotNull(fields))); + } + + /** Returns the name of the Variant column. */ + public String getVariantColumnName() { + return variantColumnName; + } + + /** Returns the list of shredded fields. */ + public List getFields() { + return fields; + } + + // -------------------------------------------------------------------------------------------- + // JSON serialization / deserialization + // -------------------------------------------------------------------------------------------- + + /** + * Serializes this shredding schema to a JSON string. + * + *

Format: + * + *

{@code
+     * {
+     *   "variant_column": "event",
+     *   "fields": [
+     *     {"path": "age", "type": "INT"},
+     *     {"path": "name", "type": "STRING"}
+     *   ]
+     * }
+     * }
+ */ + public String toJson() { + StringBuilder sb = new StringBuilder(); + sb.append("{\"variant_column\":\""); + sb.append(escapeJson(variantColumnName)); + sb.append("\",\"fields\":["); + for (int i = 0; i < fields.size(); i++) { + if (i > 0) { + sb.append(","); + } + ShreddedField field = fields.get(i); + sb.append("{\"path\":\""); + sb.append(escapeJson(field.getFieldPath())); + sb.append("\",\"type\":\""); + sb.append(escapeJson(field.getShreddedType().asSerializableString())); + sb.append("\"}"); + } + sb.append("]}"); + return sb.toString(); + } + + /** + * Deserializes a shredding schema from a JSON string. + * + * @param json the JSON string + * @return the deserialized ShreddingSchema + */ + public static ShreddingSchema fromJson(String json) { + SimpleJsonReader reader = new SimpleJsonReader(json); + reader.expect('{'); + String variantColumn = null; + List fields = new ArrayList<>(); + + while (!reader.isEndOfObject()) { + String key = reader.readString(); + reader.expect(':'); + if ("variant_column".equals(key)) { + variantColumn = reader.readString(); + } else if ("fields".equals(key)) { + reader.expect('['); + while (!reader.isEndOfArray()) { + reader.expect('{'); + String path = null; + String typeStr = null; + while (!reader.isEndOfObject()) { + String fieldKey = reader.readString(); + reader.expect(':'); + if ("path".equals(fieldKey)) { + path = reader.readString(); + } else if ("type".equals(fieldKey)) { + typeStr = reader.readString(); + } else { + reader.skipValue(); + } + reader.skipComma(); + } + reader.expect('}'); + if (path == null || typeStr == null) { + throw new IllegalArgumentException( + "Missing 'path' or 'type' in field JSON: " + json); + } + DataType dataType = DataTypeParser.parse(typeStr); + fields.add(new ShreddedField(path, dataType)); + reader.skipComma(); + } + reader.expect(']'); + } else { + reader.skipValue(); + } + reader.skipComma(); + } + reader.expect('}'); + + if (variantColumn == null) { + throw new IllegalArgumentException("Missing 'variant_column' in JSON: " + json); + } + return new ShreddingSchema(variantColumn, fields); + } + + /** + * Minimal JSON reader for deserializing the shredding schema format. Handles only the subset of + * JSON produced by {@link #toJson()}: string values, integer values, objects, and arrays. + */ + private static class SimpleJsonReader { + private final String json; + private int pos; + + SimpleJsonReader(String json) { + this.json = json; + this.pos = 0; + } + + void skipWhitespace() { + while (pos < json.length() && Character.isWhitespace(json.charAt(pos))) { + pos++; + } + } + + void expect(char c) { + skipWhitespace(); + if (pos >= json.length() || json.charAt(pos) != c) { + throw new IllegalArgumentException( + "Expected '" + c + "' at pos " + pos + " in: " + json); + } + pos++; + } + + void skipComma() { + skipWhitespace(); + if (pos < json.length() && json.charAt(pos) == ',') { + pos++; + } + } + + boolean isEndOfObject() { + skipWhitespace(); + return pos < json.length() && json.charAt(pos) == '}'; + } + + boolean isEndOfArray() { + skipWhitespace(); + return pos < json.length() && json.charAt(pos) == ']'; + } + + String readString() { + skipWhitespace(); + expect('"'); + StringBuilder sb = new StringBuilder(); + while (pos < json.length()) { + char c = json.charAt(pos++); + if (c == '"') { + return sb.toString(); + } + if (c == '\\') { + if (pos >= json.length()) { + throw new IllegalArgumentException("Unexpected end of string in: " + json); + } + char escaped = json.charAt(pos++); + switch (escaped) { + case '"': + sb.append('"'); + break; + case '\\': + sb.append('\\'); + break; + case '/': + sb.append('/'); + break; + case 'n': + sb.append('\n'); + break; + case 'r': + sb.append('\r'); + break; + case 't': + sb.append('\t'); + break; + case 'b': + sb.append('\b'); + break; + case 'f': + sb.append('\f'); + break; + case 'u': + if (pos + 4 > json.length()) { + throw new IllegalArgumentException( + "Invalid unicode escape in: " + json); + } + int codeUnit = Integer.parseInt(json.substring(pos, pos + 4), 16); + pos += 4; + // Handle surrogate pairs so supplementary code points (e.g. emoji + // encoded as \\uD83D\\uDE00) are preserved correctly. + if (Character.isHighSurrogate((char) codeUnit) + && pos + 6 <= json.length() + && json.charAt(pos) == '\\' + && json.charAt(pos + 1) == 'u') { + int low = Integer.parseInt(json.substring(pos + 2, pos + 6), 16); + if (Character.isLowSurrogate((char) low)) { + sb.appendCodePoint( + Character.toCodePoint((char) codeUnit, (char) low)); + pos += 6; + break; + } + } + sb.append((char) codeUnit); + break; + default: + sb.append(escaped); + } + } else { + sb.append(c); + } + } + throw new IllegalArgumentException("Unterminated string in: " + json); + } + + int readInt() { + skipWhitespace(); + int start = pos; + if (pos < json.length() && json.charAt(pos) == '-') { + pos++; + } + while (pos < json.length() && Character.isDigit(json.charAt(pos))) { + pos++; + } + return Integer.parseInt(json.substring(start, pos)); + } + + void skipValue() { + skipWhitespace(); + if (pos >= json.length()) { + return; + } + char c = json.charAt(pos); + if (c == '"') { + readString(); + } else if (c == '{') { + expect('{'); + while (!isEndOfObject()) { + readString(); + expect(':'); + skipValue(); + skipComma(); + } + expect('}'); + } else if (c == '[') { + expect('['); + while (!isEndOfArray()) { + skipValue(); + skipComma(); + } + expect(']'); + } else { + // number or boolean or null - skip until delimiter + while (pos < json.length()) { + char ch = json.charAt(pos); + if (ch == ',' || ch == '}' || ch == ']' || Character.isWhitespace(ch)) { + break; + } + pos++; + } + } + } + } + + // -------------------------------------------------------------------------------------------- + // Helper methods + // -------------------------------------------------------------------------------------------- + + private static String escapeJson(String s) { + return s.replace("\\", "\\\\").replace("\"", "\\\""); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ShreddingSchema)) { + return false; + } + ShreddingSchema that = (ShreddingSchema) o; + return Objects.equals(variantColumnName, that.variantColumnName) + && Objects.equals(fields, that.fields); + } + + @Override + public int hashCode() { + return Objects.hash(variantColumnName, fields); + } + + @Override + public String toString() { + return String.format( + "ShreddingSchema{variantColumn='%s', fields=%s}", variantColumnName, fields); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddingSchemaInferrer.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddingSchemaInferrer.java new file mode 100644 index 0000000000..f08dc5b712 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/ShreddingSchemaInferrer.java @@ -0,0 +1,336 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeRoot; +import org.apache.fluss.types.DataTypes; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * Infers a {@link ShreddingSchema} from field statistics collected by {@link + * VariantStatisticsCollector}. The inference algorithm selects fields that appear frequently with + * consistent types and creates typed columns for them. + * + *

Algorithm

+ * + *

For each field in the statistics: + * + *

    + *
  1. Compute {@code presence_ratio = presence_count / total_records} + *
  2. Compute {@code dominant_type = argmax(type_counts)} + *
  3. Compute {@code type_consistency = type_counts[dominant_type] / presence_count} + *
  4. If {@code presence_ratio >= PRESENCE_THRESHOLD} AND {@code type_consistency >= + * TYPE_CONSISTENCY_THRESHOLD} AND {@code dominant_type} is shreddable: mark this field for + * shredding + *
+ * + *

Fields are ranked by {@code presence_ratio * type_consistency} and limited to {@code + * maxShreddedFields}. + * + * @since 0.7 + */ +@Internal +public class ShreddingSchemaInferrer implements Serializable { + + private static final long serialVersionUID = 1L; + + /** Default presence threshold: field must appear in >= 50% of records. */ + public static final float DEFAULT_PRESENCE_THRESHOLD = 0.5f; + + /** Default type consistency threshold: >= 90% of occurrences must have the same type. */ + public static final float DEFAULT_TYPE_CONSISTENCY_THRESHOLD = 0.9f; + + /** Default maximum number of shredded fields per Variant column. */ + public static final int DEFAULT_MAX_SHREDDED_FIELDS = 20; + + /** Default minimum number of records required before inference can run. */ + public static final int DEFAULT_MIN_SAMPLE_SIZE = 1000; + + /** + * Types that can be shredded into independent typed columns. + * + *

All integer sub-types (TINYINT, SMALLINT, INTEGER, BIGINT) are included. When statistics + * are collected via {@link VariantStatisticsCollector}, all Variant integer encodings (INT8, + * INT16, INT32, INT64) are already widened to {@link DataTypeRoot#BIGINT} before being + * recorded, so only BIGINT will appear in practice. The smaller types remain here to support + * manually constructed {@link FieldStatistics} (e.g., in tests or external integrations). + */ + private static final Set SHREDDABLE_TYPES = + EnumSet.of( + DataTypeRoot.BOOLEAN, + DataTypeRoot.TINYINT, + DataTypeRoot.SMALLINT, + DataTypeRoot.INTEGER, + DataTypeRoot.BIGINT, + DataTypeRoot.FLOAT, + DataTypeRoot.DOUBLE, + DataTypeRoot.STRING, + DataTypeRoot.DATE, + DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, + DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + + private float presenceThreshold; + private float typeConsistencyThreshold; + private int maxShreddedFields; + private int minSampleSize; + + public ShreddingSchemaInferrer() { + this.presenceThreshold = DEFAULT_PRESENCE_THRESHOLD; + this.typeConsistencyThreshold = DEFAULT_TYPE_CONSISTENCY_THRESHOLD; + this.maxShreddedFields = DEFAULT_MAX_SHREDDED_FIELDS; + this.minSampleSize = DEFAULT_MIN_SAMPLE_SIZE; + } + + public ShreddingSchemaInferrer setPresenceThreshold(float presenceThreshold) { + this.presenceThreshold = presenceThreshold; + return this; + } + + public ShreddingSchemaInferrer setTypeConsistencyThreshold(float typeConsistencyThreshold) { + this.typeConsistencyThreshold = typeConsistencyThreshold; + return this; + } + + public ShreddingSchemaInferrer setMaxShreddedFields(int maxShreddedFields) { + this.maxShreddedFields = maxShreddedFields; + return this; + } + + public ShreddingSchemaInferrer setMinSampleSize(int minSampleSize) { + this.minSampleSize = minSampleSize; + return this; + } + + public float getPresenceThreshold() { + return presenceThreshold; + } + + public float getTypeConsistencyThreshold() { + return typeConsistencyThreshold; + } + + public int getMaxShreddedFields() { + return maxShreddedFields; + } + + public int getMinSampleSize() { + return minSampleSize; + } + + /** + * Infers a shredding schema based on collected field statistics. + * + * @param variantColumnName the name of the Variant column + * @param stats the field statistics map + * @param totalRecords the total number of records analyzed + * @return the inferred ShreddingSchema + */ + public ShreddingSchema infer( + String variantColumnName, Map stats, long totalRecords) { + if (totalRecords < minSampleSize) { + // Not enough data to infer reliably + return new ShreddingSchema(variantColumnName, new ArrayList<>()); + } + + List candidates = new ArrayList<>(); + for (FieldStatistics fieldStats : stats.values()) { + float presenceRatio = fieldStats.presenceRatio(totalRecords); + float consistency = fieldStats.typeConsistency(); + DataTypeRoot dominantType = fieldStats.dominantType(); + + if (presenceRatio >= presenceThreshold + && consistency >= typeConsistencyThreshold + && dominantType != null + && SHREDDABLE_TYPES.contains(dominantType)) { + candidates.add( + new CandidateField( + fieldStats.getFieldPath(), + dominantType, + presenceRatio * consistency)); + } + } + + // Sort by score descending, take top-N + candidates.sort(Comparator.comparingDouble(c -> -c.score)); + if (candidates.size() > maxShreddedFields) { + candidates = candidates.subList(0, maxShreddedFields); + } + + // Build the shredded fields + List fields = new ArrayList<>(); + for (int i = 0; i < candidates.size(); i++) { + CandidateField candidate = candidates.get(i); + DataType dataType = mapToDataType(candidate.typeRoot); + fields.add(new ShreddedField(candidate.fieldPath, dataType)); + } + + return new ShreddingSchema(variantColumnName, fields); + } + + /** + * Incrementally updates a shredding schema based on new statistics. Determines if changes are + * needed by comparing new statistics against the existing schema. + * + * @param current the current shredding schema + * @param newStats the new field statistics + * @param totalRecords the total number of records in the new statistics + * @return an updated ShreddingSchema if changes are needed, or empty if no changes required + */ + public Optional update( + ShreddingSchema current, Map newStats, long totalRecords) { + if (totalRecords < minSampleSize) { + return Optional.empty(); + } + + // Get currently shredded field paths + Set currentPaths = new HashSet<>(); + for (ShreddedField field : current.getFields()) { + currentPaths.add(field.getFieldPath()); + } + + // Check if any existing field should be demoted (type changed or presence dropped) + boolean hasChanges = false; + List updatedFields = new ArrayList<>(); + for (ShreddedField existing : current.getFields()) { + FieldStatistics stats = newStats.get(existing.getFieldPath()); + if (stats != null) { + float presenceRatio = stats.presenceRatio(totalRecords); + float consistency = stats.typeConsistency(); + DataTypeRoot dominantType = stats.dominantType(); + + if (presenceRatio >= presenceThreshold + && consistency >= typeConsistencyThreshold + && dominantType != null + && SHREDDABLE_TYPES.contains(dominantType) + && mapToDataType(dominantType) + .equals(existing.getShreddedType().copy(true))) { + // Field still qualifies and type hasn't changed - keep it + updatedFields.add(existing); + } else { + // Field no longer qualifies or type changed - demote + hasChanges = true; + } + } else { + // Field not in new stats - demote + hasChanges = true; + } + } + + // Check for new fields that should be promoted + List newCandidates = new ArrayList<>(); + for (FieldStatistics stats : newStats.values()) { + if (currentPaths.contains(stats.getFieldPath())) { + continue; // Already handled + } + float presenceRatio = stats.presenceRatio(totalRecords); + float consistency = stats.typeConsistency(); + DataTypeRoot dominantType = stats.dominantType(); + + if (presenceRatio >= presenceThreshold + && consistency >= typeConsistencyThreshold + && dominantType != null + && SHREDDABLE_TYPES.contains(dominantType)) { + newCandidates.add( + new CandidateField( + stats.getFieldPath(), dominantType, presenceRatio * consistency)); + } + } + + if (!newCandidates.isEmpty()) { + hasChanges = true; + // Sort new candidates by score descending + newCandidates.sort(Comparator.comparingDouble(c -> -c.score)); + int remaining = maxShreddedFields - updatedFields.size(); + for (int i = 0; i < Math.min(newCandidates.size(), remaining); i++) { + CandidateField candidate = newCandidates.get(i); + DataType dataType = mapToDataType(candidate.typeRoot); + updatedFields.add(new ShreddedField(candidate.fieldPath, dataType)); + } + } + + if (!hasChanges) { + return Optional.empty(); + } + + return Optional.of(new ShreddingSchema(current.getVariantColumnName(), updatedFields)); + } + + // -------------------------------------------------------------------------------------------- + // Internal helpers + // -------------------------------------------------------------------------------------------- + + /** + * Maps a DataTypeRoot to the corresponding nullable DataType for shredded columns. + * + *

When statistics are collected via {@link VariantStatisticsCollector}, all Variant integer + * encodings are widened to {@link DataTypeRoot#BIGINT}. The smaller integer types are still + * handled here to support manually constructed {@link FieldStatistics}. + */ + private static DataType mapToDataType(DataTypeRoot typeRoot) { + switch (typeRoot) { + case BOOLEAN: + return DataTypes.BOOLEAN(); + case TINYINT: + return DataTypes.TINYINT(); + case SMALLINT: + return DataTypes.SMALLINT(); + case INTEGER: + return DataTypes.INT(); + case BIGINT: + return DataTypes.BIGINT(); + case FLOAT: + return DataTypes.FLOAT(); + case DOUBLE: + return DataTypes.DOUBLE(); + case STRING: + return DataTypes.STRING(); + case DATE: + return DataTypes.DATE(); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return DataTypes.TIMESTAMP(6); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return DataTypes.TIMESTAMP_LTZ(6); + default: + throw new IllegalArgumentException("Unsupported shreddable type: " + typeRoot); + } + } + + /** Internal candidate field for ranking. */ + private static class CandidateField { + final String fieldPath; + final DataTypeRoot typeRoot; + final double score; + + CandidateField(String fieldPath, DataTypeRoot typeRoot, double score) { + this.fieldPath = fieldPath; + this.typeRoot = typeRoot; + this.score = score; + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/Variant.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/Variant.java new file mode 100644 index 0000000000..b7e36d7d9a --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/Variant.java @@ -0,0 +1,396 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.PublicStable; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Represents a semi-structured variant value, stored using the Parquet Variant Binary Encoding + * format. A Variant consists of two byte arrays: + * + *

    + *
  • metadata: A dictionary of field names used by object values. Contains a header byte, + * dictionary size, offsets, and UTF-8 encoded string bytes. + *
  • value: A self-describing binary-encoded value. The first byte encodes the basic type + * (Primitive, ShortString, Object, Array) and type-specific header information. + *
+ * + *

For storage in Arrow {@code VarBinaryVector}, the two arrays are concatenated as: {@code + * [4-byte metadata length (little-endian)][metadata bytes][value bytes]}. + * + * @since 0.7 + */ +@PublicStable +public class Variant implements Serializable { + + private static final long serialVersionUID = 1L; + + private final byte[] metadata; + private final byte[] value; + + public Variant(byte[] metadata, byte[] value) { + this.metadata = checkNotNull(metadata, "metadata must not be null"); + this.value = checkNotNull(value, "value must not be null"); + } + + /** + * Protected constructor for subclasses that override {@link #metadata()} and {@link #value()} + * to provide lazy initialization. All public methods in this class access data through the + * accessor methods, so subclasses can safely override them. + */ + protected Variant() { + this.metadata = null; + this.value = null; + } + + /** Returns the metadata bytes (field name dictionary). */ + public byte[] metadata() { + return metadata; + } + + /** Returns the value bytes (self-describing encoded value). */ + public byte[] value() { + return value; + } + + // -------------------------------------------------------------------------------------------- + // Binary serialization (for Arrow VarBinaryVector storage) + // -------------------------------------------------------------------------------------------- + + /** + * Serializes this variant to a single byte array for storage. + * + *

Format: {@code [4-byte metadata length (little-endian)][metadata bytes][value bytes]} + */ + public byte[] toBytes() { + byte[] m = metadata(); + byte[] v = value(); + byte[] result = new byte[4 + m.length + v.length]; + ByteBuffer buf = ByteBuffer.wrap(result).order(ByteOrder.LITTLE_ENDIAN); + buf.putInt(m.length); + buf.put(m); + buf.put(v); + return result; + } + + /** + * Deserializes a variant from a byte array. + * + * @param bytes the serialized bytes + * @return the deserialized Variant + */ + public static Variant fromBytes(byte[] bytes) { + return fromBytes(bytes, 0, bytes.length); + } + + /** + * Deserializes a variant from a byte array with offset and length. + * + * @param bytes the byte array containing the serialized variant + * @param offset the starting offset + * @param length the total length + * @return the deserialized Variant + */ + public static Variant fromBytes(byte[] bytes, int offset, int length) { + ByteBuffer buf = ByteBuffer.wrap(bytes, offset, length).order(ByteOrder.LITTLE_ENDIAN); + int metadataLen = buf.getInt(); + byte[] meta = new byte[metadataLen]; + buf.get(meta); + byte[] val = new byte[length - 4 - metadataLen]; + buf.get(val); + return new Variant(meta, val); + } + + // -------------------------------------------------------------------------------------------- + // Value type inspection + // -------------------------------------------------------------------------------------------- + + /** Returns the basic type of this variant value. */ + public int basicType() { + return VariantUtil.basicType(value(), 0); + } + + /** Returns true if this variant value is null. */ + public boolean isNull() { + return VariantUtil.isNull(value(), 0); + } + + /** Returns true if this variant is a primitive type. */ + public boolean isPrimitive() { + return basicType() == VariantUtil.BASIC_TYPE_PRIMITIVE; + } + + /** Returns true if this variant is a short string. */ + public boolean isShortString() { + return basicType() == VariantUtil.BASIC_TYPE_SHORT_STRING; + } + + /** Returns true if this variant is an object. */ + public boolean isObject() { + return basicType() == VariantUtil.BASIC_TYPE_OBJECT; + } + + /** Returns true if this variant is an array. */ + public boolean isArray() { + return basicType() == VariantUtil.BASIC_TYPE_ARRAY; + } + + // -------------------------------------------------------------------------------------------- + // Primitive value accessors + // -------------------------------------------------------------------------------------------- + + /** Returns the boolean value of this variant. */ + public boolean getBoolean() { + return VariantUtil.getBoolean(value(), 0); + } + + /** Returns the byte value of this variant. */ + public byte getByte() { + return VariantUtil.getByte(value(), 0); + } + + /** Returns the short value of this variant. */ + public short getShort() { + return VariantUtil.getShort(value(), 0); + } + + /** Returns the int value of this variant. */ + public int getInt() { + return VariantUtil.getInt(value(), 0); + } + + /** Returns the long value of this variant. */ + public long getLong() { + return VariantUtil.getLong(value(), 0); + } + + /** Returns the float value of this variant. */ + public float getFloat() { + return VariantUtil.getFloat(value(), 0); + } + + /** Returns the double value of this variant. */ + public double getDouble() { + return VariantUtil.getDouble(value(), 0); + } + + /** Returns the string value of this variant. */ + public String getString() { + return VariantUtil.getString(value(), 0); + } + + /** Returns the binary value of this variant. */ + public byte[] getBinary() { + return VariantUtil.getBinary(value(), 0); + } + + /** Returns the decimal value of this variant. */ + public BigDecimal getDecimal() { + return VariantUtil.getDecimal(value(), 0); + } + + /** Returns the date value (days since epoch) of this variant. */ + public int getDate() { + return VariantUtil.getDate(value(), 0); + } + + /** Returns the timestamp value (microseconds since epoch) of this variant. */ + public long getTimestamp() { + return VariantUtil.getTimestamp(value(), 0); + } + + /** Returns the timestamp without timezone value (microseconds since epoch) of this variant. */ + public long getTimestampNtz() { + return VariantUtil.getTimestampNtz(value(), 0); + } + + /** + * Returns the total size of this variant in bytes (metadata + value). + * + * @return total byte size + */ + public int sizeInBytes() { + return metadata().length + value().length; + } + + // -------------------------------------------------------------------------------------------- + // Object field access + // -------------------------------------------------------------------------------------------- + + /** + * Returns the number of fields in this object variant. + * + * @throws IllegalStateException if this variant is not an object + */ + public int objectSize() { + return VariantUtil.objectSize(value(), 0); + } + + /** + * Gets a field value from this object variant by field name. + * + * @param fieldName the field name to look up + * @return a new Variant for the field value, or null if the field does not exist + */ + public Variant getFieldByName(String fieldName) { + byte[] m = metadata(); + byte[] v = value(); + int fieldId = VariantUtil.findFieldId(m, fieldName); + if (fieldId < 0) { + return null; + } + int valueOffset = VariantUtil.findFieldValueOffset(v, 0, fieldId); + if (valueOffset < 0) { + return null; + } + int valueSize = VariantUtil.valueSize(v, valueOffset); + byte[] fieldValue = Arrays.copyOfRange(v, valueOffset, valueOffset + valueSize); + return new Variant(m, fieldValue); + } + + // -------------------------------------------------------------------------------------------- + // Array element access + // -------------------------------------------------------------------------------------------- + + /** + * Returns the number of elements in this array variant. + * + * @throws IllegalStateException if this variant is not an array + */ + public int arraySize() { + return VariantUtil.arraySize(value(), 0); + } + + /** + * Gets an element from this array variant by index. + * + * @param index the element index + * @return a new Variant for the element value + */ + public Variant getElementAt(int index) { + byte[] v = value(); + int elementOffset = VariantUtil.arrayElementOffset(v, 0, index); + int elementSize = VariantUtil.valueSize(v, elementOffset); + byte[] elementValue = Arrays.copyOfRange(v, elementOffset, elementOffset + elementSize); + return new Variant(metadata(), elementValue); + } + + // -------------------------------------------------------------------------------------------- + // JSON conversion + // -------------------------------------------------------------------------------------------- + + /** + * Creates a Variant from a JSON string. + * + * @param json the JSON string to parse + * @return a new Variant representing the parsed JSON + */ + public static Variant fromJson(String json) { + return VariantBuilder.parseJson(json); + } + + /** + * Converts this variant to its JSON string representation. + * + * @return a JSON string + */ + public String toJson() { + StringBuilder sb = new StringBuilder(); + VariantUtil.toJson(metadata(), value(), 0, sb); + return sb.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof Variant)) { + return false; + } + Variant variant = (Variant) o; + return Arrays.equals(metadata(), variant.metadata()) + && Arrays.equals(value(), variant.value()); + } + + @Override + public int hashCode() { + int result = Arrays.hashCode(metadata()); + result = 31 * result + Arrays.hashCode(value()); + return result; + } + + @Override + public String toString() { + return toJson(); + } + + // -------------------------------------------------------------------------------------------- + // Utility: empty metadata for primitive-only variants + // -------------------------------------------------------------------------------------------- + + /** Creates an empty metadata (no field names, for primitive/array-only variants). */ + public static byte[] emptyMetadata() { + return VariantUtil.EMPTY_METADATA; + } + + /** Creates a Variant representing a JSON null value. */ + public static Variant ofNull() { + return new Variant(emptyMetadata(), VariantUtil.encodeNull()); + } + + /** Creates a Variant representing a boolean value. */ + public static Variant ofBoolean(boolean value) { + return new Variant(emptyMetadata(), VariantUtil.encodeBoolean(value)); + } + + /** Creates a Variant representing an int value. */ + public static Variant ofInt(int value) { + return new Variant(emptyMetadata(), VariantUtil.encodeInt(value)); + } + + /** Creates a Variant representing a long value. */ + public static Variant ofLong(long value) { + return new Variant(emptyMetadata(), VariantUtil.encodeLong(value)); + } + + /** Creates a Variant representing a float value. */ + public static Variant ofFloat(float value) { + return new Variant(emptyMetadata(), VariantUtil.encodeFloat(value)); + } + + /** Creates a Variant representing a double value. */ + public static Variant ofDouble(double value) { + return new Variant(emptyMetadata(), VariantUtil.encodeDouble(value)); + } + + /** Creates a Variant representing a string value. */ + public static Variant ofString(String value) { + byte[] stringBytes = value.getBytes(StandardCharsets.UTF_8); + return new Variant(emptyMetadata(), VariantUtil.encodeString(stringBytes)); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantBuilder.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantBuilder.java new file mode 100644 index 0000000000..46b6d2d7f6 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantBuilder.java @@ -0,0 +1,457 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.Internal; + +import java.io.ByteArrayOutputStream; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Builder for constructing {@link Variant} values from JSON or programmatically. + * + *

The builder collects field names for the metadata dictionary and encodes values into the + * Parquet Variant Binary Encoding format. + */ +@Internal +public class VariantBuilder { + + private final Map fieldNameToId = new HashMap<>(); + private final List fieldNames = new ArrayList<>(); + + /** Registers a field name and returns its dictionary ID. */ + public int addFieldName(String name) { + Integer id = fieldNameToId.get(name); + if (id != null) { + return id; + } + int newId = fieldNames.size(); + fieldNameToId.put(name, newId); + fieldNames.add(name); + return newId; + } + + /** Builds the metadata bytes from the collected field names. */ + public byte[] buildMetadata() { + if (fieldNames.isEmpty()) { + return VariantUtil.EMPTY_METADATA; + } + + // Sort field names and build sorted_strings dictionary + List sortedNames = new ArrayList<>(fieldNames); + Collections.sort(sortedNames); + + // Encode metadata: header(1) + dict_size(4) + offsets(4*(dict_size+1)) + string_bytes + // Header: version in lower nibble (bits 0-3), sorted_strings flag in bit 4 + byte[] headerByte = + new byte[] { + (byte) + (VariantUtil.METADATA_VERSION + | VariantUtil.METADATA_SORTED_STRINGS_BIT + | VariantUtil.METADATA_OFFSET_SIZE_BITS) + }; + // METADATA_VERSION=1 | METADATA_SORTED_STRINGS_BIT=0x10 | METADATA_OFFSET_SIZE_BITS=0xC0 + // -> 0xD1 + + int dictSize = sortedNames.size(); + ByteArrayOutputStream stringBuf = new ByteArrayOutputStream(); + int[] offsets = new int[dictSize + 1]; + offsets[0] = 0; + for (int i = 0; i < dictSize; i++) { + byte[] nameBytes = sortedNames.get(i).getBytes(StandardCharsets.UTF_8); + stringBuf.write(nameBytes, 0, nameBytes.length); + offsets[i + 1] = stringBuf.size(); + } + + byte[] stringBytes = stringBuf.toByteArray(); + + // header(1) + dict_size(4) + offsets(4*(dictSize+1)) + stringBytes + int totalSize = 1 + 4 + 4 * (dictSize + 1) + stringBytes.length; + byte[] metadata = new byte[totalSize]; + ByteBuffer buf = ByteBuffer.wrap(metadata).order(ByteOrder.LITTLE_ENDIAN); + buf.put(headerByte[0]); + buf.putInt(dictSize); + for (int offset : offsets) { + buf.putInt(offset); + } + buf.put(stringBytes); + + // Rebuild fieldNameToId to match sorted order + fieldNameToId.clear(); + fieldNames.clear(); + for (int i = 0; i < sortedNames.size(); i++) { + fieldNameToId.put(sortedNames.get(i), i); + fieldNames.add(sortedNames.get(i)); + } + + return metadata; + } + + // -------------------------------------------------------------------------------------------- + // JSON parsing + // -------------------------------------------------------------------------------------------- + + /** + * Parses a JSON string into a Variant. + * + * @param json the JSON string to parse + * @return a new Variant + */ + public static Variant parseJson(String json) { + VariantBuilder builder = new VariantBuilder(); + JsonParser parser = new JsonParser(json, builder); + // Pass 1: parse JSON to collect all field names encountered at every nesting level. + // The resulting 'value' bytes are discarded; only 'builder.fieldNames' is used. + parser.parseValue(); + + // Pass 2: pre-register field names in sorted order so that during re-encoding every + // addFieldName() call returns the stable, sorted field-ID that will match the metadata + // dictionary. Without this step, field IDs in the value bytes would reflect insertion + // order, which may differ from the sorted metadata dictionary produced by buildMetadata(). + VariantBuilder finalBuilder = new VariantBuilder(); + // Pre-register all names in sorted order + List sortedNames = new ArrayList<>(builder.fieldNames); + Collections.sort(sortedNames); + for (String name : sortedNames) { + finalBuilder.addFieldName(name); + } + byte[] finalMetadata = finalBuilder.buildMetadata(); + + JsonParser finalParser = new JsonParser(json, finalBuilder); + byte[] finalValue = finalParser.parseValue(); + + return new Variant(finalMetadata, finalValue); + } + + // -------------------------------------------------------------------------------------------- + // Simple recursive-descent JSON parser + // -------------------------------------------------------------------------------------------- + + /** A minimal recursive-descent JSON parser that produces Variant-encoded binary values. */ + private static class JsonParser { + private final String json; + private final VariantBuilder builder; + private int pos; + + JsonParser(String json, VariantBuilder builder) { + this.json = json; + this.builder = builder; + this.pos = 0; + } + + byte[] parseValue() { + skipWhitespace(); + if (pos >= json.length()) { + return VariantUtil.encodeNull(); + } + char c = json.charAt(pos); + switch (c) { + case '"': + return parseString(); + case '{': + return parseObject(); + case '[': + return parseArray(); + case 't': + case 'f': + return parseBoolean(); + case 'n': + return parseNull(); + default: + if (c == '-' || (c >= '0' && c <= '9')) { + return parseNumber(); + } + throw new IllegalArgumentException( + "Unexpected character '" + c + "' at position " + pos); + } + } + + private byte[] parseString() { + String str = parseStringValue(); + byte[] strBytes = str.getBytes(StandardCharsets.UTF_8); + return VariantUtil.encodeString(strBytes); + } + + private String parseStringValue() { + expect('"'); + StringBuilder sb = new StringBuilder(); + while (pos < json.length()) { + char c = json.charAt(pos++); + if (c == '"') { + return sb.toString(); + } + if (c == '\\') { + if (pos >= json.length()) { + throw new IllegalArgumentException("Unexpected end of string"); + } + char escaped = json.charAt(pos++); + switch (escaped) { + case '"': + sb.append('"'); + break; + case '\\': + sb.append('\\'); + break; + case '/': + sb.append('/'); + break; + case 'b': + sb.append('\b'); + break; + case 'f': + sb.append('\f'); + break; + case 'n': + sb.append('\n'); + break; + case 'r': + sb.append('\r'); + break; + case 't': + sb.append('\t'); + break; + case 'u': + if (pos + 4 > json.length()) { + throw new IllegalArgumentException("Invalid unicode escape"); + } + int codeUnit = Integer.parseInt(json.substring(pos, pos + 4), 16); + pos += 4; + // Handle surrogate pairs: a high surrogate must be followed by + // a low surrogate escape to form a valid supplementary code point + // (e.g. emoji encoded as \\uD83D\\uDE00 -> U+1F600). + if (Character.isHighSurrogate((char) codeUnit) + && pos + 6 <= json.length() + && json.charAt(pos) == '\\' + && json.charAt(pos + 1) == 'u') { + int low = Integer.parseInt(json.substring(pos + 2, pos + 6), 16); + if (Character.isLowSurrogate((char) low)) { + sb.appendCodePoint( + Character.toCodePoint((char) codeUnit, (char) low)); + pos += 6; + break; + } + } + sb.append((char) codeUnit); + break; + default: + throw new IllegalArgumentException("Invalid escape: \\" + escaped); + } + } else { + sb.append(c); + } + } + throw new IllegalArgumentException("Unterminated string"); + } + + private byte[] parseObject() { + expect('{'); + skipWhitespace(); + + List fieldIds = new ArrayList<>(); + List fieldValues = new ArrayList<>(); + + if (pos < json.length() && json.charAt(pos) != '}') { + parseObjectField(fieldIds, fieldValues); + skipWhitespace(); + while (pos < json.length() && json.charAt(pos) == ',') { + pos++; // skip ',' + parseObjectField(fieldIds, fieldValues); + skipWhitespace(); + } + } + expect('}'); + + return VariantUtil.encodeObject(fieldIds, fieldValues); + } + + private void parseObjectField(List fieldIds, List fieldValues) { + skipWhitespace(); + String fieldName = parseStringValue(); + skipWhitespace(); + expect(':'); + byte[] value = parseValue(); + int fieldId = builder.addFieldName(fieldName); + fieldIds.add(fieldId); + fieldValues.add(value); + } + + private byte[] parseArray() { + expect('['); + skipWhitespace(); + + List elements = new ArrayList<>(); + if (pos < json.length() && json.charAt(pos) != ']') { + elements.add(parseValue()); + skipWhitespace(); + while (pos < json.length() && json.charAt(pos) == ',') { + pos++; // skip ',' + elements.add(parseValue()); + skipWhitespace(); + } + } + expect(']'); + + return encodeArray(elements); + } + + private byte[] parseBoolean() { + if (json.startsWith("true", pos)) { + pos += 4; + return VariantUtil.encodeBoolean(true); + } else if (json.startsWith("false", pos)) { + pos += 5; + return VariantUtil.encodeBoolean(false); + } + throw new IllegalArgumentException("Expected boolean at position " + pos); + } + + private byte[] parseNull() { + if (json.startsWith("null", pos)) { + pos += 4; + return VariantUtil.encodeNull(); + } + throw new IllegalArgumentException("Expected null at position " + pos); + } + + private byte[] parseNumber() { + int start = pos; + boolean hasDecimalPoint = false; + boolean hasExponent = false; + + if (pos < json.length() && json.charAt(pos) == '-') { + pos++; + } + while (pos < json.length() && json.charAt(pos) >= '0' && json.charAt(pos) <= '9') { + pos++; + } + if (pos < json.length() && json.charAt(pos) == '.') { + hasDecimalPoint = true; + pos++; + while (pos < json.length() && json.charAt(pos) >= '0' && json.charAt(pos) <= '9') { + pos++; + } + } + if (pos < json.length() && (json.charAt(pos) == 'e' || json.charAt(pos) == 'E')) { + hasExponent = true; + pos++; + if (pos < json.length() && (json.charAt(pos) == '+' || json.charAt(pos) == '-')) { + pos++; + } + while (pos < json.length() && json.charAt(pos) >= '0' && json.charAt(pos) <= '9') { + pos++; + } + } + + String numStr = json.substring(start, pos); + if (hasDecimalPoint && !hasExponent) { + // Decimal point without exponent -> encode as Decimal (matching Paimon behavior) + BigDecimal decimal = new BigDecimal(numStr); + return VariantUtil.encodeDecimal(decimal); + } else if (hasExponent) { + // Has exponent -> encode as Double + double d = Double.parseDouble(numStr); + return VariantUtil.encodeDouble(d); + } else { + try { + long val = Long.parseLong(numStr); + if (val >= Byte.MIN_VALUE && val <= Byte.MAX_VALUE) { + return VariantUtil.encodeByte((byte) val); + } else if (val >= Short.MIN_VALUE && val <= Short.MAX_VALUE) { + return VariantUtil.encodeShort((short) val); + } else if (val >= Integer.MIN_VALUE && val <= Integer.MAX_VALUE) { + return VariantUtil.encodeInt((int) val); + } else { + return VariantUtil.encodeLong(val); + } + } catch (NumberFormatException e) { + // Fallback to double for very large numbers + return VariantUtil.encodeDouble(Double.parseDouble(numStr)); + } + } + } + + private void skipWhitespace() { + while (pos < json.length() && Character.isWhitespace(json.charAt(pos))) { + pos++; + } + } + + private void expect(char expected) { + skipWhitespace(); + if (pos >= json.length() || json.charAt(pos) != expected) { + throw new IllegalArgumentException( + "Expected '" + + expected + + "' at position " + + pos + + " but got '" + + (pos < json.length() ? json.charAt(pos) : "EOF") + + "'"); + } + pos++; + skipWhitespace(); + } + + private byte[] encodeArray(List elements) { + int n = elements.size(); + + // Calculate total data size + int totalDataSize = 0; + for (byte[] e : elements) { + totalDataSize += e.length; + } + + // Fixed 4-byte format: header(1) + numElements(4) + offsets(4*(n+1)) + data + int totalSize = 1 + 4 + 4 * (n + 1) + totalDataSize; + byte[] result = new byte[totalSize]; + + // Header: ARRAY with offset_size=4 (3<<2), is_large=true (1<<6) + result[0] = (byte) (VariantUtil.BASIC_TYPE_ARRAY | (3 << 2) | (1 << 6)); + VariantUtil.writeIntLE(result, 1, n); + + int writePos = 5; + + // Offsets (n+1 entries, last = totalDataSize) + int dataOffset = 0; + for (int i = 0; i < n; i++) { + VariantUtil.writeIntLE(result, writePos, dataOffset); + writePos += 4; + dataOffset += elements.get(i).length; + } + VariantUtil.writeIntLE(result, writePos, totalDataSize); + writePos += 4; + + // Element data + for (int i = 0; i < n; i++) { + byte[] e = elements.get(i); + System.arraycopy(e, 0, result, writePos, e.length); + writePos += e.length; + } + + return result; + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantStatisticsCollector.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantStatisticsCollector.java new file mode 100644 index 0000000000..3fcbbb4f82 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantStatisticsCollector.java @@ -0,0 +1,234 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.types.DataTypeRoot; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Collects statistics about fields within Variant records. Analyzes each Variant value to track + * field presence and type distribution, which is used by {@link ShreddingSchemaInferrer} to + * determine optimal shredding strategies. + * + *

This collector supports: + * + *

    + *
  • Single record analysis via {@link #collect(Variant)} + *
  • Batch analysis via {@link #collectBatch(List)} + *
  • Distributed merging via {@link #merge(VariantStatisticsCollector)} + *
+ * + * @since 0.7 + */ +@Internal +public class VariantStatisticsCollector { + + /** Per-field statistics, keyed by field path. */ + private final Map fieldStats; + + /** Total number of records analyzed. */ + private long totalRecords; + + public VariantStatisticsCollector() { + this.fieldStats = new HashMap<>(); + this.totalRecords = 0; + } + + /** + * Analyzes a single Variant record and updates field statistics. + * + *

Only top-level object fields are analyzed in the current implementation. Nested objects + * and arrays are not recursively traversed. + * + * @param variant the Variant value to analyze + */ + public void collect(Variant variant) { + totalRecords++; + if (variant == null || variant.isNull()) { + return; + } + if (!variant.isObject()) { + // Only analyze object-type variants for shredding + return; + } + collectObjectFields(variant, ""); + } + + /** + * Analyzes a batch of Variant records. + * + * @param variants the list of Variant values to analyze + */ + public void collectBatch(List variants) { + for (Variant variant : variants) { + collect(variant); + } + } + + /** + * Merges statistics from another collector into this one. Used for distributed scenarios where + * different partitions/buckets collect statistics independently. + * + * @param other the other collector to merge + */ + public void merge(VariantStatisticsCollector other) { + this.totalRecords += other.totalRecords; + for (Map.Entry entry : other.fieldStats.entrySet()) { + FieldStatistics existing = this.fieldStats.get(entry.getKey()); + if (existing != null) { + existing.merge(entry.getValue()); + } else { + this.fieldStats.put(entry.getKey(), new FieldStatistics(entry.getValue())); + } + } + } + + /** Returns the total number of records analyzed. */ + public long getTotalRecords() { + return totalRecords; + } + + /** Returns an unmodifiable view of the current field statistics. */ + public Map getStatistics() { + return Collections.unmodifiableMap(fieldStats); + } + + /** Resets the collector to its initial state. */ + public void reset() { + fieldStats.clear(); + totalRecords = 0; + } + + // -------------------------------------------------------------------------------------------- + // Internal methods + // -------------------------------------------------------------------------------------------- + + /** + * Recursively collects field statistics from an object variant. + * + * @param variant the object variant to analyze + * @param prefix the field path prefix (empty string for top-level) + */ + private void collectObjectFields(Variant variant, String prefix) { + // Iterate over all fields in the object using metadata + byte[] metadata = variant.metadata(); + byte[] value = variant.value(); + + int numFields = variant.objectSize(); + for (int i = 0; i < numFields; i++) { + // Get field name from metadata using the field ID stored in the object + int fieldId = VariantUtil.objectFieldId(value, 0, i); + String fieldName = VariantUtil.metadataFieldName(metadata, fieldId); + String fieldPath = prefix.isEmpty() ? fieldName : prefix + "." + fieldName; + + // Get the field value + int fieldValueOffset = VariantUtil.objectFieldValueOffset(value, 0, i); + int basicType = VariantUtil.basicType(value, fieldValueOffset); + + DataTypeRoot typeRoot = mapToDataTypeRoot(value, fieldValueOffset, basicType); + if (typeRoot != null) { + getOrCreateStats(fieldPath).record(typeRoot); + } + } + } + + /** + * Maps a Variant value's type information to a Fluss DataTypeRoot. + * + * @param value the value bytes + * @param offset the offset to the value + * @param basicType the basic type (0-3) + * @return the corresponding DataTypeRoot, or null if not mappable + */ + private static DataTypeRoot mapToDataTypeRoot(byte[] value, int offset, int basicType) { + switch (basicType) { + case VariantUtil.BASIC_TYPE_PRIMITIVE: + return mapPrimitiveType(value, offset); + case VariantUtil.BASIC_TYPE_SHORT_STRING: + return DataTypeRoot.STRING; + case VariantUtil.BASIC_TYPE_OBJECT: + // Nested objects are not shredded in v1 + return null; + case VariantUtil.BASIC_TYPE_ARRAY: + // Arrays are not shredded in v1 + return null; + default: + return null; + } + } + + /** + * Maps a Variant primitive type to a Fluss DataTypeRoot. + * + *

All integer sub-types (INT8, INT16, INT32, INT64) are mapped to {@link + * DataTypeRoot#BIGINT} to avoid false type-inconsistency when a field alternates between small + * integers (encoded as INT8) and larger values (encoded as INT32 or INT64). This mirrors + * Paimon's approach of widening all integers to a common type before shredding. + * + * @param value the value bytes + * @param offset the offset to the value + * @return the corresponding DataTypeRoot, or null for unsupported or null types + */ + private static DataTypeRoot mapPrimitiveType(byte[] value, int offset) { + int primitiveTypeId = VariantUtil.primitiveTypeId(value, offset); + switch (primitiveTypeId) { + case VariantUtil.PRIMITIVE_TYPE_NULL: + return null; // null values don't contribute to type inference + case VariantUtil.PRIMITIVE_TYPE_TRUE: + case VariantUtil.PRIMITIVE_TYPE_FALSE: + return DataTypeRoot.BOOLEAN; + case VariantUtil.PRIMITIVE_TYPE_INT8: + case VariantUtil.PRIMITIVE_TYPE_INT16: + case VariantUtil.PRIMITIVE_TYPE_INT32: + case VariantUtil.PRIMITIVE_TYPE_INT64: + // Widen all integer sub-types to BIGINT to avoid spurious type-inconsistency. + // A field that appears as INT8 in some rows and INT16/INT32 in others should still + // be considered a consistent integer field and shredded as BIGINT. + return DataTypeRoot.BIGINT; + case VariantUtil.PRIMITIVE_TYPE_FLOAT: + return DataTypeRoot.FLOAT; + case VariantUtil.PRIMITIVE_TYPE_DOUBLE: + return DataTypeRoot.DOUBLE; + case VariantUtil.PRIMITIVE_TYPE_STRING: + return DataTypeRoot.STRING; + case VariantUtil.PRIMITIVE_TYPE_DATE: + return DataTypeRoot.DATE; + case VariantUtil.PRIMITIVE_TYPE_TIMESTAMP: + return DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE; + case VariantUtil.PRIMITIVE_TYPE_TIMESTAMP_NTZ: + return DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE; + case VariantUtil.PRIMITIVE_TYPE_DECIMAL4: + case VariantUtil.PRIMITIVE_TYPE_DECIMAL8: + case VariantUtil.PRIMITIVE_TYPE_DECIMAL16: + return DataTypeRoot.DECIMAL; + case VariantUtil.PRIMITIVE_TYPE_BINARY: + return DataTypeRoot.BYTES; + default: + return null; + } + } + + private FieldStatistics getOrCreateStats(String fieldPath) { + return fieldStats.computeIfAbsent(fieldPath, FieldStatistics::new); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantUtil.java b/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantUtil.java new file mode 100644 index 0000000000..d360b439af --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/types/variant/VariantUtil.java @@ -0,0 +1,1088 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.annotation.Internal; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; + +/** + * Low-level utility methods for Variant binary encoding/decoding following the Parquet Variant + * Binary Encoding specification. + * + *

Metadata format

+ * + *
+ * [header byte] [dictionary_size (offset_size bytes LE)] [offsets (offset_size bytes each LE)] [string bytes]
+ *
+ * header: bits 0-3 = version (must be 1), bit 4 = sorted_strings flag, bits 6-7 = offset_size_minus_1
+ * offset_size: (header bits 6-7) + 1, giving 1-4 bytes per offset
+ * dictionary_size: number of strings in the dictionary
+ * offsets: (dictionary_size + 1) offsets into the string bytes area
+ * 
+ * + *

Value format

+ * + *
+ * [value_header byte] [value_data...]
+ *
+ * value_header bits 0-1 = basic_type:
+ *   0 = Primitive (bits 2-6 encode primitive type ID)
+ *   1 = ShortString (bits 2-6 encode string length)
+ *   2 = Object
+ *   3 = Array
+ * 
+ */ +@Internal +public final class VariantUtil { + + private VariantUtil() {} + + // -------------------------------------------------------------------------------------------- + // Constants: Basic Types (2-bit, stored in value_header bits 0-1) + // -------------------------------------------------------------------------------------------- + + public static final int BASIC_TYPE_PRIMITIVE = 0; + public static final int BASIC_TYPE_SHORT_STRING = 1; + public static final int BASIC_TYPE_OBJECT = 2; + public static final int BASIC_TYPE_ARRAY = 3; + public static final int BASIC_TYPE_MASK = 0x03; + + // -------------------------------------------------------------------------------------------- + // Constants: Primitive Type IDs (5-bit, stored in value_header bits 2-6) + // -------------------------------------------------------------------------------------------- + + public static final int PRIMITIVE_TYPE_NULL = 0; + public static final int PRIMITIVE_TYPE_TRUE = 1; + public static final int PRIMITIVE_TYPE_FALSE = 2; + public static final int PRIMITIVE_TYPE_INT8 = 3; + public static final int PRIMITIVE_TYPE_INT16 = 4; + public static final int PRIMITIVE_TYPE_INT32 = 5; + public static final int PRIMITIVE_TYPE_INT64 = 6; + public static final int PRIMITIVE_TYPE_DOUBLE = 7; + public static final int PRIMITIVE_TYPE_DECIMAL4 = 8; + public static final int PRIMITIVE_TYPE_DECIMAL8 = 9; + public static final int PRIMITIVE_TYPE_DECIMAL16 = 10; + public static final int PRIMITIVE_TYPE_DATE = 11; + public static final int PRIMITIVE_TYPE_TIMESTAMP = 12; + public static final int PRIMITIVE_TYPE_TIMESTAMP_NTZ = 13; + public static final int PRIMITIVE_TYPE_FLOAT = 14; + public static final int PRIMITIVE_TYPE_BINARY = 15; + public static final int PRIMITIVE_TYPE_STRING = 16; + + // -------------------------------------------------------------------------------------------- + // Constants: Metadata + // -------------------------------------------------------------------------------------------- + + public static final int METADATA_VERSION = 1; + public static final int METADATA_VERSION_MASK = 0x0F; + public static final int METADATA_SORTED_STRINGS_BIT = 0x10; + + /** + * Bits 6-7 of the metadata header encode {@code offset_size_minus_1}. Fluss always writes + * 4-byte offsets (offset_size=4, so offset_size_minus_1=3). Shifting 3 left by 6 positions + * yields {@code 0xC0}. Setting these bits makes the metadata header + * Parquet-Variant-spec-compliant so that external systems (e.g. Flink's {@code BinaryVariant}) + * can correctly interpret the variable-width offset encoding. + */ + public static final int METADATA_OFFSET_SIZE_BITS = 3 << 6; // 0xC0 + + /** Empty metadata: version 1, 0 dictionary entries. */ + public static final byte[] EMPTY_METADATA; + + static { + ByteBuffer buf = ByteBuffer.allocate(5).order(ByteOrder.LITTLE_ENDIAN); + buf.put( + (byte) + (METADATA_VERSION + | METADATA_SORTED_STRINGS_BIT + | METADATA_OFFSET_SIZE_BITS)); + buf.putInt(0); + EMPTY_METADATA = buf.array(); + } + + // -------------------------------------------------------------------------------------------- + // Value header decoding + // -------------------------------------------------------------------------------------------- + + /** Returns the basic type (0-3) from the value bytes at the given offset. */ + public static int basicType(byte[] value, int offset) { + return (value[offset] & 0xFF) & BASIC_TYPE_MASK; + } + + /** Returns the primitive type ID from the value header (6-bit type_info field). */ + public static int primitiveTypeId(byte[] value, int offset) { + return ((value[offset] & 0xFF) >> 2) & 0x3F; + } + + /** Returns the short string length from the value header (6-bit type_info field, max 63). */ + public static int shortStringLength(byte[] value, int offset) { + return ((value[offset] & 0xFF) >> 2) & 0x3F; + } + + /** Returns true if the value at the given offset represents null. */ + public static boolean isNull(byte[] value, int offset) { + return basicType(value, offset) == BASIC_TYPE_PRIMITIVE + && primitiveTypeId(value, offset) == PRIMITIVE_TYPE_NULL; + } + + // -------------------------------------------------------------------------------------------- + // Primitive value reading + // -------------------------------------------------------------------------------------------- + + /** Reads a boolean from the value bytes. */ + public static boolean getBoolean(byte[] value, int offset) { + int typeId = primitiveTypeId(value, offset); + if (typeId == PRIMITIVE_TYPE_TRUE) { + return true; + } else if (typeId == PRIMITIVE_TYPE_FALSE) { + return false; + } + throw new IllegalStateException("Not a boolean variant value"); + } + + /** Reads an int8 from the value bytes. */ + public static byte getByte(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_INT8); + return value[offset + 1]; + } + + /** Reads an int16 from the value bytes. */ + public static short getShort(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_INT16); + return readShortLE(value, offset + 1); + } + + /** Reads an int32 from the value bytes. */ + public static int getInt(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_INT32); + return readIntLE(value, offset + 1); + } + + /** Reads an int64 from the value bytes. */ + public static long getLong(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_INT64); + return readLongLE(value, offset + 1); + } + + /** Reads a float from the value bytes. */ + public static float getFloat(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_FLOAT); + return Float.intBitsToFloat(readIntLE(value, offset + 1)); + } + + /** Reads a double from the value bytes. */ + public static double getDouble(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_DOUBLE); + return Double.longBitsToDouble(readLongLE(value, offset + 1)); + } + + /** Reads a string from the value bytes (handles both short string and primitive string). */ + public static String getString(byte[] value, int offset) { + int basic = basicType(value, offset); + if (basic == BASIC_TYPE_SHORT_STRING) { + int len = shortStringLength(value, offset); + return new String(value, offset + 1, len, StandardCharsets.UTF_8); + } else if (basic == BASIC_TYPE_PRIMITIVE) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_STRING); + int len = readIntLE(value, offset + 1); + return new String(value, offset + 5, len, StandardCharsets.UTF_8); + } + throw new IllegalStateException("Not a string variant value"); + } + + /** Reads binary data from the value bytes. */ + public static byte[] getBinary(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_BINARY); + int len = readIntLE(value, offset + 1); + return Arrays.copyOfRange(value, offset + 5, offset + 5 + len); + } + + // -------------------------------------------------------------------------------------------- + // Decimal value reading/writing + // -------------------------------------------------------------------------------------------- + + /** + * Reads a decimal value from the value bytes. + * + *

Decimal encoding: [header(1)][scale(1)][mantissa(4/8/16 bytes little-endian)] + * + * @return the BigDecimal value + */ + public static BigDecimal getDecimal(byte[] value, int offset) { + int typeId = primitiveTypeId(value, offset); + int scale = value[offset + 1] & 0xFF; + int mantissaOffset = offset + 2; + BigInteger unscaled; + switch (typeId) { + case PRIMITIVE_TYPE_DECIMAL4: + unscaled = BigInteger.valueOf(readIntLE(value, mantissaOffset)); + break; + case PRIMITIVE_TYPE_DECIMAL8: + unscaled = BigInteger.valueOf(readLongLE(value, mantissaOffset)); + break; + case PRIMITIVE_TYPE_DECIMAL16: + // 16-byte little-endian to big-endian for BigInteger + byte[] be = new byte[16]; + for (int i = 0; i < 16; i++) { + be[15 - i] = value[mantissaOffset + i]; + } + unscaled = new BigInteger(be); + break; + default: + throw new IllegalStateException("Not a decimal variant value, type: " + typeId); + } + return new BigDecimal(unscaled, scale); + } + + /** + * Encodes a decimal value. Automatically selects DECIMAL4, DECIMAL8, or DECIMAL16 based on the + * unscaled value magnitude. + */ + public static byte[] encodeDecimal(BigDecimal value) { + int scale = value.scale(); + BigInteger unscaled = value.unscaledValue(); + int bitLen = unscaled.bitLength(); // sign bit not counted + + if (bitLen < 32) { + // DECIMAL4: header(1) + scale(1) + 4 bytes + byte[] result = new byte[6]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_DECIMAL4); + result[1] = (byte) scale; + writeIntLE(result, 2, unscaled.intValue()); + return result; + } else if (bitLen < 64) { + // DECIMAL8: header(1) + scale(1) + 8 bytes + byte[] result = new byte[10]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_DECIMAL8); + result[1] = (byte) scale; + writeLongLE(result, 2, unscaled.longValue()); + return result; + } else { + // DECIMAL16: header(1) + scale(1) + 16 bytes + byte[] result = new byte[18]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_DECIMAL16); + result[1] = (byte) scale; + // Big-endian BigInteger to 16-byte little-endian + byte[] be = unscaled.toByteArray(); + // Sign-extend to 16 bytes and reverse to little-endian + byte fill = (byte) (unscaled.signum() < 0 ? 0xFF : 0x00); + Arrays.fill(result, 2, 18, fill); + for (int i = 0; i < be.length; i++) { + result[2 + (be.length - 1 - i)] = be[i]; + } + return result; + } + } + + // -------------------------------------------------------------------------------------------- + // Timestamp value reading/writing + // -------------------------------------------------------------------------------------------- + + /** + * Reads a timestamp value (microseconds since epoch) from the value bytes. + * + * @return microseconds since epoch + */ + public static long getTimestamp(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_TIMESTAMP); + return readLongLE(value, offset + 1); + } + + /** + * Reads a timestamp without timezone value (microseconds since epoch) from the value bytes. + * + * @return microseconds since epoch + */ + public static long getTimestampNtz(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_TIMESTAMP_NTZ); + return readLongLE(value, offset + 1); + } + + /** Encodes a timestamp value (microseconds since epoch). */ + public static byte[] encodeTimestamp(long microsSinceEpoch) { + byte[] result = new byte[9]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_TIMESTAMP); + writeLongLE(result, 1, microsSinceEpoch); + return result; + } + + /** Encodes a timestamp without timezone value (microseconds since epoch). */ + public static byte[] encodeTimestampNtz(long microsSinceEpoch) { + byte[] result = new byte[9]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_TIMESTAMP_NTZ); + writeLongLE(result, 1, microsSinceEpoch); + return result; + } + + // -------------------------------------------------------------------------------------------- + // Object field access + // -------------------------------------------------------------------------------------------- + + /** + * Returns the number of fields in an object value. + * + *

Object encoding: [header(1)] [num_fields(4)] [field_id_list] [field_offset_list] + * [field_values] + */ + public static int objectSize(byte[] value, int offset) { + if (basicType(value, offset) != BASIC_TYPE_OBJECT) { + throw new IllegalStateException("Not an object variant value"); + } + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + return readUnsignedLE(value, offset + 1, sizeBytes); + } + + /** + * Finds the field ID for a given field name in the metadata dictionary. + * + * @return the field ID, or -1 if not found + */ + public static int findFieldId(byte[] metadata, String fieldName) { + int dictSize = metadataDictSize(metadata); + boolean sorted = isMetadataSorted(metadata); + byte[] nameBytes = fieldName.getBytes(StandardCharsets.UTF_8); + + if (sorted) { + return binarySearchFieldName(metadata, dictSize, nameBytes); + } else { + return linearSearchFieldName(metadata, dictSize, nameBytes); + } + } + + /** + * Finds the value offset for a given field ID in an object value. Uses binary search since + * field IDs are stored in sorted order by {@link #encodeObject}. + * + * @return the value offset within the value byte array, or -1 if field ID not found + */ + public static int findFieldValueOffset(byte[] value, int offset, int fieldId) { + if (basicType(value, offset) != BASIC_TYPE_OBJECT) { + return -1; + } + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + int numFields = readUnsignedLE(value, offset + 1, sizeBytes); + int idSize = ((typeInfo >> 2) & 0x3) + 1; + int offsetSize = (typeInfo & 0x3) + 1; + int idStart = offset + 1 + sizeBytes; + int offsetStart = idStart + numFields * idSize; + int dataStart = offsetStart + (numFields + 1) * offsetSize; + + // Binary search on sorted field IDs + int lo = 0, hi = numFields - 1; + while (lo <= hi) { + int mid = (lo + hi) >>> 1; + int midId = readUnsignedLE(value, idStart + mid * idSize, idSize); + if (midId < fieldId) { + lo = mid + 1; + } else if (midId > fieldId) { + hi = mid - 1; + } else { + int fieldOffset = readUnsignedLE(value, offsetStart + mid * offsetSize, offsetSize); + return dataStart + fieldOffset; + } + } + return -1; + } + + /** + * Returns the field ID (metadata dictionary index) of the i-th field in an object value. + * + * @param value the value bytes + * @param offset the offset to the object value + * @param fieldIndex the field index (0-based) + * @return the field ID (metadata dictionary index) + */ + public static int objectFieldId(byte[] value, int offset, int fieldIndex) { + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + int idSize = ((typeInfo >> 2) & 0x3) + 1; + int idStart = offset + 1 + sizeBytes; + return readUnsignedLE(value, idStart + fieldIndex * idSize, idSize); + } + + /** + * Returns the value offset of the i-th field in an object value. + * + * @param value the value bytes + * @param offset the offset to the object value + * @param fieldIndex the field index (0-based) + * @return the absolute offset of the field value within the value byte array + */ + public static int objectFieldValueOffset(byte[] value, int offset, int fieldIndex) { + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + int numFields = readUnsignedLE(value, offset + 1, sizeBytes); + int idSize = ((typeInfo >> 2) & 0x3) + 1; + int offsetSize = (typeInfo & 0x3) + 1; + int idStart = offset + 1 + sizeBytes; + int offsetStart = idStart + numFields * idSize; + int dataStart = offsetStart + (numFields + 1) * offsetSize; + int fieldOffset = readUnsignedLE(value, offsetStart + fieldIndex * offsetSize, offsetSize); + return dataStart + fieldOffset; + } + + // -------------------------------------------------------------------------------------------- + // Array element access + // -------------------------------------------------------------------------------------------- + + /** + * Returns the number of elements in an array value. + * + *

Array encoding: [header(1)] [num_elements(4)] [offset_list] [element_values] + */ + public static int arraySize(byte[] value, int offset) { + if (basicType(value, offset) != BASIC_TYPE_ARRAY) { + throw new IllegalStateException("Not an array variant value"); + } + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + return readUnsignedLE(value, offset + 1, sizeBytes); + } + + public static int arrayElementOffset(byte[] value, int offset, int index) { + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + int numElements = readUnsignedLE(value, offset + 1, sizeBytes); + if (index < 0 || index >= numElements) { + throw new IndexOutOfBoundsException( + "Array index " + index + " out of bounds [0, " + numElements + ")"); + } + int offsetSize = (typeInfo & 0x3) + 1; + int offsetListStart = offset + 1 + sizeBytes; + int elementValuesStart = offsetListStart + (numElements + 1) * offsetSize; + int elementOffset = readUnsignedLE(value, offsetListStart + index * offsetSize, offsetSize); + return elementValuesStart + elementOffset; + } + + // -------------------------------------------------------------------------------------------- + // Value size calculation + // -------------------------------------------------------------------------------------------- + + /** Computes the total byte size of a value starting at the given offset. */ + public static int valueSize(byte[] value, int offset) { + int basic = basicType(value, offset); + switch (basic) { + case BASIC_TYPE_PRIMITIVE: + return primitiveValueSize(value, offset); + case BASIC_TYPE_SHORT_STRING: + return 1 + shortStringLength(value, offset); + case BASIC_TYPE_OBJECT: + return objectValueSize(value, offset); + case BASIC_TYPE_ARRAY: + return arrayValueSize(value, offset); + default: + throw new IllegalStateException("Unknown basic type: " + basic); + } + } + + private static int primitiveValueSize(byte[] value, int offset) { + int typeId = primitiveTypeId(value, offset); + switch (typeId) { + case PRIMITIVE_TYPE_NULL: + case PRIMITIVE_TYPE_TRUE: + case PRIMITIVE_TYPE_FALSE: + return 1; // header only + case PRIMITIVE_TYPE_INT8: + return 2; // header + 1 byte + case PRIMITIVE_TYPE_INT16: + return 3; // header + 2 bytes + case PRIMITIVE_TYPE_INT32: + case PRIMITIVE_TYPE_DATE: + case PRIMITIVE_TYPE_FLOAT: + return 5; // header + 4 bytes + case PRIMITIVE_TYPE_INT64: + case PRIMITIVE_TYPE_DOUBLE: + case PRIMITIVE_TYPE_TIMESTAMP: + case PRIMITIVE_TYPE_TIMESTAMP_NTZ: + return 9; // header + 8 bytes + case PRIMITIVE_TYPE_DECIMAL4: + return 6; // header(1) + scale(1) + 4 bytes mantissa + case PRIMITIVE_TYPE_DECIMAL8: + return 10; // header(1) + scale(1) + 8 bytes mantissa + case PRIMITIVE_TYPE_DECIMAL16: + return 18; // header(1) + scale(1) + 16 bytes mantissa + case PRIMITIVE_TYPE_BINARY: + case PRIMITIVE_TYPE_STRING: + { + int len = readIntLE(value, offset + 1); + return 5 + len; // header + 4-byte length + data + } + default: + throw new IllegalStateException("Unknown primitive type: " + typeId); + } + } + + private static int objectValueSize(byte[] value, int offset) { + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + int numFields = readUnsignedLE(value, offset + 1, sizeBytes); + int idSize = ((typeInfo >> 2) & 0x3) + 1; + int offsetSize = (typeInfo & 0x3) + 1; + int idStart = offset + 1 + sizeBytes; + int offsetStart = idStart + numFields * idSize; + int totalValueSize = + readUnsignedLE(value, offsetStart + numFields * offsetSize, offsetSize); + int dataStart = offsetStart + (numFields + 1) * offsetSize; + return (dataStart - offset) + totalValueSize; + } + + private static int arrayValueSize(byte[] value, int offset) { + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + int numElements = readUnsignedLE(value, offset + 1, sizeBytes); + int offsetSize = (typeInfo & 0x3) + 1; + int offsetListStart = offset + 1 + sizeBytes; + int totalValueSize = + readUnsignedLE(value, offsetListStart + numElements * offsetSize, offsetSize); + int dataStart = offsetListStart + (numElements + 1) * offsetSize; + return (dataStart - offset) + totalValueSize; + } + + // -------------------------------------------------------------------------------------------- + // Metadata helpers + // -------------------------------------------------------------------------------------------- + + /** Returns the number of strings in the metadata dictionary. */ + public static int metadataDictSize(byte[] metadata) { + int offsetSize = metadataOffsetSize(metadata); + return readUnsignedLE(metadata, 1, offsetSize); + } + + /** Returns true if the metadata dictionary strings are sorted. */ + public static boolean isMetadataSorted(byte[] metadata) { + return (metadata[0] & METADATA_SORTED_STRINGS_BIT) != 0; + } + + /** Returns the field name at the given dictionary index. */ + public static String metadataFieldName(byte[] metadata, int index) { + int offsetSize = metadataOffsetSize(metadata); + int dictSize = readUnsignedLE(metadata, 1, offsetSize); + if (index < 0 || index >= dictSize) { + throw new IndexOutOfBoundsException( + "Dict index " + index + " out of bounds [0, " + dictSize + ")"); + } + int offsetsStart = 1 + offsetSize; + int strStart = readUnsignedLE(metadata, offsetsStart + index * offsetSize, offsetSize); + int strEnd = readUnsignedLE(metadata, offsetsStart + (index + 1) * offsetSize, offsetSize); + int strBytesStart = offsetsStart + (dictSize + 1) * offsetSize; + return new String( + metadata, strBytesStart + strStart, strEnd - strStart, StandardCharsets.UTF_8); + } + + // -------------------------------------------------------------------------------------------- + // Object value encoding + // -------------------------------------------------------------------------------------------- + + /** + * Encodes an object value from field IDs and corresponding field value byte arrays. Fields are + * sorted by field ID in the output. + * + *

This implementation uses the "large" (4-byte) format for field ID size, offset size, and + * element count uniformly, for simplicity. All reader methods ({@link #objectSize}, {@link + * #objectFieldId}, {@link #objectFieldValueOffset}) are consistent with this choice. A future + * optimization could use variable-width encoding for smaller objects. + * + *

Object header typeInfo (6 bits): bits[1:0]=field_id_size-1, bits[3:2]=offset_size-1, + * bit[4]=is_large. With 4-byte widths: field_id_size=4 (bits[1:0]=3), offset_size=4 + * (bits[3:2]=3), is_large=1 (bit[4]=1) → typeInfo = 0b011111 = 0x1F. Header byte = (typeInfo + * << 2) | BASIC_TYPE_OBJECT = 0x7E. + * + * @param fieldIds the field IDs (metadata dictionary indices) for each field + * @param fieldValues the encoded Variant value bytes for each field + * @return the encoded object value bytes + */ + public static byte[] encodeObject(List fieldIds, List fieldValues) { + int n = fieldIds.size(); + // Object header: field_id_size=4 (3<<2), offset_size=4 (3<<4), is_large=1 (1<<6) + byte objectHeader = (byte) (BASIC_TYPE_OBJECT | (3 << 2) | (3 << 4) | (1 << 6)); + if (n == 0) { + // Empty object: header(1) + numFields(4) + lastOffset(4) + byte[] result = new byte[9]; + result[0] = objectHeader; + writeIntLE(result, 1, 0); + writeIntLE(result, 5, 0); + return result; + } + + // Sort fields by field ID + Integer[] indices = new Integer[n]; + for (int i = 0; i < n; i++) { + indices[i] = i; + } + Arrays.sort(indices, (a, b) -> Integer.compare(fieldIds.get(a), fieldIds.get(b))); + + // Calculate total field values size + int totalDataSize = 0; + for (byte[] v : fieldValues) { + totalDataSize += v.length; + } + + // Total: header(1) + numFields(4) + fieldIds(4*n) + offsets(4*(n+1)) + data + int totalSize = 1 + 4 + 4 * n + 4 * (n + 1) + totalDataSize; + byte[] result = new byte[totalSize]; + + result[0] = objectHeader; + writeIntLE(result, 1, n); + + int pos = 5; + + // Field IDs in sorted order + for (int i = 0; i < n; i++) { + writeIntLE(result, pos, fieldIds.get(indices[i])); + pos += 4; + } + + // Offsets (n+1 entries, last = totalDataSize) + int dataOffset = 0; + for (int i = 0; i < n; i++) { + writeIntLE(result, pos, dataOffset); + pos += 4; + dataOffset += fieldValues.get(indices[i]).length; + } + writeIntLE(result, pos, totalDataSize); + pos += 4; + + // Field value data in sorted field ID order + for (int i = 0; i < n; i++) { + byte[] v = fieldValues.get(indices[i]); + System.arraycopy(v, 0, result, pos, v.length); + pos += v.length; + } + + return result; + } + + // -------------------------------------------------------------------------------------------- + // Primitive value encoding + // -------------------------------------------------------------------------------------------- + + /** Encodes a null value. */ + public static byte[] encodeNull() { + return new byte[] {primitiveHeader(PRIMITIVE_TYPE_NULL)}; + } + + /** Encodes a boolean value. */ + public static byte[] encodeBoolean(boolean value) { + return new byte[] {primitiveHeader(value ? PRIMITIVE_TYPE_TRUE : PRIMITIVE_TYPE_FALSE)}; + } + + /** Encodes an int8 value. */ + public static byte[] encodeByte(byte value) { + return new byte[] {primitiveHeader(PRIMITIVE_TYPE_INT8), value}; + } + + /** Encodes an int16 value. */ + public static byte[] encodeShort(short value) { + byte[] result = new byte[3]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_INT16); + writeShortLE(result, 1, value); + return result; + } + + /** Encodes an int32 value. */ + public static byte[] encodeInt(int value) { + byte[] result = new byte[5]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_INT32); + writeIntLE(result, 1, value); + return result; + } + + /** Encodes an int64 value. */ + public static byte[] encodeLong(long value) { + byte[] result = new byte[9]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_INT64); + writeLongLE(result, 1, value); + return result; + } + + /** Encodes a float value. */ + public static byte[] encodeFloat(float value) { + byte[] result = new byte[5]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_FLOAT); + writeIntLE(result, 1, Float.floatToIntBits(value)); + return result; + } + + /** Encodes a double value. */ + public static byte[] encodeDouble(double value) { + byte[] result = new byte[9]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_DOUBLE); + writeLongLE(result, 1, Double.doubleToLongBits(value)); + return result; + } + + /** Encodes a date value (days since epoch). */ + public static byte[] encodeDate(int daysSinceEpoch) { + byte[] result = new byte[5]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_DATE); + writeIntLE(result, 1, daysSinceEpoch); + return result; + } + + /** Reads a date value (days since epoch) from the value bytes. */ + public static int getDate(byte[] value, int offset) { + checkPrimitiveType(value, offset, PRIMITIVE_TYPE_DATE); + return readIntLE(value, offset + 1); + } + + /** Encodes a string value (uses short string encoding if length <= 63). */ + public static byte[] encodeString(byte[] utf8Bytes) { + if (utf8Bytes.length <= 63) { + // Short string encoding + byte[] result = new byte[1 + utf8Bytes.length]; + result[0] = (byte) ((utf8Bytes.length << 2) | BASIC_TYPE_SHORT_STRING); + System.arraycopy(utf8Bytes, 0, result, 1, utf8Bytes.length); + return result; + } else { + // Long string encoding (primitive type STRING) + byte[] result = new byte[5 + utf8Bytes.length]; + result[0] = primitiveHeader(PRIMITIVE_TYPE_STRING); + writeIntLE(result, 1, utf8Bytes.length); + System.arraycopy(utf8Bytes, 0, result, 5, utf8Bytes.length); + return result; + } + } + + // -------------------------------------------------------------------------------------------- + // JSON serialization + // -------------------------------------------------------------------------------------------- + + /** Appends the JSON representation of the variant value to the StringBuilder. */ + public static void toJson(byte[] metadata, byte[] value, int offset, StringBuilder sb) { + int basic = basicType(value, offset); + switch (basic) { + case BASIC_TYPE_PRIMITIVE: + primitiveToJson(value, offset, sb); + break; + case BASIC_TYPE_SHORT_STRING: + sb.append('"'); + escapeJsonString(getString(value, offset), sb); + sb.append('"'); + break; + case BASIC_TYPE_OBJECT: + objectToJson(metadata, value, offset, sb); + break; + case BASIC_TYPE_ARRAY: + arrayToJson(metadata, value, offset, sb); + break; + default: + throw new IllegalStateException("Unknown basic type: " + basic); + } + } + + private static void primitiveToJson(byte[] value, int offset, StringBuilder sb) { + int typeId = primitiveTypeId(value, offset); + switch (typeId) { + case PRIMITIVE_TYPE_NULL: + sb.append("null"); + break; + case PRIMITIVE_TYPE_TRUE: + sb.append("true"); + break; + case PRIMITIVE_TYPE_FALSE: + sb.append("false"); + break; + case PRIMITIVE_TYPE_INT8: + sb.append(getByte(value, offset)); + break; + case PRIMITIVE_TYPE_INT16: + sb.append(getShort(value, offset)); + break; + case PRIMITIVE_TYPE_INT32: + sb.append(getInt(value, offset)); + break; + case PRIMITIVE_TYPE_DATE: + sb.append(getDate(value, offset)); + break; + case PRIMITIVE_TYPE_INT64: + sb.append(getLong(value, offset)); + break; + case PRIMITIVE_TYPE_TIMESTAMP: + sb.append(getTimestamp(value, offset)); + break; + case PRIMITIVE_TYPE_TIMESTAMP_NTZ: + sb.append(getTimestampNtz(value, offset)); + break; + case PRIMITIVE_TYPE_FLOAT: + sb.append(getFloat(value, offset)); + break; + case PRIMITIVE_TYPE_DOUBLE: + sb.append(getDouble(value, offset)); + break; + case PRIMITIVE_TYPE_STRING: + sb.append('"'); + escapeJsonString(getString(value, offset), sb); + sb.append('"'); + break; + case PRIMITIVE_TYPE_BINARY: + // Encode binary as base64 string + sb.append('"'); + sb.append(java.util.Base64.getEncoder().encodeToString(getBinary(value, offset))); + sb.append('"'); + break; + case PRIMITIVE_TYPE_DECIMAL4: + case PRIMITIVE_TYPE_DECIMAL8: + case PRIMITIVE_TYPE_DECIMAL16: + sb.append(getDecimal(value, offset).toPlainString()); + break; + default: + sb.append("\"\""); + } + } + + private static void objectToJson(byte[] metadata, byte[] value, int offset, StringBuilder sb) { + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + int numFields = readUnsignedLE(value, offset + 1, sizeBytes); + int idSize = ((typeInfo >> 2) & 0x3) + 1; + int offsetSize = (typeInfo & 0x3) + 1; + int idStart = offset + 1 + sizeBytes; + int offsetStart = idStart + numFields * idSize; + int dataStart = offsetStart + (numFields + 1) * offsetSize; + + sb.append('{'); + for (int i = 0; i < numFields; i++) { + if (i > 0) { + sb.append(','); + } + int fieldId = readUnsignedLE(value, idStart + i * idSize, idSize); + String fieldName = metadataFieldName(metadata, fieldId); + sb.append('"'); + escapeJsonString(fieldName, sb); + sb.append("\":"); + + int fieldOffset = readUnsignedLE(value, offsetStart + i * offsetSize, offsetSize); + toJson(metadata, value, dataStart + fieldOffset, sb); + } + sb.append('}'); + } + + private static void arrayToJson(byte[] metadata, byte[] value, int offset, StringBuilder sb) { + int typeInfo = ((value[offset] & 0xFF) >> 2) & 0x3F; + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = largeSize ? 4 : 1; + int numElements = readUnsignedLE(value, offset + 1, sizeBytes); + int offsetSize = (typeInfo & 0x3) + 1; + int offsetListStart = offset + 1 + sizeBytes; + int dataStart = offsetListStart + (numElements + 1) * offsetSize; + + sb.append('['); + for (int i = 0; i < numElements; i++) { + if (i > 0) { + sb.append(','); + } + int elementOffset = readUnsignedLE(value, offsetListStart + i * offsetSize, offsetSize); + toJson(metadata, value, dataStart + elementOffset, sb); + } + sb.append(']'); + } + + private static void escapeJsonString(String s, StringBuilder sb) { + for (int i = 0; i < s.length(); i++) { + char c = s.charAt(i); + switch (c) { + case '"': + sb.append("\\\""); + break; + case '\\': + sb.append("\\\\"); + break; + case '\b': + sb.append("\\b"); + break; + case '\f': + sb.append("\\f"); + break; + case '\n': + sb.append("\\n"); + break; + case '\r': + sb.append("\\r"); + break; + case '\t': + sb.append("\\t"); + break; + default: + if (c < 0x20) { + sb.append(String.format("\\u%04x", (int) c)); + } else { + sb.append(c); + } + } + } + } + + // -------------------------------------------------------------------------------------------- + // Internal helpers + // -------------------------------------------------------------------------------------------- + + static byte primitiveHeader(int primitiveTypeId) { + return (byte) ((primitiveTypeId << 2) | BASIC_TYPE_PRIMITIVE); + } + + private static void checkPrimitiveType(byte[] value, int offset, int expectedTypeId) { + if (basicType(value, offset) != BASIC_TYPE_PRIMITIVE) { + throw new IllegalStateException("Not a primitive variant value"); + } + int typeId = primitiveTypeId(value, offset); + if (typeId != expectedTypeId) { + throw new IllegalStateException( + "Expected primitive type " + expectedTypeId + " but got " + typeId); + } + } + + private static int binarySearchFieldName(byte[] metadata, int dictSize, byte[] nameBytes) { + int offsetSize = metadataOffsetSize(metadata); + int offsetsStart = 1 + offsetSize; + int strBytesStart = offsetsStart + (dictSize + 1) * offsetSize; + int lo = 0, hi = dictSize - 1; + while (lo <= hi) { + int mid = (lo + hi) >>> 1; + int strStart = readUnsignedLE(metadata, offsetsStart + mid * offsetSize, offsetSize); + int strEnd = + readUnsignedLE(metadata, offsetsStart + (mid + 1) * offsetSize, offsetSize); + int strLen = strEnd - strStart; + + int cmp = compareBytes(metadata, strBytesStart + strStart, strLen, nameBytes); + if (cmp < 0) { + lo = mid + 1; + } else if (cmp > 0) { + hi = mid - 1; + } else { + return mid; + } + } + return -1; + } + + private static int linearSearchFieldName(byte[] metadata, int dictSize, byte[] nameBytes) { + int offsetSize = metadataOffsetSize(metadata); + int offsetsStart = 1 + offsetSize; + int strBytesStart = offsetsStart + (dictSize + 1) * offsetSize; + for (int i = 0; i < dictSize; i++) { + int strStart = readUnsignedLE(metadata, offsetsStart + i * offsetSize, offsetSize); + int strEnd = readUnsignedLE(metadata, offsetsStart + (i + 1) * offsetSize, offsetSize); + int strLen = strEnd - strStart; + if (strLen == nameBytes.length + && compareBytes(metadata, strBytesStart + strStart, strLen, nameBytes) == 0) { + return i; + } + } + return -1; + } + + private static int compareBytes(byte[] a, int aOffset, int aLen, byte[] b) { + int minLen = Math.min(aLen, b.length); + for (int i = 0; i < minLen; i++) { + int cmp = (a[aOffset + i] & 0xFF) - (b[i] & 0xFF); + if (cmp != 0) { + return cmp; + } + } + return aLen - b.length; + } + + // -------------------------------------------------------------------------------------------- + // Little-endian read/write helpers + // -------------------------------------------------------------------------------------------- + + static short readShortLE(byte[] bytes, int offset) { + return (short) ((bytes[offset] & 0xFF) | ((bytes[offset + 1] & 0xFF) << 8)); + } + + static int readIntLE(byte[] bytes, int offset) { + return (bytes[offset] & 0xFF) + | ((bytes[offset + 1] & 0xFF) << 8) + | ((bytes[offset + 2] & 0xFF) << 16) + | ((bytes[offset + 3] & 0xFF) << 24); + } + + static long readLongLE(byte[] bytes, int offset) { + return (bytes[offset] & 0xFFL) + | ((bytes[offset + 1] & 0xFFL) << 8) + | ((bytes[offset + 2] & 0xFFL) << 16) + | ((bytes[offset + 3] & 0xFFL) << 24) + | ((bytes[offset + 4] & 0xFFL) << 32) + | ((bytes[offset + 5] & 0xFFL) << 40) + | ((bytes[offset + 6] & 0xFFL) << 48) + | ((bytes[offset + 7] & 0xFFL) << 56); + } + + static void writeShortLE(byte[] bytes, int offset, short value) { + bytes[offset] = (byte) value; + bytes[offset + 1] = (byte) (value >>> 8); + } + + public static void writeIntLE(byte[] bytes, int offset, int value) { + bytes[offset] = (byte) value; + bytes[offset + 1] = (byte) (value >>> 8); + bytes[offset + 2] = (byte) (value >>> 16); + bytes[offset + 3] = (byte) (value >>> 24); + } + + static void writeLongLE(byte[] bytes, int offset, long value) { + bytes[offset] = (byte) value; + bytes[offset + 1] = (byte) (value >>> 8); + bytes[offset + 2] = (byte) (value >>> 16); + bytes[offset + 3] = (byte) (value >>> 24); + bytes[offset + 4] = (byte) (value >>> 32); + bytes[offset + 5] = (byte) (value >>> 40); + bytes[offset + 6] = (byte) (value >>> 48); + bytes[offset + 7] = (byte) (value >>> 56); + } + + /** + * Reads a variable-width unsigned little-endian integer (1-4 bytes) from the byte array at the + * given position. This is required for reading values encoded with variable-width Parquet + * Variant encoding, where metadata offsets, object field IDs, and object/array offsets may be + * 1, 2, 3, or 4 bytes wide. + */ + static int readUnsignedLE(byte[] bytes, int offset, int numBytes) { + int result = 0; + for (int i = 0; i < numBytes; i++) { + result |= (bytes[offset + i] & 0xFF) << (8 * i); + } + return result; + } + + /** + * Returns the metadata offset size (1-4 bytes) from the metadata header byte. Bits 6-7 of the + * header encode {@code offset_size_minus_1}. + */ + static int metadataOffsetSize(byte[] metadata) { + return ((metadata[0] >> 6) & 0x3) + 1; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/ArrowUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/ArrowUtils.java index 4219d45ffd..0854b0453f 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/ArrowUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/ArrowUtils.java @@ -41,6 +41,8 @@ import org.apache.fluss.row.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.fluss.row.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.fluss.row.arrow.vectors.ArrowVarCharColumnVector; +import org.apache.fluss.row.arrow.vectors.ArrowVariantColumnVector; +import org.apache.fluss.row.arrow.vectors.ShreddedVariantColumnVector; import org.apache.fluss.row.arrow.writers.ArrowArrayWriter; import org.apache.fluss.row.arrow.writers.ArrowBigIntWriter; import org.apache.fluss.row.arrow.writers.ArrowBinaryWriter; @@ -53,6 +55,7 @@ import org.apache.fluss.row.arrow.writers.ArrowIntWriter; import org.apache.fluss.row.arrow.writers.ArrowMapWriter; import org.apache.fluss.row.arrow.writers.ArrowRowWriter; +import org.apache.fluss.row.arrow.writers.ArrowShreddedVariantWriter; import org.apache.fluss.row.arrow.writers.ArrowSmallIntWriter; import org.apache.fluss.row.arrow.writers.ArrowTimeWriter; import org.apache.fluss.row.arrow.writers.ArrowTimestampLtzWriter; @@ -60,9 +63,11 @@ import org.apache.fluss.row.arrow.writers.ArrowTinyIntWriter; import org.apache.fluss.row.arrow.writers.ArrowVarBinaryWriter; import org.apache.fluss.row.arrow.writers.ArrowVarCharWriter; +import org.apache.fluss.row.arrow.writers.ArrowVariantWriter; import org.apache.fluss.row.columnar.ColumnVector; import org.apache.fluss.row.columnar.VectorizedColumnBatch; import org.apache.fluss.shaded.arrow.com.google.flatbuffers.FlatBufferBuilder; +import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.Message; import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.MessageHeader; import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.RecordBatch; import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.ArrowBuf; @@ -132,14 +137,19 @@ import org.apache.fluss.types.TimeType; import org.apache.fluss.types.TimestampType; import org.apache.fluss.types.TinyIntType; +import org.apache.fluss.types.VariantType; +import org.apache.fluss.types.variant.ShreddedField; +import org.apache.fluss.types.variant.ShreddingSchema; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.channels.Channels; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import static org.apache.fluss.shaded.arrow.org.apache.arrow.vector.ipc.message.MessageSerializer.deserializeRecordBatch; @@ -160,6 +170,15 @@ public static Schema toArrowSchema(RowType rowType) { /** * Creates an {@link ArrowReader} for the specified memory segment and {@link VectorSchemaRoot}. + * + *

When the serialized data contains an embedded Arrow Schema message (present when variant + * shredding is active), the Schema is parsed to create a VectorSchemaRoot with the actual + * structure (including typed_value children). This root is owned by the returned ArrowReader + * and released via {@link ArrowReader#close()}. + * + *

When the rowType contains Variant columns backed by StructVector with typed_value + * children, they are automatically detected and wrapped into a {@link + * ShreddedVariantColumnVector}. */ public static ArrowReader createArrowReader( MemorySegment segment, @@ -169,25 +188,161 @@ public static ArrowReader createArrowReader( BufferAllocator allocator, RowType rowType) { ByteBuffer arrowBatchBuffer = segment.wrap(arrowOffset, arrowLength); + + VectorSchemaRoot actualRoot = schemaRoot; + VectorSchemaRoot ownedRoot = null; + + // Detect embedded Arrow Schema message (present when variant shredding is used). + // The writer serializes Schema + RecordBatch when shredded columns are present. + boolean hasVariantColumns = + rowType.getFields().stream().anyMatch(f -> f.getType() instanceof VariantType); + if (hasVariantColumns && arrowBatchBuffer.remaining() >= 8) { + arrowBatchBuffer.mark(); + arrowBatchBuffer.order(ByteOrder.LITTLE_ENDIAN); + int continuation = arrowBatchBuffer.getInt(); + int metadataSize = arrowBatchBuffer.getInt(); + + if (continuation == -1 /* IPC continuation 0xFFFFFFFF */ + && metadataSize > 0 + && arrowBatchBuffer.remaining() >= metadataSize) { + ByteBuffer metadataBuf = arrowBatchBuffer.slice(); + metadataBuf.limit(metadataSize); + Message message = Message.getRootAsMessage(metadataBuf); + + if (message.headerType() == MessageHeader.Schema) { + // Schema message found - create VectorSchemaRoot from actual schema + Schema actualSchema = MessageSerializer.deserializeSchema(message); + ownedRoot = VectorSchemaRoot.create(actualSchema, allocator); + actualRoot = ownedRoot; + + // Advance past Schema message (metadata padded to 8-byte boundary) + int paddedMetadataSize = (metadataSize + 7) & ~7; + arrowBatchBuffer.position(arrowBatchBuffer.position() + paddedMetadataSize); + } else { + arrowBatchBuffer.reset(); + } + } else { + arrowBatchBuffer.reset(); + } + } + try (ReadChannel channel = new ReadChannel(new ByteBufferReadableChannel(arrowBatchBuffer)); ArrowRecordBatch batch = deserializeRecordBatch(channel, allocator)) { FlussVectorLoader vectorLoader = - new FlussVectorLoader(schemaRoot, ArrowCompressionFactory.INSTANCE); + new FlussVectorLoader(actualRoot, ArrowCompressionFactory.INSTANCE); vectorLoader.load(batch); + + List fieldVectors = actualRoot.getFieldVectors(); + int fieldCount = rowType.getFieldCount(); + + // Build column vectors - Variant columns with typed_value are auto-detected List columnVectors = new ArrayList<>(); - List fieldVectors = schemaRoot.getFieldVectors(); - for (int i = 0; i < fieldVectors.size(); i++) { - columnVectors.add( - createArrowColumnVector(fieldVectors.get(i), rowType.getTypeAt(i))); + for (int i = 0; i < fieldCount; i++) { + FieldVector fieldVector = fieldVectors.get(i); + DataType dataType = rowType.getTypeAt(i); + + if (dataType instanceof VariantType && fieldVector instanceof StructVector) { + StructVector variantStruct = (StructVector) fieldVector; + // Check if this Variant has typed_value (shredding) + FieldVector typedValueChild = variantStruct.getChild("typed_value"); + if (typedValueChild instanceof StructVector + && !((StructVector) typedValueChild) + .getChildrenFromFields() + .isEmpty()) { + // Build ShreddingSchema from the Arrow schema structure + ShreddingSchema schema = + buildShreddingSchemaFromArrow( + rowType.getFields().get(i).getName(), + (StructVector) typedValueChild); + columnVectors.add(createShreddedVariantColumnVector(variantStruct, schema)); + } else { + // Basic Variant (no shredding) + columnVectors.add(new ArrowVariantColumnVector(variantStruct)); + } + } else { + columnVectors.add(createArrowColumnVector(fieldVector, dataType)); + } } + return new ArrowReader( - columnVectors.toArray(new ColumnVector[0]), schemaRoot.getRowCount()); - } catch (IOException e) { + columnVectors.toArray(new ColumnVector[0]), + actualRoot.getRowCount(), + ownedRoot); + } catch (Exception e) { + if (ownedRoot != null) { + ownedRoot.close(); + } + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } throw new RuntimeException("Failed to deserialize ArrowRecordBatch.", e); } } + /** + * Builds a ShreddingSchema by inspecting the Arrow typed_value StructVector structure. Each + * child of typed_value is a field-level StructVector{value: Binary, typed_value: <Type>}. + */ + private static ShreddingSchema buildShreddingSchemaFromArrow( + String variantName, StructVector typedValueVector) { + List fields = new ArrayList<>(); + List children = typedValueVector.getChildrenFromFields(); + for (int i = 0; i < children.size(); i++) { + FieldVector child = children.get(i); + String fieldPath = child.getName(); + // The typed_value sub-vector's type determines the shredded type + if (child instanceof StructVector) { + StructVector fieldStruct = (StructVector) child; + FieldVector typedVec = fieldStruct.getChild("typed_value"); + if (typedVec != null) { + DataType shreddedType = arrowTypeToDataType(typedVec.getField().getType()); + fields.add(new ShreddedField(fieldPath, shreddedType)); + } + } + } + return new ShreddingSchema(variantName, fields); + } + + /** + * Converts an Arrow type back to a Fluss DataType. Used when reconstructing ShreddingSchema + * from Arrow schema during reading. + */ + private static DataType arrowTypeToDataType(ArrowType arrowType) { + if (arrowType instanceof ArrowType.Int) { + ArrowType.Int intType = (ArrowType.Int) arrowType; + switch (intType.getBitWidth()) { + case 8: + return org.apache.fluss.types.DataTypes.TINYINT(); + case 16: + return org.apache.fluss.types.DataTypes.SMALLINT(); + case 32: + return org.apache.fluss.types.DataTypes.INT(); + case 64: + return org.apache.fluss.types.DataTypes.BIGINT(); + } + } else if (arrowType instanceof ArrowType.Bool) { + return org.apache.fluss.types.DataTypes.BOOLEAN(); + } else if (arrowType instanceof ArrowType.FloatingPoint) { + ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) arrowType; + if (fpType.getPrecision() == FloatingPointPrecision.SINGLE) { + return org.apache.fluss.types.DataTypes.FLOAT(); + } else { + return org.apache.fluss.types.DataTypes.DOUBLE(); + } + } else if (arrowType instanceof ArrowType.Utf8) { + return org.apache.fluss.types.DataTypes.STRING(); + } else if (arrowType instanceof ArrowType.Binary) { + return org.apache.fluss.types.DataTypes.BYTES(); + } else if (arrowType instanceof ArrowType.Date) { + return org.apache.fluss.types.DataTypes.DATE(); + } else if (arrowType instanceof ArrowType.Timestamp) { + return org.apache.fluss.types.DataTypes.TIMESTAMP(6); + } + throw new UnsupportedOperationException( + "Cannot convert Arrow type to DataType: " + arrowType); + } + /** * Serialize metadata of a {@link ArrowRecordBatch} into write channel. This avoids to create an * instance of {@link ArrowRecordBatch}. @@ -314,6 +469,8 @@ public static ArrowFieldWriter createArrowFieldWriter(FieldVector vector, DataTy return new ArrowVarCharWriter((VarCharVector) vector); } else if (vector instanceof FixedSizeBinaryVector) { return new ArrowBinaryWriter((FixedSizeBinaryVector) vector); + } else if (vector instanceof StructVector && dataType instanceof VariantType) { + return new ArrowVariantWriter((StructVector) vector); } else if (vector instanceof VarBinaryVector) { return new ArrowVarBinaryWriter((VarBinaryVector) vector); } else if (vector instanceof DecimalVector) { @@ -388,6 +545,8 @@ public static ColumnVector createArrowColumnVector(ValueVector vector, DataType return new ArrowVarCharColumnVector((VarCharVector) vector); } else if (vector instanceof FixedSizeBinaryVector) { return new ArrowBinaryColumnVector((FixedSizeBinaryVector) vector); + } else if (vector instanceof StructVector && dataType instanceof VariantType) { + return new ArrowVariantColumnVector((StructVector) vector); } else if (vector instanceof VarBinaryVector) { return new ArrowVarBinaryColumnVector((VarBinaryVector) vector); } else if (vector instanceof DecimalVector) { @@ -433,6 +592,106 @@ public static ColumnVector createArrowColumnVector(ValueVector vector, DataType } } + /** + * Returns the Arrow schema for a RowType with shredding support. For each Variant column with a + * ShreddingSchema, the Variant Field includes an additional typed_value child StructVector. + * + *

Layout: Struct{metadata: Binary, value: Binary, typed_value: Struct{ fieldName: + * Struct{value: Binary, typed_value: <Type>}, ... }} + * + * @param rowType the row type + * @param shreddingSchemas map of variant column name to its shredding schema + * @return the Arrow schema including shredded typed_value fields + */ + public static Schema toArrowSchemaWithShredding( + RowType rowType, Map shreddingSchemas) { + List fields = new ArrayList<>(); + for (DataField f : rowType.getFields()) { + if (f.getType() instanceof VariantType && shreddingSchemas.containsKey(f.getName())) { + fields.add( + toVariantFieldWithShredding( + f.getName(), shreddingSchemas.get(f.getName()))); + } else { + fields.add(toArrowField(f.getName(), f.getType())); + } + } + return new Schema(fields); + } + + /** + * Builds a Variant Arrow Field with typed_value support for shredding. + * + *

Layout: + * + *

+     * Struct "variantName" (nullable)
+     *   ├── VarBinary "metadata" (nullable)
+     *   ├── VarBinary "value"    (nullable, residual)
+     *   └── Struct "typed_value" (nullable)
+     *         ├── Struct "fieldName1" (nullable)
+     *         │     ├── VarBinary "value"       (nullable, per-field fallback)
+     *         │     └── <TypedVector> "typed_value" (nullable)
+     *         └── Struct "fieldName2" (nullable)
+     *               ├── VarBinary "value"       (nullable)
+     *               └── <TypedVector> "typed_value" (nullable)
+     * 
+ */ + public static Field toVariantFieldWithShredding( + String variantName, ShreddingSchema shreddingSchema) { + List typedValueChildren = new ArrayList<>(); + for (ShreddedField sf : shreddingSchema.getFields()) { + // Each field is: Struct{value: Binary, typed_value: } + DataType shreddedType = sf.getShreddedType().copy(true); + List fieldChildren = new ArrayList<>(2); + fieldChildren.add(Field.nullable("value", ArrowType.Binary.INSTANCE)); + fieldChildren.add(toArrowField("typed_value", shreddedType)); + typedValueChildren.add( + new Field( + sf.getFieldPath(), + FieldType.notNullable(ArrowType.Struct.INSTANCE), + fieldChildren)); + } + + Field typedValueField = + new Field( + "typed_value", + FieldType.notNullable(ArrowType.Struct.INSTANCE), + typedValueChildren); + + List variantChildren = new ArrayList<>(3); + variantChildren.add(Field.nullable("metadata", ArrowType.Binary.INSTANCE)); + variantChildren.add(Field.nullable("value", ArrowType.Binary.INSTANCE)); + variantChildren.add(typedValueField); + + return new Field( + variantName, FieldType.nullable(ArrowType.Struct.INSTANCE), variantChildren); + } + + /** + * Creates an {@link ArrowShreddedVariantWriter} for a Variant column with shredding. + * + * @param variantStructVector the top-level StructVector for the Variant column (contains + * metadata, value, typed_value children) + * @param shreddingSchema the shredding schema describing which fields to extract + * @return the shredded variant writer + */ + public static ArrowFieldWriter createArrowShreddedVariantWriter( + StructVector variantStructVector, ShreddingSchema shreddingSchema) { + return new ArrowShreddedVariantWriter(variantStructVector, shreddingSchema); + } + + /** + * Creates a {@link ShreddedVariantColumnVector} for reading shredded Variant columns. + * + * @param variantStructVector the top-level StructVector for the Variant column + * @param shreddingSchema the shredding schema + * @return the shredded variant column vector + */ + public static ColumnVector createShreddedVariantColumnVector( + StructVector variantStructVector, ShreddingSchema shreddingSchema) { + return new ShreddedVariantColumnVector(variantStructVector, shreddingSchema); + } + private static Field toArrowField(String fieldName, DataType logicalType) { FieldType fieldType = new FieldType( @@ -462,6 +721,11 @@ private static Field toArrowField(String fieldName, DataType logicalType) { Field structField = new Field(MapVector.DATA_VECTOR_NAME, structFieldType, structChildren); children = Collections.singletonList(structField); + } else if (logicalType instanceof VariantType) { + // Variant is represented as Struct{metadata: Binary, value: Binary} + children = new ArrayList<>(2); + children.add(Field.nullable("metadata", ArrowType.Binary.INSTANCE)); + children.add(Field.nullable("value", ArrowType.Binary.INSTANCE)); } return new Field(fieldName, fieldType, children); } @@ -592,6 +856,11 @@ public ArrowType visit(RowType rowType) { return ArrowType.Struct.INSTANCE; } + @Override + public ArrowType visit(VariantType variantType) { + return ArrowType.Struct.INSTANCE; + } + @Override protected ArrowType defaultMethod(DataType dataType) { throw new UnsupportedOperationException( diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/DataTypeJsonSerde.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/DataTypeJsonSerde.java index 12e6fb1c54..1be159021a 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/json/DataTypeJsonSerde.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/json/DataTypeJsonSerde.java @@ -107,6 +107,7 @@ private static void serializeTypeWithGenericSerialization( case DATE: case BYTES: case STRING: + case VARIANT: // do nothing. break; case CHAR: @@ -235,6 +236,8 @@ private static DataType deserializeFromRoot(JsonNode dataTypeNode) { return DataTypes.TIME(dataTypeNode.get(FIELD_NAME_PRECISION).asInt()); case STRING: return DataTypes.STRING(); + case VARIANT: + return DataTypes.VARIANT(); case BYTES: return DataTypes.BYTES(); case BINARY: diff --git a/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java b/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java index bacd2d1ab6..3caa7828ee 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java @@ -19,11 +19,24 @@ import org.apache.fluss.exception.InvalidColumnProjectionException; import org.apache.fluss.exception.SchemaNotExistException; +import org.apache.fluss.memory.ManagedPagedOutputView; +import org.apache.fluss.memory.TestingMemorySegmentPool; import org.apache.fluss.metadata.LogFormat; +import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.record.bytesview.BytesView; import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.arrow.ArrowWriter; +import org.apache.fluss.row.arrow.ArrowWriterPool; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.RootAllocator; +import org.apache.fluss.testutils.DataTestUtils; +import org.apache.fluss.types.DataField; +import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.ShreddedField; +import org.apache.fluss.types.variant.ShreddingSchema; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.CloseableIterator; import org.junit.jupiter.api.BeforeEach; @@ -39,7 +52,10 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.stream.Stream; @@ -194,18 +210,22 @@ void testProjectionOldDataWithNewSchema() throws Exception { new Object[] {"i"}, new Object[] {"j"}); - assertThatThrownBy( - () -> - doProjection( - 1L, - 2, - projection, - records, - new int[] {0, 2}, - records.sizeInBytes())) - .isInstanceOf(InvalidColumnProjectionException.class) - .hasMessage( - "Projected fields [0, 2] is out of bound for schema with 2 fields."); + // Schema evolution: [0, 2] is valid against the latest schema (3 fields). + // For old data (schemaId=1, 2 fields), createProjectionInfo filters [0, 2] + // to [0]. In production, LogRecordReadContext applies the same filtering + // on the client side. + projection.setCurrentProjection( + 1L, testingSchemaGetter, DEFAULT_COMPRESSION, new int[] {0, 2}); + LogRecords projected = + projection.project( + records.channel(), 0, records.sizeInBytes(), records.sizeInBytes()); + assertThat(projected.sizeInBytes()).isGreaterThan(0); + // Verify the cached projection info has filtered positions for old schema + FileLogProjection.ProjectionInfo info = + cache.getProjectionInfo(1L, (short) 1, new int[] {0, 2}); + assertThat(info).isNotNull(); + assertThat(info.selectedFieldPositions).isEqualTo(new int[] {0}); + assertThat(info.nodesProjection.stream().toArray()).isEqualTo(new int[] {0}); } } @@ -983,4 +1003,320 @@ void testProjectStatisticsClearing(byte recordBatchMagic) throws Exception { } } } + + // -------------------------------------------------------------------------------------------- + // Shredding schema projection tests + // -------------------------------------------------------------------------------------------- + + private static final RowType VARIANT_ROW_TYPE = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("data", DataTypes.VARIANT())); + + private static final Schema VARIANT_SCHEMA = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("data", DataTypes.VARIANT()) + .build(); + + private static final short VARIANT_SCHEMA_ID = 3; + + private static ShreddingSchema defaultShreddingSchema() { + return new ShreddingSchema( + "data", + Arrays.asList( + new ShreddedField("name", DataTypes.STRING()), + new ShreddedField("age", DataTypes.INT()))); + } + + private static List variantTestData() { + return Arrays.asList( + new Object[] { + 1, Variant.fromJson("{\"name\":\"Alice\",\"age\":25,\"city\":\"NYC\"}") + }, + new Object[] {2, Variant.fromJson("{\"name\":\"Bob\",\"age\":30}")}, + new Object[] { + 3, Variant.fromJson("{\"name\":\"Carol\",\"age\":35,\"extra\":true}") + }); + } + + @SafeVarargs + private final FileLogRecords createShreddedFileLogRecords( + short schemaId, + byte recordBatchMagic, + RowType rowType, + Map shreddingSchemas, + List... inputs) + throws Exception { + FileLogRecords fileLogRecords = + FileLogRecords.open(new File(tempDir, UUID.randomUUID() + ".log")); + long offsetBase = 0L; + for (List input : inputs) { + try (BufferAllocator allocator = new RootAllocator(Integer.MAX_VALUE); + ArrowWriterPool provider = new ArrowWriterPool(allocator)) { + ArrowWriter writer = + provider.getOrCreateWriter( + 1L, + schemaId, + Integer.MAX_VALUE, + rowType, + DEFAULT_COMPRESSION, + shreddingSchemas); + MemoryLogRecordsArrowBuilder builder = + MemoryLogRecordsArrowBuilder.builder( + offsetBase, + recordBatchMagic, + schemaId, + writer, + new ManagedPagedOutputView( + new TestingMemorySegmentPool(10 * 1024))); + for (Object[] objs : input) { + builder.append(ChangeType.APPEND_ONLY, DataTestUtils.row(rowType, objs)); + } + builder.close(); + MemoryLogRecords memoryLogRecords = + MemoryLogRecords.pointToBytesView(builder.build()); + ((DefaultLogRecordBatch) memoryLogRecords.batches().iterator().next()) + .setCommitTimestamp(System.currentTimeMillis()); + memoryLogRecords.ensureValid(recordBatchMagic); + fileLogRecords.append(memoryLogRecords); + offsetBase += input.size(); + } + } + fileLogRecords.flush(); + return fileLogRecords; + } + + /** Read projected records and extract field values. Supports INT, STRING, and VARIANT types. */ + private List doShreddedProjection( + long tableId, + int schemaId, + TestingSchemaGetter schemaGetter, + FileLogProjection projection, + FileLogRecords fileLogRecords, + int[] projectedFields, + int fetchMaxBytes) + throws Exception { + projection.setCurrentProjection( + tableId, schemaGetter, DEFAULT_COMPRESSION, projectedFields); + LogRecords project = + projection.project( + fileLogRecords.channel(), 0, fileLogRecords.sizeInBytes(), fetchMaxBytes); + RowType rowType = schemaGetter.getSchema(schemaId).getRowType(); + RowType projectedType = rowType.project(projectedFields); + List results = new ArrayList<>(); + try (LogRecordReadContext context = + createArrowReadContext(projectedType, schemaId, schemaGetter, true)) { + for (LogRecordBatch batch : project.batches()) { + try (CloseableIterator records = batch.records(context)) { + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + Object[] objs = new Object[projectedFields.length]; + for (int i = 0; i < projectedFields.length; i++) { + if (row.isNullAt(i)) { + objs[i] = null; + continue; + } + switch (projectedType.getTypeAt(i).getTypeRoot()) { + case INTEGER: + objs[i] = row.getInt(i); + break; + case STRING: + objs[i] = row.getString(i).toString(); + break; + case VARIANT: + objs[i] = row.getVariant(i); + break; + default: + throw new IllegalArgumentException( + "Unsupported type: " + projectedType.getTypeAt(i)); + } + } + results.add(objs); + } + } + } + } + return results; + } + + private static void assertVariantField( + Variant variant, String fieldName, Object expectedValue) { + Variant field = variant.getFieldByName(fieldName); + assertThat(field).as("field '" + fieldName + "' should exist").isNotNull(); + if (expectedValue instanceof String) { + assertThat(field.getString()).isEqualTo(expectedValue); + } else if (expectedValue instanceof Integer) { + assertThat(field.getInt()).isEqualTo((int) expectedValue); + } else if (expectedValue instanceof Boolean) { + assertThat(field.getBoolean()).isEqualTo((boolean) expectedValue); + } else { + fail("Unsupported expectedValue type: " + expectedValue.getClass()); + } + } + + @Test + void testShreddedProjectionAllColumns() throws Exception { + Map shreddingSchemas = + Collections.singletonMap("data", defaultShreddingSchema()); + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(VARIANT_SCHEMA, VARIANT_SCHEMA_ID)); + + try (FileLogRecords records = + createShreddedFileLogRecords( + VARIANT_SCHEMA_ID, + LOG_MAGIC_VALUE_V1, + VARIANT_ROW_TYPE, + shreddingSchemas, + variantTestData())) { + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + List results = + doShreddedProjection( + 1L, + VARIANT_SCHEMA_ID, + schemaGetter, + projection, + records, + new int[] {0, 1}, + records.sizeInBytes()); + assertThat(results).hasSize(3); + // Row 0: id=1, data={name:Alice, age:25, city:NYC} + assertThat(results.get(0)[0]).isEqualTo(1); + Variant v0 = (Variant) results.get(0)[1]; + assertVariantField(v0, "name", "Alice"); + assertVariantField(v0, "age", 25); + assertVariantField(v0, "city", "NYC"); + // Row 1: id=2, data={name:Bob, age:30} + assertThat(results.get(1)[0]).isEqualTo(2); + Variant v1 = (Variant) results.get(1)[1]; + assertVariantField(v1, "name", "Bob"); + assertVariantField(v1, "age", 30); + // Row 2: id=3, data={name:Carol, age:35, extra:true} + assertThat(results.get(2)[0]).isEqualTo(3); + Variant v2 = (Variant) results.get(2)[1]; + assertVariantField(v2, "name", "Carol"); + assertVariantField(v2, "age", 35); + assertVariantField(v2, "extra", true); + } + } + + @Test + void testShreddedProjectionVariantOnly() throws Exception { + Map shreddingSchemas = + Collections.singletonMap("data", defaultShreddingSchema()); + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(VARIANT_SCHEMA, VARIANT_SCHEMA_ID)); + + try (FileLogRecords records = + createShreddedFileLogRecords( + VARIANT_SCHEMA_ID, + LOG_MAGIC_VALUE_V1, + VARIANT_ROW_TYPE, + shreddingSchemas, + variantTestData())) { + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + // Project only variant column [1] + List results = + doShreddedProjection( + 1L, + VARIANT_SCHEMA_ID, + schemaGetter, + projection, + records, + new int[] {1}, + records.sizeInBytes()); + assertThat(results).hasSize(3); + // Verify variant values are intact after projection + Variant v0 = (Variant) results.get(0)[0]; + assertVariantField(v0, "name", "Alice"); + assertVariantField(v0, "age", 25); + assertVariantField(v0, "city", "NYC"); + + Variant v1 = (Variant) results.get(1)[0]; + assertVariantField(v1, "name", "Bob"); + assertVariantField(v1, "age", 30); + + Variant v2 = (Variant) results.get(2)[0]; + assertVariantField(v2, "name", "Carol"); + assertVariantField(v2, "age", 35); + assertVariantField(v2, "extra", true); + } + } + + @Test + void testShreddedProjectionNonVariantOnly() throws Exception { + Map shreddingSchemas = + Collections.singletonMap("data", defaultShreddingSchema()); + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(VARIANT_SCHEMA, VARIANT_SCHEMA_ID)); + + try (FileLogRecords records = + createShreddedFileLogRecords( + VARIANT_SCHEMA_ID, + LOG_MAGIC_VALUE_V1, + VARIANT_ROW_TYPE, + shreddingSchemas, + variantTestData())) { + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + // Project only non-variant column [0] (id: INT) + // Schema prefix should NOT be included since projected field has no typed_value + List results = + doShreddedProjection( + 1L, + VARIANT_SCHEMA_ID, + schemaGetter, + projection, + records, + new int[] {0}, + records.sizeInBytes()); + assertThat(results).hasSize(3); + assertThat(results.get(0)[0]).isEqualTo(1); + assertThat(results.get(1)[0]).isEqualTo(2); + assertThat(results.get(2)[0]).isEqualTo(3); + } + } + + @Test + void testShreddedProjectionMultipleBatches() throws Exception { + Map shreddingSchemas = + Collections.singletonMap("data", defaultShreddingSchema()); + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(VARIANT_SCHEMA, VARIANT_SCHEMA_ID)); + + List batch1 = + Arrays.asList( + new Object[] {1, Variant.fromJson("{\"name\":\"Alice\",\"age\":25}")}, + new Object[] {2, Variant.fromJson("{\"name\":\"Bob\",\"age\":30}")}); + List batch2 = + Collections.singletonList( + new Object[] {3, Variant.fromJson("{\"name\":\"Carol\",\"age\":35}")}); + + try (FileLogRecords records = + createShreddedFileLogRecords( + VARIANT_SCHEMA_ID, + LOG_MAGIC_VALUE_V1, + VARIANT_ROW_TYPE, + shreddingSchemas, + batch1, + batch2)) { + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + List results = + doShreddedProjection( + 1L, + VARIANT_SCHEMA_ID, + schemaGetter, + projection, + records, + new int[] {0, 1}, + records.sizeInBytes()); + assertThat(results).hasSize(3); + assertThat(results.get(0)[0]).isEqualTo(1); + assertVariantField((Variant) results.get(0)[1], "name", "Alice"); + assertThat(results.get(1)[0]).isEqualTo(2); + assertVariantField((Variant) results.get(1)[1], "name", "Bob"); + assertThat(results.get(2)[0]).isEqualTo(3); + assertVariantField((Variant) results.get(2)[1], "name", "Carol"); + } + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordsTest.java b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordsTest.java index a8f851cee9..6c02e5147d 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordsTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordsTest.java @@ -37,6 +37,12 @@ import java.util.Iterator; import java.util.List; +import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; +import static org.apache.fluss.record.TestData.VARIANT_DATA; +import static org.apache.fluss.record.TestData.VARIANT_ROW_TYPE; +import static org.apache.fluss.record.TestData.VARIANT_SCHEMA; +import static org.apache.fluss.testutils.DataTestUtils.assertLogRecordsEquals; +import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsByObject; import static org.apache.fluss.testutils.DataTestUtils.indexedRow; import static org.assertj.core.api.Assertions.assertThat; @@ -111,6 +117,22 @@ void testAllTypeData() throws Exception { assertThat(memoryIterator.hasNext()).isFalse(); } + @Test + void testVariantData() throws Exception { + MemoryLogRecords memoryLogRecords = + genMemoryLogRecordsByObject( + VARIANT_ROW_TYPE, DEFAULT_SCHEMA_ID, magic, VARIANT_DATA); + + // Create FileLogRecords. + fileLogRecords = new FileLogRecords(file, fileChannel, 0, Integer.MAX_VALUE, false); + fileLogRecords.append(memoryLogRecords); + + // Verify data by reading back from FileLogRecords. + SchemaGetter variantSchemaGetter = + new TestingSchemaGetter(DEFAULT_SCHEMA_ID, VARIANT_SCHEMA); + assertLogRecordsEquals(VARIANT_ROW_TYPE, fileLogRecords, VARIANT_DATA, variantSchemaGetter); + } + @AfterEach public void after() throws IOException { if (fileLogRecords != null) { diff --git a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java index be4e0ad785..2185982cd9 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java @@ -27,6 +27,7 @@ import org.apache.fluss.types.DataField; import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.types.Tuple2; import java.util.Arrays; @@ -265,6 +266,39 @@ public final class TestData { // ---------------------------- data3 table info end ------------------------------ + // ------------------- variant data and related table info begin ---------------------- + public static final RowType VARIANT_ROW_TYPE = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("data", DataTypes.VARIANT())); + + public static final Schema VARIANT_SCHEMA = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .withComment("id is primary column") + .column("data", DataTypes.VARIANT()) + .withComment("data is variant column") + .build(); + + public static final long VARIANT_TABLE_ID = 150006L; + public static final TablePath VARIANT_TABLE_PATH = + TablePath.of("test_db_1", "test_variant_table"); + public static final PhysicalTablePath VARIANT_PHYSICAL_TABLE_PATH = + PhysicalTablePath.of(VARIANT_TABLE_PATH); + public static final TableDescriptor VARIANT_TABLE_DESCRIPTOR = + TableDescriptor.builder().schema(VARIANT_SCHEMA).distributedBy(3).build(); + + public static final List VARIANT_DATA = + Arrays.asList( + new Object[] { + 1, Variant.fromJson("{\"name\":\"Alice\",\"age\":25,\"city\":\"NYC\"}") + }, + new Object[] {2, Variant.fromJson("{\"name\":\"Bob\",\"age\":30}")}, + new Object[] { + 3, Variant.fromJson("{\"name\":\"Carol\",\"age\":35,\"extra\":true}") + }); + // ------------------- variant data and related table info end ------------------------ + // ------------------- Statistics test data and schemas begin ---------------------- // Schema with mixed types for comprehensive statistics testing public static final RowType STATISTICS_MIXED_TYPE_ROW_TYPE = diff --git a/fluss-common/src/test/java/org/apache/fluss/row/arrow/ShreddedVariantRoundTripTest.java b/fluss-common/src/test/java/org/apache/fluss/row/arrow/ShreddedVariantRoundTripTest.java new file mode 100644 index 0000000000..4948ffd1bf --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/row/arrow/ShreddedVariantRoundTripTest.java @@ -0,0 +1,459 @@ +/* + * 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.fluss.row.arrow; + +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.arrow.vectors.ShreddedVariantColumnVector; +import org.apache.fluss.row.arrow.writers.ArrowShreddedVariantWriter; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.RootAllocator; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.types.pojo.Field; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.types.pojo.Schema; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.variant.ShreddedField; +import org.apache.fluss.types.variant.ShreddingSchema; +import org.apache.fluss.types.variant.Variant; +import org.apache.fluss.utils.ArrowUtils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Round-trip tests for {@link ArrowShreddedVariantWriter} and {@link ShreddedVariantColumnVector}. + * Verifies that writing a Variant through shredding and reading it back produces identical results. + * + *

The new design uses StructVector layout: Struct{metadata, value, typed_value: Struct{ + * fieldName: Struct{value, typed_value}, ...}}. + */ +class ShreddedVariantRoundTripTest { + + private BufferAllocator allocator; + + @BeforeEach + void setUp() { + allocator = new RootAllocator(); + } + + @AfterEach + void tearDown() { + allocator.close(); + } + + /** + * Creates a StructVector with the full shredding layout and returns the VectorSchemaRoot for + * lifecycle management. + */ + private VectorSchemaRoot createVariantRoot(ShreddingSchema shreddingSchema) { + Field variantField = ArrowUtils.toVariantFieldWithShredding("data", shreddingSchema); + Schema schema = new Schema(Collections.singletonList(variantField)); + VectorSchemaRoot root = VectorSchemaRoot.create(schema, allocator); + // Allocate all vectors recursively + for (org.apache.fluss.shaded.arrow.org.apache.arrow.vector.FieldVector fv : + root.getFieldVectors()) { + fv.allocateNew(); + } + return root; + } + + // -------------------------------------------------------------------------------------------- + // All fields shredded (null residual) + // -------------------------------------------------------------------------------------------- + + @Test + void testAllFieldsShredded() { + ShreddingSchema schema = + new ShreddingSchema( + "data", + Arrays.asList( + new ShreddedField("age", DataTypes.INT()), + new ShreddedField("name", DataTypes.STRING()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = Variant.fromJson("{\"age\":25,\"name\":\"Alice\"}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + assertThat(reader.isNullAt(0)).isFalse(); + + Variant result = reader.getVariant(0); + assertThat(result).isNotNull(); + assertThat(result.isObject()).isTrue(); + assertThat(result.getFieldByName("age").getInt()).isEqualTo(25); + assertThat(result.getFieldByName("name").getString()).isEqualTo("Alice"); + } + } + + // -------------------------------------------------------------------------------------------- + // Partial shredding (some fields in residual) + // -------------------------------------------------------------------------------------------- + + @Test + void testPartialShredding() { + ShreddingSchema schema = + new ShreddingSchema( + "data", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = Variant.fromJson("{\"age\":30,\"name\":\"Bob\",\"active\":true}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + Variant result = reader.getVariant(0); + assertThat(result.getFieldByName("age").getInt()).isEqualTo(30); + assertThat(result.getFieldByName("name").getString()).isEqualTo("Bob"); + assertThat(result.getFieldByName("active").getBoolean()).isTrue(); + } + } + + // -------------------------------------------------------------------------------------------- + // Nested objects in residual + // -------------------------------------------------------------------------------------------- + + @Test + void testNestedObjectInResidual() { + ShreddingSchema schema = + new ShreddingSchema( + "data", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = + Variant.fromJson( + "{\"age\":28,\"address\":{\"city\":\"Beijing\",\"zip\":\"100000\"}}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + Variant result = reader.getVariant(0); + assertThat(result.getFieldByName("age").getInt()).isEqualTo(28); + Variant address = result.getFieldByName("address"); + assertThat(address).isNotNull(); + assertThat(address.isObject()).isTrue(); + assertThat(address.getFieldByName("city").getString()).isEqualTo("Beijing"); + assertThat(address.getFieldByName("zip").getString()).isEqualTo("100000"); + } + } + + @Test + void testDeeplyNestedObjectInResidual() { + ShreddingSchema schema = + new ShreddingSchema( + "data", Arrays.asList(new ShreddedField("name", DataTypes.STRING()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = + Variant.fromJson( + "{\"name\":\"Charlie\",\"info\":{\"contact\":{\"email\":\"a@b.com\"}}}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + Variant result = reader.getVariant(0); + assertThat(result.getFieldByName("name").getString()).isEqualTo("Charlie"); + Variant info = result.getFieldByName("info"); + assertThat(info.isObject()).isTrue(); + Variant contact = info.getFieldByName("contact"); + assertThat(contact.isObject()).isTrue(); + assertThat(contact.getFieldByName("email").getString()).isEqualTo("a@b.com"); + } + } + + // -------------------------------------------------------------------------------------------- + // Non-object variant (scalar value) + // -------------------------------------------------------------------------------------------- + + @Test + void testScalarVariantGoesToResidual() { + ShreddingSchema schema = + new ShreddingSchema( + "data", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = Variant.ofString("just a string"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + Variant result = reader.getVariant(0); + assertThat(result.getString()).isEqualTo("just a string"); + } + } + + // -------------------------------------------------------------------------------------------- + // Null variant + // -------------------------------------------------------------------------------------------- + + @Test + void testNullVariant() { + ShreddingSchema schema = + new ShreddingSchema( + "data", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + writer.write(0, GenericRow.of((Variant) null), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + assertThat(reader.isNullAt(0)).isTrue(); + } + } + + // -------------------------------------------------------------------------------------------- + // Type mismatch — field goes to per-field value fallback + // -------------------------------------------------------------------------------------------- + + @Test + void testTypeMismatchKeepsInPerFieldValue() { + ShreddingSchema schema = + new ShreddingSchema( + "data", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + // "age" is a STRING — type mismatch for INT shredded column + Variant input = Variant.fromJson("{\"age\":\"not-a-number\",\"name\":\"Dave\"}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + Variant result = reader.getVariant(0); + assertThat(result.getFieldByName("age").getString()).isEqualTo("not-a-number"); + assertThat(result.getFieldByName("name").getString()).isEqualTo("Dave"); + } + } + + // -------------------------------------------------------------------------------------------- + // Multiple rows with mixed data + // -------------------------------------------------------------------------------------------- + + @Test + void testMultipleRows() { + ShreddingSchema schema = + new ShreddingSchema( + "data", + Arrays.asList( + new ShreddedField("id", DataTypes.BIGINT()), + new ShreddedField("name", DataTypes.STRING()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + + // Row 0: all fields present + Variant v0 = Variant.fromJson("{\"id\":1,\"name\":\"Alice\",\"extra\":\"x\"}"); + writer.write(0, GenericRow.of(v0), 0, true); + + // Row 1: missing "name" + Variant v1 = Variant.fromJson("{\"id\":2,\"extra\":\"y\"}"); + writer.write(1, GenericRow.of(v1), 0, true); + + // Row 2: null variant + writer.write(2, GenericRow.of((Variant) null), 0, true); + + // Row 3: nested object with all shredded fields + Variant v3 = + Variant.fromJson( + "{\"id\":3,\"name\":\"Carol\",\"meta\":{\"tag\":\"important\"}}"); + writer.write(3, GenericRow.of(v3), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + // Row 0 + Variant r0 = reader.getVariant(0); + assertThat(r0.getFieldByName("id").getLong()).isEqualTo(1L); + assertThat(r0.getFieldByName("name").getString()).isEqualTo("Alice"); + assertThat(r0.getFieldByName("extra").getString()).isEqualTo("x"); + + // Row 1 + Variant r1 = reader.getVariant(1); + assertThat(r1.getFieldByName("id").getLong()).isEqualTo(2L); + assertThat(r1.getFieldByName("extra").getString()).isEqualTo("y"); + assertThat(r1.getFieldByName("name")).isNull(); + + // Row 2 + assertThat(reader.isNullAt(2)).isTrue(); + + // Row 3 + Variant r3 = reader.getVariant(3); + assertThat(r3.getFieldByName("id").getLong()).isEqualTo(3L); + assertThat(r3.getFieldByName("name").getString()).isEqualTo("Carol"); + Variant meta = r3.getFieldByName("meta"); + assertThat(meta.isObject()).isTrue(); + assertThat(meta.getFieldByName("tag").getString()).isEqualTo("important"); + } + } + + // -------------------------------------------------------------------------------------------- + // Explicit null values in fields + // -------------------------------------------------------------------------------------------- + + @Test + void testExplicitNullFieldValueAllShredded() { + ShreddingSchema schema = + new ShreddingSchema( + "data", + Arrays.asList( + new ShreddedField("age", DataTypes.BIGINT()), + new ShreddedField("name", DataTypes.STRING()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = Variant.fromJson("{\"age\":null,\"name\":\"Alice\"}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + Variant result = reader.getVariant(0); + assertThat(result).isNotNull(); + assertThat(result.isObject()).isTrue(); + + // "age" should still exist as an explicit null + Variant ageField = result.getFieldByName("age"); + assertThat(ageField).isNotNull(); + assertThat(ageField.isNull()).isTrue(); + + assertThat(result.getFieldByName("name").getString()).isEqualTo("Alice"); + } + } + + @Test + void testExplicitNullFieldValuePartialShredding() { + ShreddingSchema schema = + new ShreddingSchema( + "data", Arrays.asList(new ShreddedField("age", DataTypes.BIGINT()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = Variant.fromJson("{\"age\":null,\"name\":\"Bob\",\"extra\":\"x\"}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + Variant result = reader.getVariant(0); + assertThat(result).isNotNull(); + + Variant ageField = result.getFieldByName("age"); + assertThat(ageField).isNotNull(); + assertThat(ageField.isNull()).isTrue(); + assertThat(result.getFieldByName("name").getString()).isEqualTo("Bob"); + assertThat(result.getFieldByName("extra").getString()).isEqualTo("x"); + } + } + + // -------------------------------------------------------------------------------------------- + // Empty object {} + // -------------------------------------------------------------------------------------------- + + @Test + void testEmptyObjectRoundTrip() { + ShreddingSchema schema = + new ShreddingSchema( + "data", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = Variant.fromJson("{}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + assertThat(reader.isNullAt(0)).isFalse(); + + Variant result = reader.getVariant(0); + assertThat(result).isNotNull(); + assertThat(result.isObject()).isTrue(); + assertThat(result.getFieldByName("age")).isNull(); + } + } + + // -------------------------------------------------------------------------------------------- + // Field missing in input + // -------------------------------------------------------------------------------------------- + + @Test + void testFieldNotPresentInInput() { + ShreddingSchema schema = + new ShreddingSchema( + "data", + Arrays.asList( + new ShreddedField("age", DataTypes.INT()), + new ShreddedField("name", DataTypes.STRING()))); + + try (VectorSchemaRoot root = createVariantRoot(schema)) { + StructVector variantVec = (StructVector) root.getVector(0); + + ArrowShreddedVariantWriter writer = new ArrowShreddedVariantWriter(variantVec, schema); + Variant input = Variant.fromJson("{\"age\":42}"); + writer.write(0, GenericRow.of(input), 0, true); + + ShreddedVariantColumnVector reader = + new ShreddedVariantColumnVector(variantVec, schema); + + Variant result = reader.getVariant(0); + assertThat(result.getFieldByName("age").getInt()).isEqualTo(42); + assertThat(result.getFieldByName("name")).isNull(); + } + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/types/variant/JsonToVariantTest.java b/fluss-common/src/test/java/org/apache/fluss/types/variant/JsonToVariantTest.java new file mode 100644 index 0000000000..8105dee69a --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/types/variant/JsonToVariantTest.java @@ -0,0 +1,460 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.memory.AbstractPagedOutputView; +import org.apache.fluss.memory.ManagedPagedOutputView; +import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.memory.TestingMemorySegmentPool; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.arrow.ArrowReader; +import org.apache.fluss.row.arrow.ArrowWriter; +import org.apache.fluss.row.arrow.ArrowWriterPool; +import org.apache.fluss.row.columnar.ColumnarRow; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.RootAllocator; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; +import org.apache.fluss.utils.ArrowUtils; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.function.Consumer; + +import static org.apache.fluss.compression.ArrowCompressionInfo.NO_COMPRESSION; +import static org.apache.fluss.record.LogRecordBatch.CURRENT_LOG_MAGIC_VALUE; +import static org.apache.fluss.record.LogRecordBatchFormat.recordBatchHeaderSize; +import static org.apache.fluss.row.BinaryString.fromString; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration test for JSON to Variant conversion with Arrow read/write roundtrip. + * + *

This test covers the full pipeline: JSON String → Variant → Arrow write → Arrow read → Variant + * → JSON String, verifying data integrity through the entire flow. + */ +class JsonToVariantTest { + + // -------------------------------------------------------------------------------------------- + // Simple primitives through JSON → Variant → Arrow + // -------------------------------------------------------------------------------------------- + + @Test + void testJsonPrimitivesRoundtrip() throws IOException { + // Schema: (id INT, data VARIANT) + RowType rowType = DataTypes.ROW(new DataType[] {DataTypes.INT(), DataTypes.VARIANT()}); + + List rows = + Arrays.asList( + GenericRow.of(1, Variant.fromJson("null")), + GenericRow.of(2, Variant.fromJson("true")), + GenericRow.of(3, Variant.fromJson("false")), + GenericRow.of(4, Variant.fromJson("42")), + GenericRow.of(5, Variant.fromJson("3.14")), + GenericRow.of(6, Variant.fromJson("\"hello world\""))); + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(6); + + // Row 1: null variant + assertThat(results.get(0).getInt(0)).isEqualTo(1); + assertThat(results.get(0).getVariant(1).isNull()).isTrue(); + + // Row 2: true + assertThat(results.get(1).getInt(0)).isEqualTo(2); + assertThat(results.get(1).getVariant(1).getBoolean()).isTrue(); + + // Row 3: false + assertThat(results.get(2).getVariant(1).getBoolean()).isFalse(); + + // Row 4: integer 42 (encoded as int8 by JSON parser) + assertThat(results.get(3).getVariant(1).getByte()).isEqualTo((byte) 42); + + // Row 5: decimal 3.14 (decimal point without exponent => Decimal) + assertThat(results.get(4).getVariant(1).getDecimal()) + .isEqualByComparingTo(new java.math.BigDecimal("3.14")); + + // Row 6: string + assertThat(results.get(5).getVariant(1).getString()).isEqualTo("hello world"); + }); + } + + // -------------------------------------------------------------------------------------------- + // JSON objects through Arrow + // -------------------------------------------------------------------------------------------- + + @Test + void testJsonObjectsRoundtrip() throws IOException { + RowType rowType = DataTypes.ROW(new DataType[] {DataTypes.STRING(), DataTypes.VARIANT()}); + + List rows = + Arrays.asList( + GenericRow.of( + fromString("simple"), + Variant.fromJson("{\"name\":\"Alice\",\"age\":30}")), + GenericRow.of( + fromString("nested"), + Variant.fromJson( + "{\"user\":{\"name\":\"Bob\"},\"scores\":[90,85,95]}")), + GenericRow.of(fromString("empty"), Variant.fromJson("{}"))); + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(3); + + // Row 1: simple object + Variant v1 = results.get(0).getVariant(1); + assertThat(v1.isObject()).isTrue(); + assertThat(v1.getFieldByName("name").getString()).isEqualTo("Alice"); + assertThat(v1.getFieldByName("age").getByte()).isEqualTo((byte) 30); + + // Row 2: nested object + Variant v2 = results.get(1).getVariant(1); + assertThat(v2.isObject()).isTrue(); + assertThat(v2.getFieldByName("user").isObject()).isTrue(); + assertThat(v2.getFieldByName("user").getFieldByName("name").getString()) + .isEqualTo("Bob"); + assertThat(v2.getFieldByName("scores").isArray()).isTrue(); + assertThat(v2.getFieldByName("scores").arraySize()).isEqualTo(3); + + // Row 3: empty object + Variant v3 = results.get(2).getVariant(1); + assertThat(v3.isObject()).isTrue(); + assertThat(v3.objectSize()).isZero(); + }); + } + + // -------------------------------------------------------------------------------------------- + // JSON arrays through Arrow + // -------------------------------------------------------------------------------------------- + + @Test + void testJsonArraysRoundtrip() throws IOException { + RowType rowType = DataTypes.ROW(new DataType[] {DataTypes.INT(), DataTypes.VARIANT()}); + + List rows = + Arrays.asList( + GenericRow.of(1, Variant.fromJson("[1, 2, 3]")), + GenericRow.of(2, Variant.fromJson("[\"a\", \"b\", \"c\"]")), + GenericRow.of(3, Variant.fromJson("[1, \"mixed\", true, null]")), + GenericRow.of(4, Variant.fromJson("[]")), + GenericRow.of(5, Variant.fromJson("[[1,2],[3,4]]"))); + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(5); + + // Row 1: int array + Variant arr1 = results.get(0).getVariant(1); + assertThat(arr1.isArray()).isTrue(); + assertThat(arr1.arraySize()).isEqualTo(3); + assertThat(arr1.getElementAt(0).getByte()).isEqualTo((byte) 1); + assertThat(arr1.getElementAt(2).getByte()).isEqualTo((byte) 3); + + // Row 2: string array + Variant arr2 = results.get(1).getVariant(1); + assertThat(arr2.isArray()).isTrue(); + assertThat(arr2.getElementAt(0).getString()).isEqualTo("a"); + + // Row 3: mixed array + Variant arr3 = results.get(2).getVariant(1); + assertThat(arr3.arraySize()).isEqualTo(4); + assertThat(arr3.getElementAt(3).isNull()).isTrue(); + + // Row 4: empty array + assertThat(results.get(3).getVariant(1).arraySize()).isZero(); + + // Row 5: nested array + Variant arr5 = results.get(4).getVariant(1); + assertThat(arr5.arraySize()).isEqualTo(2); + assertThat(arr5.getElementAt(0).isArray()).isTrue(); + }); + } + + // -------------------------------------------------------------------------------------------- + // JSON string roundtrip (JSON → Variant → Arrow → Variant → JSON) + // -------------------------------------------------------------------------------------------- + + @Test + void testJsonStringRoundtrip() throws IOException { + RowType rowType = DataTypes.ROW(new DataType[] {DataTypes.VARIANT()}); + + String[] jsonStrings = { + "null", + "true", + "false", + "42", + "3.14", + "\"hello\"", + "{}", + "[]", + "{\"key\":\"value\"}", + "[1,2,3]", + }; + + List rows = new ArrayList<>(); + for (String json : jsonStrings) { + rows.add(GenericRow.of(Variant.fromJson(json))); + } + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(jsonStrings.length); + for (int i = 0; i < jsonStrings.length; i++) { + Variant readBack = results.get(i).getVariant(0); + assertThat(readBack.toJson()).isEqualTo(jsonStrings[i]); + } + }); + } + + // -------------------------------------------------------------------------------------------- + // Complex real-world JSON documents + // -------------------------------------------------------------------------------------------- + + @Test + void testComplexJsonDocuments() throws IOException { + RowType rowType = DataTypes.ROW(new DataType[] {DataTypes.INT(), DataTypes.VARIANT()}); + + String json1 = + "{\"event\":\"page_view\"," + + "\"timestamp\":1234567890," + + "\"user\":{\"id\":1001,\"name\":\"Alice\",\"tags\":[\"premium\",\"active\"]}," + + "\"properties\":{\"page\":\"/home\",\"duration\":5.2}}"; + + String json2 = + "{\"event\":\"purchase\"," + + "\"timestamp\":1234567891," + + "\"user\":{\"id\":1002,\"name\":\"Bob\"}," + + "\"properties\":{\"item\":\"widget\",\"price\":29.99,\"quantity\":2}}"; + + List rows = + Arrays.asList( + GenericRow.of(1, Variant.fromJson(json1)), + GenericRow.of(2, Variant.fromJson(json2))); + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(2); + + // Verify first document + Variant v1 = results.get(0).getVariant(1); + assertThat(v1.getFieldByName("event").getString()).isEqualTo("page_view"); + Variant user1 = v1.getFieldByName("user"); + assertThat(user1.getFieldByName("name").getString()).isEqualTo("Alice"); + Variant tags = user1.getFieldByName("tags"); + assertThat(tags.isArray()).isTrue(); + assertThat(tags.getElementAt(0).getString()).isEqualTo("premium"); + + // Verify second document + Variant v2 = results.get(1).getVariant(1); + assertThat(v2.getFieldByName("event").getString()).isEqualTo("purchase"); + Variant props2 = v2.getFieldByName("properties"); + assertThat(props2.getFieldByName("price").getDecimal()) + .isEqualByComparingTo(new java.math.BigDecimal("29.99")); + }); + } + + // -------------------------------------------------------------------------------------------- + // Null variant column values + // -------------------------------------------------------------------------------------------- + + @Test + void testNullVariantColumn() throws IOException { + RowType rowType = DataTypes.ROW(new DataType[] {DataTypes.INT(), DataTypes.VARIANT()}); + + List rows = + Arrays.asList( + GenericRow.of(1, Variant.fromJson("{\"key\":\"value\"}")), + GenericRow.of(2, null), // null column value (not JSON null) + GenericRow.of(3, Variant.fromJson("42"))); + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(3); + assertThat(results.get(0).getVariant(1).isObject()).isTrue(); + assertThat(results.get(1).isNullAt(1)).isTrue(); + assertThat(results.get(2).getVariant(1).getByte()).isEqualTo((byte) 42); + }); + } + + // -------------------------------------------------------------------------------------------- + // Multiple variant columns + // -------------------------------------------------------------------------------------------- + + @Test + void testMultipleVariantColumns() throws IOException { + RowType rowType = + DataTypes.ROW( + new DataType[] {DataTypes.INT(), DataTypes.VARIANT(), DataTypes.VARIANT()}); + + List rows = + Arrays.asList( + GenericRow.of( + 1, + Variant.fromJson("{\"type\":\"a\"}"), + Variant.fromJson("[1,2,3]")), + GenericRow.of( + 2, + Variant.fromJson("{\"type\":\"b\"}"), + Variant.fromJson("\"text\""))); + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(2); + + // Row 1 + assertThat(results.get(0).getVariant(1).getFieldByName("type").getString()) + .isEqualTo("a"); + assertThat(results.get(0).getVariant(2).isArray()).isTrue(); + + // Row 2 + assertThat(results.get(1).getVariant(1).getFieldByName("type").getString()) + .isEqualTo("b"); + assertThat(results.get(1).getVariant(2).getString()).isEqualTo("text"); + }); + } + + // -------------------------------------------------------------------------------------------- + // JSON with special characters + // -------------------------------------------------------------------------------------------- + + @Test + void testJsonWithSpecialCharacters() throws IOException { + RowType rowType = DataTypes.ROW(new DataType[] {DataTypes.VARIANT()}); + + List rows = + Arrays.asList( + GenericRow.of(Variant.fromJson("{\"msg\":\"hello\\nworld\"}")), + GenericRow.of(Variant.fromJson("{\"path\":\"C:\\\\Users\\\\test\"}")), + GenericRow.of(Variant.fromJson("{\"quote\":\"He said \\\"hi\\\"\"}"))); + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(3); + assertThat(results.get(0).getVariant(0).getFieldByName("msg").getString()) + .isEqualTo("hello\nworld"); + assertThat(results.get(1).getVariant(0).getFieldByName("path").getString()) + .isEqualTo("C:\\Users\\test"); + assertThat(results.get(2).getVariant(0).getFieldByName("quote").getString()) + .isEqualTo("He said \"hi\""); + }); + } + + // -------------------------------------------------------------------------------------------- + // Variant binary serialization roundtrip through Arrow + // -------------------------------------------------------------------------------------------- + + @Test + void testVariantBinarySerializationIntegrity() throws IOException { + RowType rowType = DataTypes.ROW(new DataType[] {DataTypes.VARIANT()}); + + // Create variants with factory methods (not from JSON) + List rows = + Arrays.asList( + GenericRow.of(Variant.ofNull()), + GenericRow.of(Variant.ofBoolean(true)), + GenericRow.of(Variant.ofInt(Integer.MAX_VALUE)), + GenericRow.of(Variant.ofLong(Long.MAX_VALUE)), + GenericRow.of(Variant.ofFloat(Float.MAX_VALUE)), + GenericRow.of(Variant.ofDouble(Double.MAX_VALUE)), + GenericRow.of(Variant.ofString("test string"))); + + writeReadAndVerify( + rowType, + rows, + results -> { + assertThat(results).hasSize(7); + assertThat(results.get(0).getVariant(0).isNull()).isTrue(); + assertThat(results.get(1).getVariant(0).getBoolean()).isTrue(); + assertThat(results.get(2).getVariant(0).getInt()).isEqualTo(Integer.MAX_VALUE); + assertThat(results.get(3).getVariant(0).getLong()).isEqualTo(Long.MAX_VALUE); + assertThat(results.get(4).getVariant(0).getFloat()).isEqualTo(Float.MAX_VALUE); + assertThat(results.get(5).getVariant(0).getDouble()) + .isEqualTo(Double.MAX_VALUE); + assertThat(results.get(6).getVariant(0).getString()).isEqualTo("test string"); + }); + } + + // -------------------------------------------------------------------------------------------- + // Helper: write rows to Arrow and read them back, then verify within resource scope + // -------------------------------------------------------------------------------------------- + + private void writeReadAndVerify( + RowType rowType, List rows, Consumer> verifier) + throws IOException { + try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE); + VectorSchemaRoot root = + VectorSchemaRoot.create(ArrowUtils.toArrowSchema(rowType), allocator); + ArrowWriterPool pool = new ArrowWriterPool(allocator); + ArrowWriter writer = + pool.getOrCreateWriter(1L, 1, Integer.MAX_VALUE, rowType, NO_COMPRESSION)) { + + for (InternalRow row : rows) { + writer.writeRow(row); + } + + AbstractPagedOutputView pagedOutputView = + new ManagedPagedOutputView(new TestingMemorySegmentPool(64 * 1024)); + + int size = + writer.serializeToOutputView( + pagedOutputView, recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE)); + int heapMemorySize = Math.max(size, writer.estimatedSizeInBytes()); + MemorySegment segment = MemorySegment.allocateHeapMemory(heapMemorySize); + + MemorySegment firstSegment = pagedOutputView.getCurrentSegment(); + firstSegment.copyTo(recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE), segment, 0, size); + + ArrowReader reader = + ArrowUtils.createArrowReader(segment, 0, size, root, allocator, rowType); + + List results = new ArrayList<>(); + int rowCount = reader.getRowCount(); + for (int i = 0; i < rowCount; i++) { + ColumnarRow row = reader.read(i); + row.setRowId(i); + results.add(row); + } + verifier.accept(results); + } + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/types/variant/ShreddingSchemaInferrerTest.java b/fluss-common/src/test/java/org/apache/fluss/types/variant/ShreddingSchemaInferrerTest.java new file mode 100644 index 0000000000..4b46fba0b0 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/types/variant/ShreddingSchemaInferrerTest.java @@ -0,0 +1,317 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.types.DataTypeRoot; +import org.apache.fluss.types.DataTypes; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link ShreddingSchemaInferrer}. */ +class ShreddingSchemaInferrerTest { + + // -------------------------------------------------------------------------------------------- + // Basic inference + // -------------------------------------------------------------------------------------------- + + @Test + void testInferWithSufficientData() { + // 2000 records, "age" always INT, "name" always STRING + Map stats = new HashMap<>(); + FieldStatistics ageStats = new FieldStatistics("age"); + for (int i = 0; i < 2000; i++) { + ageStats.record(DataTypeRoot.INTEGER); + } + stats.put("age", ageStats); + + FieldStatistics nameStats = new FieldStatistics("name"); + for (int i = 0; i < 1800; i++) { + nameStats.record(DataTypeRoot.STRING); + } + stats.put("name", nameStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer(); + ShreddingSchema schema = inferrer.infer("event", stats, 2000); + + assertThat(schema.getVariantColumnName()).isEqualTo("event"); + assertThat(schema.getFields()).hasSize(2); + + // "age" has presence 1.0, consistency 1.0 => score 1.0 + // "name" has presence 0.9, consistency 1.0 => score 0.9 + // Both should qualify + List fieldPaths = new ArrayList<>(); + for (ShreddedField f : schema.getFields()) { + fieldPaths.add(f.getFieldPath()); + } + assertThat(fieldPaths).contains("age", "name"); + } + + @Test + void testInferInsufficientSampleSize() { + Map stats = new HashMap<>(); + FieldStatistics ageStats = new FieldStatistics("age"); + for (int i = 0; i < 500; i++) { + ageStats.record(DataTypeRoot.INTEGER); + } + stats.put("age", ageStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer(); + // Only 500 records, default min is 1000 + ShreddingSchema schema = inferrer.infer("event", stats, 500); + + assertThat(schema.getFields()).isEmpty(); + } + + @Test + void testInferCustomMinSampleSize() { + Map stats = new HashMap<>(); + FieldStatistics ageStats = new FieldStatistics("age"); + for (int i = 0; i < 100; i++) { + ageStats.record(DataTypeRoot.INTEGER); + } + stats.put("age", ageStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer().setMinSampleSize(50); + ShreddingSchema schema = inferrer.infer("event", stats, 100); + + assertThat(schema.getFields()).hasSize(1); + assertThat(schema.getFields().get(0).getFieldPath()).isEqualTo("age"); + } + + // -------------------------------------------------------------------------------------------- + // Threshold filtering + // -------------------------------------------------------------------------------------------- + + @Test + void testInferPresenceThresholdFiltering() { + Map stats = new HashMap<>(); + + // "age" appears in 30% of records => below default 50% threshold + FieldStatistics ageStats = new FieldStatistics("age"); + for (int i = 0; i < 300; i++) { + ageStats.record(DataTypeRoot.INTEGER); + } + stats.put("age", ageStats); + + // "name" appears in 80% of records => above threshold + FieldStatistics nameStats = new FieldStatistics("name"); + for (int i = 0; i < 800; i++) { + nameStats.record(DataTypeRoot.STRING); + } + stats.put("name", nameStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer().setMinSampleSize(100); + ShreddingSchema schema = inferrer.infer("event", stats, 1000); + + assertThat(schema.getFields()).hasSize(1); + assertThat(schema.getFields().get(0).getFieldPath()).isEqualTo("name"); + } + + @Test + void testInferTypeConsistencyFiltering() { + Map stats = new HashMap<>(); + + // "value" has 50% INT and 50% STRING => consistency = 0.5 < 0.9 threshold + FieldStatistics valueStats = new FieldStatistics("value"); + for (int i = 0; i < 500; i++) { + valueStats.record(DataTypeRoot.INTEGER); + } + for (int i = 0; i < 500; i++) { + valueStats.record(DataTypeRoot.STRING); + } + stats.put("value", valueStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer().setMinSampleSize(100); + ShreddingSchema schema = inferrer.infer("event", stats, 1000); + + assertThat(schema.getFields()).isEmpty(); + } + + // -------------------------------------------------------------------------------------------- + // Max fields limiting + // -------------------------------------------------------------------------------------------- + + @Test + void testInferMaxFieldsLimiting() { + Map stats = new HashMap<>(); + for (int i = 0; i < 10; i++) { + FieldStatistics fs = new FieldStatistics("field" + i); + for (int j = 0; j < 1000; j++) { + fs.record(DataTypeRoot.INTEGER); + } + stats.put("field" + i, fs); + } + + ShreddingSchemaInferrer inferrer = + new ShreddingSchemaInferrer().setMinSampleSize(100).setMaxShreddedFields(3); + ShreddingSchema schema = inferrer.infer("event", stats, 1000); + + assertThat(schema.getFields()).hasSize(3); + } + + // -------------------------------------------------------------------------------------------- + // Type mapping + // -------------------------------------------------------------------------------------------- + + @Test + void testInferTypeMapping() { + Map stats = new HashMap<>(); + + FieldStatistics boolStats = new FieldStatistics("flag"); + for (int i = 0; i < 1000; i++) { + boolStats.record(DataTypeRoot.BOOLEAN); + } + stats.put("flag", boolStats); + + FieldStatistics doubleStats = new FieldStatistics("ratio"); + for (int i = 0; i < 1000; i++) { + doubleStats.record(DataTypeRoot.DOUBLE); + } + stats.put("ratio", doubleStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer().setMinSampleSize(100); + ShreddingSchema schema = inferrer.infer("event", stats, 1000); + + assertThat(schema.getFields()).hasSize(2); + for (ShreddedField field : schema.getFields()) { + if (field.getFieldPath().equals("flag")) { + assertThat(field.getShreddedType()).isEqualTo(DataTypes.BOOLEAN()); + } else if (field.getFieldPath().equals("ratio")) { + assertThat(field.getShreddedType()).isEqualTo(DataTypes.DOUBLE()); + } + } + } + + // -------------------------------------------------------------------------------------------- + // Incremental update + // -------------------------------------------------------------------------------------------- + + @Test + void testUpdateAddNewFields() { + // Start with schema having "age" shredded + ShreddingSchema current = + new ShreddingSchema( + "event", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + // New stats show "age" still qualifies + "name" now qualifies + Map newStats = new HashMap<>(); + FieldStatistics ageStats = new FieldStatistics("age"); + for (int i = 0; i < 1000; i++) { + ageStats.record(DataTypeRoot.INTEGER); + } + newStats.put("age", ageStats); + + FieldStatistics nameStats = new FieldStatistics("name"); + for (int i = 0; i < 800; i++) { + nameStats.record(DataTypeRoot.STRING); + } + newStats.put("name", nameStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer().setMinSampleSize(100); + Optional updated = inferrer.update(current, newStats, 1000); + + assertThat(updated).isPresent(); + assertThat(updated.get().getFields()).hasSize(2); + } + + @Test + void testUpdateNoChanges() { + ShreddingSchema current = + new ShreddingSchema( + "event", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + // "age" still qualifies, no new fields + Map newStats = new HashMap<>(); + FieldStatistics ageStats = new FieldStatistics("age"); + for (int i = 0; i < 1000; i++) { + ageStats.record(DataTypeRoot.INTEGER); + } + newStats.put("age", ageStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer().setMinSampleSize(100); + Optional updated = inferrer.update(current, newStats, 1000); + + assertThat(updated).isEmpty(); + } + + @Test + void testUpdateDemoteField() { + ShreddingSchema current = + new ShreddingSchema( + "event", + Arrays.asList( + new ShreddedField("age", DataTypes.INT()), + new ShreddedField("name", DataTypes.STRING()))); + + // "name" no longer in stats => will be demoted + Map newStats = new HashMap<>(); + FieldStatistics ageStats = new FieldStatistics("age"); + for (int i = 0; i < 1000; i++) { + ageStats.record(DataTypeRoot.INTEGER); + } + newStats.put("age", ageStats); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer().setMinSampleSize(100); + Optional updated = inferrer.update(current, newStats, 1000); + + assertThat(updated).isPresent(); + assertThat(updated.get().getFields()).hasSize(1); + assertThat(updated.get().getFields().get(0).getFieldPath()).isEqualTo("age"); + } + + @Test + void testUpdateInsufficientSamples() { + ShreddingSchema current = + new ShreddingSchema( + "event", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + Map newStats = new HashMap<>(); + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer(); + Optional updated = inferrer.update(current, newStats, 500); + + assertThat(updated).isEmpty(); + } + + // -------------------------------------------------------------------------------------------- + // Getters/Setters + // -------------------------------------------------------------------------------------------- + + @Test + void testGettersSetters() { + ShreddingSchemaInferrer inferrer = + new ShreddingSchemaInferrer() + .setPresenceThreshold(0.7f) + .setTypeConsistencyThreshold(0.95f) + .setMaxShreddedFields(50) + .setMinSampleSize(500); + + assertThat(inferrer.getPresenceThreshold()).isEqualTo(0.7f); + assertThat(inferrer.getTypeConsistencyThreshold()).isEqualTo(0.95f); + assertThat(inferrer.getMaxShreddedFields()).isEqualTo(50); + assertThat(inferrer.getMinSampleSize()).isEqualTo(500); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/types/variant/ShreddingSchemaTest.java b/fluss-common/src/test/java/org/apache/fluss/types/variant/ShreddingSchemaTest.java new file mode 100644 index 0000000000..007be0df9e --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/types/variant/ShreddingSchemaTest.java @@ -0,0 +1,109 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.IntType; +import org.apache.fluss.types.StringType; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link ShreddingSchema}. */ +class ShreddingSchemaTest { + + @Test + void testGetters() { + ShreddedField field1 = new ShreddedField("age", new IntType()); + ShreddedField field2 = new ShreddedField("name", new StringType()); + ShreddingSchema schema = new ShreddingSchema("data", Arrays.asList(field1, field2)); + + assertThat(schema.getVariantColumnName()).isEqualTo("data"); + assertThat(schema.getFields()).hasSize(2); + assertThat(schema.getFields().get(0).getFieldPath()).isEqualTo("age"); + assertThat(schema.getFields().get(1).getFieldPath()).isEqualTo("name"); + } + + @Test + void testToJsonFromJsonRoundtrip() { + ShreddingSchema original = + new ShreddingSchema( + "event", + Arrays.asList( + new ShreddedField("age", DataTypes.INT()), + new ShreddedField("name", DataTypes.STRING()), + new ShreddedField("active", DataTypes.BOOLEAN()))); + + String json = original.toJson(); + assertThat(json).contains("\"variant_column\":\"event\""); + assertThat(json).contains("\"path\":\"age\""); + assertThat(json).contains("\"path\":\"name\""); + assertThat(json).contains("\"path\":\"active\""); + + ShreddingSchema deserialized = ShreddingSchema.fromJson(json); + assertThat(deserialized.getVariantColumnName()).isEqualTo("event"); + assertThat(deserialized.getFields()).hasSize(3); + + assertThat(deserialized.getFields().get(0).getFieldPath()).isEqualTo("age"); + assertThat(deserialized.getFields().get(1).getFieldPath()).isEqualTo("name"); + assertThat(deserialized.getFields().get(2).getFieldPath()).isEqualTo("active"); + } + + @Test + void testToJsonFromJsonEmptyFields() { + ShreddingSchema original = new ShreddingSchema("data", Collections.emptyList()); + + String json = original.toJson(); + ShreddingSchema deserialized = ShreddingSchema.fromJson(json); + + assertThat(deserialized.getVariantColumnName()).isEqualTo("data"); + assertThat(deserialized.getFields()).isEmpty(); + } + + @Test + void testEqualsAndHashCode() { + ShreddingSchema s1 = + new ShreddingSchema( + "event", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + ShreddingSchema s2 = + new ShreddingSchema( + "event", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + ShreddingSchema s3 = + new ShreddingSchema( + "other", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + assertThat(s1).isEqualTo(s2); + assertThat(s1.hashCode()).isEqualTo(s2.hashCode()); + assertThat(s1).isNotEqualTo(s3); + } + + @Test + void testFieldsListIsUnmodifiable() { + ShreddingSchema schema = + new ShreddingSchema( + "event", Arrays.asList(new ShreddedField("age", DataTypes.INT()))); + + org.assertj.core.api.Assertions.assertThatThrownBy( + () -> schema.getFields().add(new ShreddedField("x", DataTypes.INT()))) + .isInstanceOf(UnsupportedOperationException.class); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantStatisticsCollectorTest.java b/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantStatisticsCollectorTest.java new file mode 100644 index 0000000000..6e80a2a42d --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantStatisticsCollectorTest.java @@ -0,0 +1,250 @@ +/* + * 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.fluss.types.variant; + +import org.apache.fluss.types.DataTypeRoot; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link VariantStatisticsCollector}. */ +class VariantStatisticsCollectorTest { + + // -------------------------------------------------------------------------------------------- + // Single record collection + // -------------------------------------------------------------------------------------------- + + @Test + void testCollectSingleObjectRecord() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + + Variant v = Variant.fromJson("{\"name\":\"Alice\",\"age\":30,\"active\":true}"); + collector.collect(v); + + assertThat(collector.getTotalRecords()).isEqualTo(1); + Map stats = collector.getStatistics(); + assertThat(stats).containsKey("name"); + assertThat(stats).containsKey("age"); + assertThat(stats).containsKey("active"); + + assertThat(stats.get("name").getPresenceCount()).isEqualTo(1); + assertThat(stats.get("active").dominantType()).isEqualTo(DataTypeRoot.BOOLEAN); + } + + @Test + void testCollectNullVariant() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + collector.collect(null); + + assertThat(collector.getTotalRecords()).isEqualTo(1); + assertThat(collector.getStatistics()).isEmpty(); + } + + @Test + void testCollectNullValueVariant() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + collector.collect(Variant.ofNull()); + + assertThat(collector.getTotalRecords()).isEqualTo(1); + assertThat(collector.getStatistics()).isEmpty(); + } + + @Test + void testCollectNonObjectVariant() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + collector.collect(Variant.ofInt(42)); + + assertThat(collector.getTotalRecords()).isEqualTo(1); + assertThat(collector.getStatistics()).isEmpty(); + } + + // -------------------------------------------------------------------------------------------- + // Batch collection + // -------------------------------------------------------------------------------------------- + + @Test + void testCollectBatch() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + + collector.collectBatch( + Arrays.asList( + Variant.fromJson("{\"name\":\"Alice\",\"age\":30}"), + Variant.fromJson("{\"name\":\"Bob\",\"age\":25}"), + Variant.fromJson("{\"name\":\"Charlie\"}"))); + + assertThat(collector.getTotalRecords()).isEqualTo(3); + Map stats = collector.getStatistics(); + + assertThat(stats.get("name").getPresenceCount()).isEqualTo(3); + assertThat(stats.get("name").presenceRatio(3)).isEqualTo(1.0f); + assertThat(stats.get("age").getPresenceCount()).isEqualTo(2); + } + + // -------------------------------------------------------------------------------------------- + // Type inference + // -------------------------------------------------------------------------------------------- + + @Test + void testTypeInferenceFromJsonValues() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + + collector.collect(Variant.fromJson("{\"val\":42}")); + collector.collect(Variant.fromJson("{\"val\":100}")); + + Map stats = collector.getStatistics(); + FieldStatistics valStats = stats.get("val"); + assertThat(valStats.getPresenceCount()).isEqualTo(2); + // JSON parser encodes small integers as INT8 + assertThat(valStats.typeConsistency()).isEqualTo(1.0f); + } + + @Test + void testMixedTypeTracking() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + + // "value" field has different types across records + collector.collect(Variant.fromJson("{\"value\":42}")); + collector.collect(Variant.fromJson("{\"value\":\"hello\"}")); + collector.collect(Variant.fromJson("{\"value\":true}")); + + Map stats = collector.getStatistics(); + FieldStatistics valueStats = stats.get("value"); + assertThat(valueStats.getPresenceCount()).isEqualTo(3); + // Three different types => consistency should be 1/3 + assertThat(valueStats.typeConsistency()).isLessThan(0.5f); + } + + // -------------------------------------------------------------------------------------------- + // Merge + // -------------------------------------------------------------------------------------------- + + @Test + void testMerge() { + VariantStatisticsCollector collector1 = new VariantStatisticsCollector(); + collector1.collect(Variant.fromJson("{\"name\":\"Alice\",\"age\":30}")); + collector1.collect(Variant.fromJson("{\"name\":\"Bob\",\"age\":25}")); + + VariantStatisticsCollector collector2 = new VariantStatisticsCollector(); + collector2.collect(Variant.fromJson("{\"name\":\"Charlie\",\"age\":35}")); + collector2.collect(Variant.fromJson("{\"name\":\"Diana\"}")); + + collector1.merge(collector2); + + assertThat(collector1.getTotalRecords()).isEqualTo(4); + Map stats = collector1.getStatistics(); + assertThat(stats.get("name").getPresenceCount()).isEqualTo(4); + assertThat(stats.get("age").getPresenceCount()).isEqualTo(3); + } + + @Test + void testMergeEmptyCollectors() { + VariantStatisticsCollector collector1 = new VariantStatisticsCollector(); + VariantStatisticsCollector collector2 = new VariantStatisticsCollector(); + + collector1.merge(collector2); + assertThat(collector1.getTotalRecords()).isZero(); + assertThat(collector1.getStatistics()).isEmpty(); + } + + @Test + void testMergeWithDisjointFields() { + VariantStatisticsCollector collector1 = new VariantStatisticsCollector(); + collector1.collect(Variant.fromJson("{\"field_a\":1}")); + + VariantStatisticsCollector collector2 = new VariantStatisticsCollector(); + collector2.collect(Variant.fromJson("{\"field_b\":2}")); + + collector1.merge(collector2); + + assertThat(collector1.getTotalRecords()).isEqualTo(2); + assertThat(collector1.getStatistics()).containsKey("field_a"); + assertThat(collector1.getStatistics()).containsKey("field_b"); + } + + // -------------------------------------------------------------------------------------------- + // Reset + // -------------------------------------------------------------------------------------------- + + @Test + void testReset() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + collector.collect(Variant.fromJson("{\"name\":\"Alice\"}")); + assertThat(collector.getTotalRecords()).isEqualTo(1); + + collector.reset(); + assertThat(collector.getTotalRecords()).isZero(); + assertThat(collector.getStatistics()).isEmpty(); + } + + // -------------------------------------------------------------------------------------------- + // End-to-end with ShreddingSchemaInferrer + // -------------------------------------------------------------------------------------------- + + @Test + void testEndToEndWithInferrer() { + VariantStatisticsCollector collector = new VariantStatisticsCollector(); + + // Simulate collecting 1500 records + for (int i = 0; i < 1500; i++) { + if (i % 3 == 0) { + // Every 3rd record has "status" field + collector.collect( + Variant.fromJson( + "{\"name\":\"user" + + i + + "\",\"age\":" + + (20 + i % 50) + + ",\"status\":\"active\"}")); + } else { + collector.collect( + Variant.fromJson( + "{\"name\":\"user" + i + "\",\"age\":" + (20 + i % 50) + "}")); + } + } + + assertThat(collector.getTotalRecords()).isEqualTo(1500); + + ShreddingSchemaInferrer inferrer = new ShreddingSchemaInferrer(); + ShreddingSchema schema = + inferrer.infer("data", collector.getStatistics(), collector.getTotalRecords()); + + // "name" and "age" appear in 100% of records => should be shredded + // "status" appears in ~33% of records => below 50% threshold => not shredded + boolean hasName = false; + boolean hasAge = false; + boolean hasStatus = false; + for (ShreddedField field : schema.getFields()) { + if (field.getFieldPath().equals("name")) { + hasName = true; + } + if (field.getFieldPath().equals("age")) { + hasAge = true; + } + if (field.getFieldPath().equals("status")) { + hasStatus = true; + } + } + assertThat(hasName).isTrue(); + assertThat(hasAge).isTrue(); + assertThat(hasStatus).isFalse(); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantTest.java b/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantTest.java new file mode 100644 index 0000000000..673497fd79 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantTest.java @@ -0,0 +1,364 @@ +/* + * 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.fluss.types.variant; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link Variant}. */ +class VariantTest { + + // -------------------------------------------------------------------------------------------- + // Factory methods + // -------------------------------------------------------------------------------------------- + + @Test + void testOfNull() { + Variant v = Variant.ofNull(); + assertThat(v.isNull()).isTrue(); + assertThat(v.isPrimitive()).isTrue(); + assertThat(v.isObject()).isFalse(); + assertThat(v.isArray()).isFalse(); + assertThat(v.toJson()).isEqualTo("null"); + } + + @Test + void testOfBoolean() { + Variant vTrue = Variant.ofBoolean(true); + assertThat(vTrue.getBoolean()).isTrue(); + assertThat(vTrue.isPrimitive()).isTrue(); + assertThat(vTrue.toJson()).isEqualTo("true"); + + Variant vFalse = Variant.ofBoolean(false); + assertThat(vFalse.getBoolean()).isFalse(); + assertThat(vFalse.toJson()).isEqualTo("false"); + } + + @Test + void testOfInt() { + Variant v = Variant.ofInt(42); + assertThat(v.getInt()).isEqualTo(42); + assertThat(v.isPrimitive()).isTrue(); + assertThat(v.toJson()).isEqualTo("42"); + } + + @Test + void testOfLong() { + Variant v = Variant.ofLong(9876543210L); + assertThat(v.getLong()).isEqualTo(9876543210L); + assertThat(v.toJson()).isEqualTo("9876543210"); + } + + @Test + void testOfFloat() { + Variant v = Variant.ofFloat(3.14f); + assertThat(v.getFloat()).isEqualTo(3.14f); + } + + @Test + void testOfDouble() { + Variant v = Variant.ofDouble(2.718281828); + assertThat(v.getDouble()).isEqualTo(2.718281828); + } + + @Test + void testOfString() { + Variant v = Variant.ofString("hello world"); + assertThat(v.getString()).isEqualTo("hello world"); + assertThat(v.isShortString()).isTrue(); + assertThat(v.toJson()).isEqualTo("\"hello world\""); + } + + // -------------------------------------------------------------------------------------------- + // Binary serialization roundtrip + // -------------------------------------------------------------------------------------------- + + @Test + void testToBytesFromBytesRoundtrip() { + Variant original = Variant.ofInt(42); + byte[] bytes = original.toBytes(); + Variant deserialized = Variant.fromBytes(bytes); + assertThat(deserialized).isEqualTo(original); + assertThat(deserialized.getInt()).isEqualTo(42); + } + + @Test + void testToBytesFromBytesWithOffset() { + Variant original = Variant.ofString("test"); + byte[] bytes = original.toBytes(); + + // Put the bytes at an offset in a larger array + byte[] padded = new byte[bytes.length + 10]; + System.arraycopy(bytes, 0, padded, 5, bytes.length); + + Variant deserialized = Variant.fromBytes(padded, 5, bytes.length); + assertThat(deserialized).isEqualTo(original); + } + + @Test + void testToBytesFromBytesNull() { + Variant original = Variant.ofNull(); + byte[] bytes = original.toBytes(); + Variant deserialized = Variant.fromBytes(bytes); + assertThat(deserialized.isNull()).isTrue(); + } + + // -------------------------------------------------------------------------------------------- + // JSON conversion + // -------------------------------------------------------------------------------------------- + + @Test + void testFromJsonNull() { + Variant v = Variant.fromJson("null"); + assertThat(v.isNull()).isTrue(); + } + + @Test + void testFromJsonBoolean() { + Variant v1 = Variant.fromJson("true"); + assertThat(v1.getBoolean()).isTrue(); + + Variant v2 = Variant.fromJson("false"); + assertThat(v2.getBoolean()).isFalse(); + } + + @Test + void testFromJsonInteger() { + Variant v = Variant.fromJson("42"); + // Small integers are encoded as int8 + assertThat(v.getByte()).isEqualTo((byte) 42); + } + + @Test + void testFromJsonLargeInteger() { + Variant v = Variant.fromJson("1000000"); + assertThat(v.getInt()).isEqualTo(1000000); + } + + @Test + void testFromJsonNegativeInteger() { + Variant v = Variant.fromJson("-128"); + assertThat(v.getByte()).isEqualTo((byte) -128); + } + + @Test + void testFromJsonDecimal() { + // Decimal point without exponent is encoded as Decimal (matching Paimon behavior) + Variant v = Variant.fromJson("3.14"); + assertThat(v.getDecimal()).isEqualByComparingTo(new java.math.BigDecimal("3.14")); + } + + @Test + void testFromJsonDouble() { + // Exponent notation is encoded as Double + Variant v = Variant.fromJson("3.14e0"); + assertThat(v.getDouble()).isEqualTo(3.14); + } + + @Test + void testFromJsonString() { + Variant v = Variant.fromJson("\"hello world\""); + assertThat(v.getString()).isEqualTo("hello world"); + } + + @Test + void testFromJsonStringWithEscapes() { + Variant v = Variant.fromJson("\"line1\\nline2\""); + assertThat(v.getString()).isEqualTo("line1\nline2"); + } + + @Test + void testFromJsonSimpleObject() { + Variant v = Variant.fromJson("{\"name\":\"Alice\",\"age\":30}"); + assertThat(v.isObject()).isTrue(); + assertThat(v.objectSize()).isEqualTo(2); + + Variant name = v.getFieldByName("name"); + assertThat(name).isNotNull(); + assertThat(name.getString()).isEqualTo("Alice"); + + Variant age = v.getFieldByName("age"); + assertThat(age).isNotNull(); + assertThat(age.getByte()).isEqualTo((byte) 30); + } + + @Test + void testFromJsonNestedObject() { + Variant v = Variant.fromJson("{\"user\":{\"name\":\"Bob\",\"active\":true},\"count\":100}"); + assertThat(v.isObject()).isTrue(); + + Variant user = v.getFieldByName("user"); + assertThat(user).isNotNull(); + assertThat(user.isObject()).isTrue(); + + Variant name = user.getFieldByName("name"); + assertThat(name).isNotNull(); + assertThat(name.getString()).isEqualTo("Bob"); + + Variant active = user.getFieldByName("active"); + assertThat(active).isNotNull(); + assertThat(active.getBoolean()).isTrue(); + } + + @Test + void testFromJsonArray() { + Variant v = Variant.fromJson("[1, 2, 3]"); + assertThat(v.isArray()).isTrue(); + assertThat(v.arraySize()).isEqualTo(3); + assertThat(v.getElementAt(0).getByte()).isEqualTo((byte) 1); + assertThat(v.getElementAt(1).getByte()).isEqualTo((byte) 2); + assertThat(v.getElementAt(2).getByte()).isEqualTo((byte) 3); + } + + @Test + void testFromJsonMixedArray() { + Variant v = Variant.fromJson("[1, \"hello\", true, null]"); + assertThat(v.isArray()).isTrue(); + assertThat(v.arraySize()).isEqualTo(4); + assertThat(v.getElementAt(0).getByte()).isEqualTo((byte) 1); + assertThat(v.getElementAt(1).getString()).isEqualTo("hello"); + assertThat(v.getElementAt(2).getBoolean()).isTrue(); + assertThat(v.getElementAt(3).isNull()).isTrue(); + } + + @Test + void testFromJsonEmptyObject() { + Variant v = Variant.fromJson("{}"); + assertThat(v.isObject()).isTrue(); + assertThat(v.objectSize()).isZero(); + } + + @Test + void testFromJsonEmptyArray() { + Variant v = Variant.fromJson("[]"); + assertThat(v.isArray()).isTrue(); + assertThat(v.arraySize()).isZero(); + } + + // -------------------------------------------------------------------------------------------- + // JSON roundtrip + // -------------------------------------------------------------------------------------------- + + @Test + void testJsonRoundtripPrimitives() { + assertJsonRoundtrip("null"); + assertJsonRoundtrip("true"); + assertJsonRoundtrip("false"); + assertJsonRoundtrip("3.14"); + assertJsonRoundtrip("\"hello\""); + } + + @Test + void testJsonRoundtripObject() { + Variant v = Variant.fromJson("{\"a\":1,\"b\":\"test\"}"); + String json = v.toJson(); + // Re-parse and verify + Variant v2 = Variant.fromJson(json); + assertThat(v2.getFieldByName("a")).isNotNull(); + assertThat(v2.getFieldByName("b")).isNotNull(); + assertThat(v2.getFieldByName("b").getString()).isEqualTo("test"); + } + + @Test + void testJsonRoundtripArray() { + Variant v = Variant.fromJson("[1,2,3]"); + String json = v.toJson(); + Variant v2 = Variant.fromJson(json); + assertThat(v2.isArray()).isTrue(); + assertThat(v2.arraySize()).isEqualTo(3); + } + + // -------------------------------------------------------------------------------------------- + // Object field access + // -------------------------------------------------------------------------------------------- + + @Test + void testGetFieldByNameNonExistent() { + Variant v = Variant.fromJson("{\"name\":\"Alice\"}"); + assertThat(v.getFieldByName("unknown")).isNull(); + } + + // -------------------------------------------------------------------------------------------- + // Equals & HashCode + // -------------------------------------------------------------------------------------------- + + @Test + void testEquals() { + Variant v1 = Variant.ofInt(42); + Variant v2 = Variant.ofInt(42); + assertThat(v1).isEqualTo(v2); + assertThat(v1.hashCode()).isEqualTo(v2.hashCode()); + } + + @Test + void testNotEquals() { + Variant v1 = Variant.ofInt(42); + Variant v2 = Variant.ofInt(43); + assertThat(v1).isNotEqualTo(v2); + } + + // -------------------------------------------------------------------------------------------- + // toString + // -------------------------------------------------------------------------------------------- + + @Test + void testToString() { + assertThat(Variant.ofNull().toString()).isEqualTo("null"); + assertThat(Variant.ofInt(42).toString()).isEqualTo("42"); + assertThat(Variant.ofString("hi").toString()).isEqualTo("\"hi\""); + } + + // -------------------------------------------------------------------------------------------- + // Complex JSON roundtrip + // -------------------------------------------------------------------------------------------- + + @Test + void testComplexJsonRoundtrip() { + String json = + "{\"users\":[{\"name\":\"Alice\",\"age\":30},{\"name\":\"Bob\",\"age\":25}]," + + "\"count\":2,\"active\":true}"; + Variant v = Variant.fromJson(json); + assertThat(v.isObject()).isTrue(); + + Variant users = v.getFieldByName("users"); + assertThat(users).isNotNull(); + assertThat(users.isArray()).isTrue(); + assertThat(users.arraySize()).isEqualTo(2); + + Variant firstUser = users.getElementAt(0); + assertThat(firstUser.getFieldByName("name").getString()).isEqualTo("Alice"); + } + + @Test + void testJsonWithWhitespace() { + String json = " { \"key\" : \"value\" , \"num\" : 42 } "; + Variant v = Variant.fromJson(json); + assertThat(v.isObject()).isTrue(); + assertThat(v.getFieldByName("key").getString()).isEqualTo("value"); + } + + // -------------------------------------------------------------------------------------------- + // Helper + // -------------------------------------------------------------------------------------------- + + private void assertJsonRoundtrip(String json) { + Variant v = Variant.fromJson(json); + assertThat(v.toJson()).isEqualTo(json); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantUtilTest.java b/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantUtilTest.java new file mode 100644 index 0000000000..d93d08a017 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/types/variant/VariantUtilTest.java @@ -0,0 +1,398 @@ +/* + * 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.fluss.types.variant; + +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link VariantUtil}. */ +class VariantUtilTest { + + // -------------------------------------------------------------------------------------------- + // Null encoding/decoding + // -------------------------------------------------------------------------------------------- + + @Test + void testEncodeDecodeNull() { + byte[] encoded = VariantUtil.encodeNull(); + assertThat(encoded).hasSize(1); + assertThat(VariantUtil.isNull(encoded, 0)).isTrue(); + assertThat(VariantUtil.basicType(encoded, 0)).isEqualTo(VariantUtil.BASIC_TYPE_PRIMITIVE); + assertThat(VariantUtil.primitiveTypeId(encoded, 0)) + .isEqualTo(VariantUtil.PRIMITIVE_TYPE_NULL); + } + + // -------------------------------------------------------------------------------------------- + // Boolean encoding/decoding + // -------------------------------------------------------------------------------------------- + + @Test + void testEncodeBooleanTrue() { + byte[] encoded = VariantUtil.encodeBoolean(true); + assertThat(encoded).hasSize(1); + assertThat(VariantUtil.getBoolean(encoded, 0)).isTrue(); + assertThat(VariantUtil.isNull(encoded, 0)).isFalse(); + } + + @Test + void testEncodeBooleanFalse() { + byte[] encoded = VariantUtil.encodeBoolean(false); + assertThat(encoded).hasSize(1); + assertThat(VariantUtil.getBoolean(encoded, 0)).isFalse(); + } + + // -------------------------------------------------------------------------------------------- + // Integer type encoding/decoding + // -------------------------------------------------------------------------------------------- + + @Test + void testEncodeByte() { + byte[] encoded = VariantUtil.encodeByte((byte) 42); + assertThat(encoded).hasSize(2); + assertThat(VariantUtil.getByte(encoded, 0)).isEqualTo((byte) 42); + } + + @Test + void testEncodeByteNegative() { + byte[] encoded = VariantUtil.encodeByte((byte) -128); + assertThat(VariantUtil.getByte(encoded, 0)).isEqualTo((byte) -128); + } + + @Test + void testEncodeShort() { + byte[] encoded = VariantUtil.encodeShort((short) 12345); + assertThat(encoded).hasSize(3); + assertThat(VariantUtil.getShort(encoded, 0)).isEqualTo((short) 12345); + } + + @Test + void testEncodeShortNegative() { + byte[] encoded = VariantUtil.encodeShort((short) -32000); + assertThat(VariantUtil.getShort(encoded, 0)).isEqualTo((short) -32000); + } + + @Test + void testEncodeInt() { + byte[] encoded = VariantUtil.encodeInt(1234567); + assertThat(encoded).hasSize(5); + assertThat(VariantUtil.getInt(encoded, 0)).isEqualTo(1234567); + } + + @Test + void testEncodeIntNegative() { + byte[] encoded = VariantUtil.encodeInt(Integer.MIN_VALUE); + assertThat(VariantUtil.getInt(encoded, 0)).isEqualTo(Integer.MIN_VALUE); + } + + @Test + void testEncodeLong() { + byte[] encoded = VariantUtil.encodeLong(9876543210L); + assertThat(encoded).hasSize(9); + assertThat(VariantUtil.getLong(encoded, 0)).isEqualTo(9876543210L); + } + + @Test + void testEncodeLongNegative() { + byte[] encoded = VariantUtil.encodeLong(Long.MIN_VALUE); + assertThat(VariantUtil.getLong(encoded, 0)).isEqualTo(Long.MIN_VALUE); + } + + // -------------------------------------------------------------------------------------------- + // Float/Double encoding/decoding + // -------------------------------------------------------------------------------------------- + + @Test + void testEncodeFloat() { + byte[] encoded = VariantUtil.encodeFloat(3.14f); + assertThat(encoded).hasSize(5); + assertThat(VariantUtil.getFloat(encoded, 0)).isEqualTo(3.14f); + } + + @Test + void testEncodeDouble() { + byte[] encoded = VariantUtil.encodeDouble(3.14159265358979); + assertThat(encoded).hasSize(9); + assertThat(VariantUtil.getDouble(encoded, 0)).isEqualTo(3.14159265358979); + } + + // -------------------------------------------------------------------------------------------- + // String encoding/decoding + // -------------------------------------------------------------------------------------------- + + @Test + void testEncodeShortString() { + // String <= 31 bytes uses short string encoding + byte[] strBytes = "hello".getBytes(StandardCharsets.UTF_8); + byte[] encoded = VariantUtil.encodeString(strBytes); + assertThat(VariantUtil.basicType(encoded, 0)) + .isEqualTo(VariantUtil.BASIC_TYPE_SHORT_STRING); + assertThat(VariantUtil.getString(encoded, 0)).isEqualTo("hello"); + } + + @Test + void testEncodeLongString() { + // String > 31 bytes uses primitive STRING encoding + String longStr = "This is a long string that exceeds 31 bytes for testing purposes!!!"; + byte[] strBytes = longStr.getBytes(StandardCharsets.UTF_8); + assertThat(strBytes.length).isGreaterThan(31); + + byte[] encoded = VariantUtil.encodeString(strBytes); + assertThat(VariantUtil.basicType(encoded, 0)).isEqualTo(VariantUtil.BASIC_TYPE_PRIMITIVE); + assertThat(VariantUtil.primitiveTypeId(encoded, 0)) + .isEqualTo(VariantUtil.PRIMITIVE_TYPE_STRING); + assertThat(VariantUtil.getString(encoded, 0)).isEqualTo(longStr); + } + + @Test + void testEncodeEmptyString() { + byte[] strBytes = "".getBytes(StandardCharsets.UTF_8); + byte[] encoded = VariantUtil.encodeString(strBytes); + assertThat(VariantUtil.getString(encoded, 0)).isEmpty(); + } + + @Test + void testEncodeUnicodeString() { + String unicodeStr = "你好世界🌍"; + byte[] strBytes = unicodeStr.getBytes(StandardCharsets.UTF_8); + byte[] encoded = VariantUtil.encodeString(strBytes); + assertThat(VariantUtil.getString(encoded, 0)).isEqualTo(unicodeStr); + } + + // -------------------------------------------------------------------------------------------- + // Date encoding/decoding + // -------------------------------------------------------------------------------------------- + + @Test + void testEncodeDate() { + byte[] encoded = VariantUtil.encodeDate(18000); + assertThat(encoded).hasSize(5); + assertThat(VariantUtil.getDate(encoded, 0)).isEqualTo(18000); + } + + // -------------------------------------------------------------------------------------------- + // Object encoding/decoding + // -------------------------------------------------------------------------------------------- + + @Test + void testEncodeEmptyObject() { + byte[] encoded = VariantUtil.encodeObject(new ArrayList<>(), new ArrayList<>()); + assertThat(VariantUtil.basicType(encoded, 0)).isEqualTo(VariantUtil.BASIC_TYPE_OBJECT); + assertThat(VariantUtil.objectSize(encoded, 0)).isZero(); + } + + @Test + void testEncodeObjectWithFields() { + List fieldIds = Arrays.asList(0, 1, 2); + List fieldValues = + Arrays.asList( + VariantUtil.encodeInt(42), + VariantUtil.encodeBoolean(true), + VariantUtil.encodeString("test".getBytes(StandardCharsets.UTF_8))); + + byte[] encoded = VariantUtil.encodeObject(fieldIds, fieldValues); + assertThat(VariantUtil.basicType(encoded, 0)).isEqualTo(VariantUtil.BASIC_TYPE_OBJECT); + assertThat(VariantUtil.objectSize(encoded, 0)).isEqualTo(3); + } + + @Test + void testObjectFieldAccess() { + List fieldIds = Arrays.asList(0, 1); + List fieldValues = + Arrays.asList(VariantUtil.encodeInt(100), VariantUtil.encodeDouble(3.14)); + + byte[] encoded = VariantUtil.encodeObject(fieldIds, fieldValues); + + // Field 0 should be int 100 + int field0Id = VariantUtil.objectFieldId(encoded, 0, 0); + assertThat(field0Id).isEqualTo(0); + int field0Offset = VariantUtil.objectFieldValueOffset(encoded, 0, 0); + assertThat(VariantUtil.getInt(encoded, field0Offset)).isEqualTo(100); + + // Field 1 should be double 3.14 + int field1Id = VariantUtil.objectFieldId(encoded, 0, 1); + assertThat(field1Id).isEqualTo(1); + int field1Offset = VariantUtil.objectFieldValueOffset(encoded, 0, 1); + assertThat(VariantUtil.getDouble(encoded, field1Offset)).isEqualTo(3.14); + } + + // -------------------------------------------------------------------------------------------- + // Value size calculation + // -------------------------------------------------------------------------------------------- + + @Test + void testValueSizePrimitive() { + assertThat(VariantUtil.valueSize(VariantUtil.encodeNull(), 0)).isEqualTo(1); + assertThat(VariantUtil.valueSize(VariantUtil.encodeBoolean(true), 0)).isEqualTo(1); + assertThat(VariantUtil.valueSize(VariantUtil.encodeByte((byte) 1), 0)).isEqualTo(2); + assertThat(VariantUtil.valueSize(VariantUtil.encodeShort((short) 1), 0)).isEqualTo(3); + assertThat(VariantUtil.valueSize(VariantUtil.encodeInt(1), 0)).isEqualTo(5); + assertThat(VariantUtil.valueSize(VariantUtil.encodeLong(1L), 0)).isEqualTo(9); + assertThat(VariantUtil.valueSize(VariantUtil.encodeFloat(1.0f), 0)).isEqualTo(5); + assertThat(VariantUtil.valueSize(VariantUtil.encodeDouble(1.0), 0)).isEqualTo(9); + assertThat(VariantUtil.valueSize(VariantUtil.encodeDate(1), 0)).isEqualTo(5); + } + + @Test + void testValueSizeShortString() { + byte[] encoded = VariantUtil.encodeString("hi".getBytes(StandardCharsets.UTF_8)); + // short string: 1 header + 2 bytes + assertThat(VariantUtil.valueSize(encoded, 0)).isEqualTo(3); + } + + @Test + void testValueSizeObject() { + List fieldIds = Arrays.asList(0, 1); + List fieldValues = + Arrays.asList(VariantUtil.encodeInt(42), VariantUtil.encodeBoolean(true)); + byte[] encoded = VariantUtil.encodeObject(fieldIds, fieldValues); + assertThat(VariantUtil.valueSize(encoded, 0)).isEqualTo(encoded.length); + } + + // -------------------------------------------------------------------------------------------- + // Metadata operations + // -------------------------------------------------------------------------------------------- + + @Test + void testEmptyMetadata() { + byte[] metadata = VariantUtil.EMPTY_METADATA; + assertThat(VariantUtil.metadataDictSize(metadata)).isZero(); + } + + @Test + void testMetadataWithFields() { + VariantBuilder builder = new VariantBuilder(); + builder.addFieldName("age"); + builder.addFieldName("name"); + builder.addFieldName("city"); + byte[] metadata = builder.buildMetadata(); + + assertThat(VariantUtil.metadataDictSize(metadata)).isEqualTo(3); + assertThat(VariantUtil.isMetadataSorted(metadata)).isTrue(); + + // Fields should be sorted: age, city, name + assertThat(VariantUtil.metadataFieldName(metadata, 0)).isEqualTo("age"); + assertThat(VariantUtil.metadataFieldName(metadata, 1)).isEqualTo("city"); + assertThat(VariantUtil.metadataFieldName(metadata, 2)).isEqualTo("name"); + } + + @Test + void testFindFieldId() { + VariantBuilder builder = new VariantBuilder(); + builder.addFieldName("age"); + builder.addFieldName("name"); + builder.addFieldName("city"); + byte[] metadata = builder.buildMetadata(); + + assertThat(VariantUtil.findFieldId(metadata, "age")).isEqualTo(0); + assertThat(VariantUtil.findFieldId(metadata, "city")).isEqualTo(1); + assertThat(VariantUtil.findFieldId(metadata, "name")).isEqualTo(2); + assertThat(VariantUtil.findFieldId(metadata, "unknown")).isEqualTo(-1); + } + + // -------------------------------------------------------------------------------------------- + // JSON serialization + // -------------------------------------------------------------------------------------------- + + @Test + void testToJsonNull() { + byte[] value = VariantUtil.encodeNull(); + StringBuilder sb = new StringBuilder(); + VariantUtil.toJson(VariantUtil.EMPTY_METADATA, value, 0, sb); + assertThat(sb.toString()).isEqualTo("null"); + } + + @Test + void testToJsonBoolean() { + StringBuilder sb1 = new StringBuilder(); + VariantUtil.toJson(VariantUtil.EMPTY_METADATA, VariantUtil.encodeBoolean(true), 0, sb1); + assertThat(sb1.toString()).isEqualTo("true"); + + StringBuilder sb2 = new StringBuilder(); + VariantUtil.toJson(VariantUtil.EMPTY_METADATA, VariantUtil.encodeBoolean(false), 0, sb2); + assertThat(sb2.toString()).isEqualTo("false"); + } + + @Test + void testToJsonInt() { + StringBuilder sb = new StringBuilder(); + VariantUtil.toJson(VariantUtil.EMPTY_METADATA, VariantUtil.encodeInt(42), 0, sb); + assertThat(sb.toString()).isEqualTo("42"); + } + + @Test + void testToJsonDouble() { + StringBuilder sb = new StringBuilder(); + VariantUtil.toJson(VariantUtil.EMPTY_METADATA, VariantUtil.encodeDouble(3.14), 0, sb); + assertThat(sb.toString()).isEqualTo("3.14"); + } + + @Test + void testToJsonString() { + byte[] value = VariantUtil.encodeString("hello".getBytes(StandardCharsets.UTF_8)); + StringBuilder sb = new StringBuilder(); + VariantUtil.toJson(VariantUtil.EMPTY_METADATA, value, 0, sb); + assertThat(sb.toString()).isEqualTo("\"hello\""); + } + + @Test + void testToJsonStringWithEscapes() { + byte[] value = + VariantUtil.encodeString( + "line1\nline2\t\"quoted\"".getBytes(StandardCharsets.UTF_8)); + StringBuilder sb = new StringBuilder(); + VariantUtil.toJson(VariantUtil.EMPTY_METADATA, value, 0, sb); + assertThat(sb.toString()).isEqualTo("\"line1\\nline2\\t\\\"quoted\\\"\""); + } + + // -------------------------------------------------------------------------------------------- + // Little-endian read/write + // -------------------------------------------------------------------------------------------- + + @Test + void testReadWriteIntLE() { + byte[] buf = new byte[4]; + VariantUtil.writeIntLE(buf, 0, 0x12345678); + assertThat(VariantUtil.readIntLE(buf, 0)).isEqualTo(0x12345678); + } + + // -------------------------------------------------------------------------------------------- + // Error cases + // -------------------------------------------------------------------------------------------- + + @Test + void testObjectSizeOnNonObject() { + byte[] value = VariantUtil.encodeInt(42); + assertThatThrownBy(() -> VariantUtil.objectSize(value, 0)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Not an object"); + } + + @Test + void testGetBooleanOnNonBoolean() { + byte[] value = VariantUtil.encodeInt(42); + assertThatThrownBy(() -> VariantUtil.getBoolean(value, 0)) + .isInstanceOf(IllegalStateException.class); + } +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22VariantTypeITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22VariantTypeITCase.java new file mode 100644 index 0000000000..ced22bd841 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22VariantTypeITCase.java @@ -0,0 +1,248 @@ +/* + * 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.fluss.flink.sink; + +import org.apache.fluss.server.testutils.FlussClusterExtension; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.fluss.flink.FlinkConnectorOptions.BOOTSTRAP_SERVERS; +import static org.apache.fluss.flink.source.testutils.FlinkRowAssertionsUtils.assertResultsIgnoreOrder; +import static org.apache.fluss.server.testutils.FlussClusterExtension.BUILTIN_DATABASE; + +/** + * Integration tests for Variant type support in Flink 2.2 connector. + * + *

Tests the end-to-end flow of writing and reading Variant data through the Flink connector, + * including type conversion between Flink's BinaryVariant and Fluss's Variant. + */ +public class Flink22VariantTypeITCase extends AbstractTestBase { + + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder().setNumOfTabletServers(3).build(); + + static final String CATALOG_NAME = "testcatalog"; + static final String DEFAULT_DB = "defaultdb"; + + protected StreamExecutionEnvironment env; + protected StreamTableEnvironment tEnv; + protected TableEnvironment tBatchEnv; + + @BeforeEach + void before() { + String bootstrapServers = FLUSS_CLUSTER_EXTENSION.getBootstrapServers(); + + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + + tEnv = StreamTableEnvironment.create(env); + tEnv.executeSql( + String.format( + "create catalog %s with ('type' = 'fluss', '%s' = '%s')", + CATALOG_NAME, BOOTSTRAP_SERVERS.key(), bootstrapServers)); + tEnv.executeSql("use catalog " + CATALOG_NAME); + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + tBatchEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + tBatchEnv.executeSql( + String.format( + "create catalog %s with ('type' = 'fluss', '%s' = '%s')", + CATALOG_NAME, BOOTSTRAP_SERVERS.key(), bootstrapServers)); + tBatchEnv.executeSql("use catalog " + CATALOG_NAME); + tBatchEnv + .getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + tEnv.executeSql("create database " + DEFAULT_DB); + tEnv.useDatabase(DEFAULT_DB); + tBatchEnv.useDatabase(DEFAULT_DB); + } + + @AfterEach + void after() { + tEnv.useDatabase(BUILTIN_DATABASE); + tEnv.executeSql(String.format("drop database %s cascade", DEFAULT_DB)); + } + + @Test + void testVariantInLogTable() throws Exception { + tEnv.executeSql( + "create table variant_log_test (" + + "id int, " + + "data variant" + + ") with ('bucket.num' = '3')"); + + tEnv.executeSql( + "INSERT INTO variant_log_test " + + "SELECT 1, PARSE_JSON('{\"name\": \"Alice\", \"age\": 30}')") + .await(); + tEnv.executeSql("INSERT INTO variant_log_test " + "SELECT 2, PARSE_JSON('[1, 2, 3]')") + .await(); + tEnv.executeSql("INSERT INTO variant_log_test " + "SELECT 3, PARSE_JSON('\"hello\"')") + .await(); + tEnv.executeSql("INSERT INTO variant_log_test " + "SELECT 4, PARSE_JSON('42')").await(); + tEnv.executeSql("INSERT INTO variant_log_test " + "SELECT 5, PARSE_JSON('true')").await(); + tEnv.executeSql("INSERT INTO variant_log_test " + "SELECT 6, PARSE_JSON('null')").await(); + tEnv.executeSql("INSERT INTO variant_log_test " + "SELECT 7, CAST(NULL AS VARIANT)") + .await(); + + CloseableIterator rowIter = + tEnv.executeSql("select * from variant_log_test").collect(); + List expectedRows = + Arrays.asList( + "+I[1, {\"age\":30,\"name\":\"Alice\"}]", + "+I[2, [1,2,3]]", + "+I[3, \"hello\"]", + "+I[4, 42]", + "+I[5, true]", + "+I[6, null]", + "+I[7, null]"); + assertResultsIgnoreOrder(rowIter, expectedRows, true); + } + + @Test + void testVariantInPrimaryKeyTable() throws Exception { + tEnv.executeSql( + "create table variant_pk_test (" + + "id int not null, " + + "data variant, " + + "primary key (id) not enforced" + + ")"); + + // Insert initial data using streaming mode + tEnv.executeSql( + "INSERT INTO variant_pk_test " + + "SELECT 1, PARSE_JSON('{\"key\": \"value1\"}')") + .await(); + tEnv.executeSql( + "INSERT INTO variant_pk_test " + + "SELECT 2, PARSE_JSON('{\"key\": \"value2\"}')") + .await(); + tEnv.executeSql("INSERT INTO variant_pk_test " + "SELECT 3, PARSE_JSON('100')").await(); + + // Verify initial data using streaming mode + CloseableIterator rowIter = tEnv.executeSql("select * from variant_pk_test").collect(); + List expectedRows = + Arrays.asList( + "+I[1, {\"key\":\"value1\"}]", "+I[2, {\"key\":\"value2\"}]", "+I[3, 100]"); + assertResultsIgnoreOrder(rowIter, expectedRows, false); + + // Update data + tEnv.executeSql( + "INSERT INTO variant_pk_test " + + "SELECT 1, PARSE_JSON('{\"key\": \"updated\"}')") + .await(); + tEnv.executeSql("INSERT INTO variant_pk_test " + "SELECT 3, CAST(NULL AS VARIANT)").await(); + + // Verify updated data (changelog semantics: -U for old, +U for new) + expectedRows = + Arrays.asList( + "-U[1, {\"key\":\"value1\"}]", + "+U[1, {\"key\":\"updated\"}]", + "-U[3, 100]", + "+U[3, null]"); + assertResultsIgnoreOrder(rowIter, expectedRows, true); + } + + @Test + void testVariantWithMixedColumns() throws Exception { + tEnv.executeSql( + "create table variant_mixed_test (" + + "id int, " + + "name string, " + + "metadata variant, " + + "tags array" + + ") with ('bucket.num' = '3')"); + + tEnv.executeSql( + "INSERT INTO variant_mixed_test " + + "SELECT 1, 'Alice', PARSE_JSON('{\"role\": \"admin\"}'), " + + "ARRAY['tag1', 'tag2']") + .await(); + tEnv.executeSql( + "INSERT INTO variant_mixed_test " + + "SELECT 2, 'Bob', PARSE_JSON('{\"role\": \"user\", \"level\": 5}'), " + + "ARRAY['tag3']") + .await(); + tEnv.executeSql( + "INSERT INTO variant_mixed_test " + + "SELECT 3, 'Charlie', CAST(NULL AS VARIANT), CAST(NULL AS ARRAY)") + .await(); + + CloseableIterator rowIter = + tEnv.executeSql("select * from variant_mixed_test").collect(); + List expectedRows = + Arrays.asList( + "+I[1, Alice, {\"role\":\"admin\"}, [tag1, tag2]]", + "+I[2, Bob, {\"level\":5,\"role\":\"user\"}, [tag3]]", + "+I[3, Charlie, null, null]"); + assertResultsIgnoreOrder(rowIter, expectedRows, true); + } + + @Test + void testVariantWithNestedJson() throws Exception { + tEnv.executeSql( + "create table variant_nested_test (" + + "id int, " + + "doc variant" + + ") with ('bucket.num' = '3')"); + + tEnv.executeSql( + "INSERT INTO variant_nested_test " + + "SELECT 1, PARSE_JSON('" + + "{\"user\": {\"name\": \"Alice\", \"address\": " + + "{\"city\": \"NYC\", \"zip\": \"10001\"}}, " + + "\"scores\": [95, 87, 92]}" + + "')") + .await(); + tEnv.executeSql( + "INSERT INTO variant_nested_test " + + "SELECT 2, PARSE_JSON('" + + "{\"items\": [{\"id\": 1, \"name\": \"a\"}, " + + "{\"id\": 2, \"name\": \"b\"}]}" + + "')") + .await(); + + CloseableIterator rowIter = + tEnv.executeSql("select * from variant_nested_test").collect(); + List expectedRows = + Arrays.asList( + "+I[1, {\"scores\":[95,87,92],\"user\":{\"address\":" + + "{\"city\":\"NYC\",\"zip\":\"10001\"},\"name\":\"Alice\"}}]", + "+I[2, {\"items\":[{\"id\":1,\"name\":\"a\"}," + + "{\"id\":2,\"name\":\"b\"}]}]"); + assertResultsIgnoreOrder(rowIter, expectedRows, true); + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java index 0caf10d55d..e5ea65fefb 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java @@ -146,7 +146,8 @@ private int[] getNeedProjectFields(Table flussTable, @Nullable int[] projectedFi this.adjustProjectedFields = adjustProjectedFields; return newProjection; } else { - // no projectedFields, use all fields + // no projectedFields, use all user-visible fields + // (getUserRowType() excludes internal shredded columns like $v.x) keyIndexesInRow = pkIndexes; return IntStream.range(0, flussTable.getTableInfo().getRowType().getFieldCount()) .toArray(); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussArray.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussArray.java index 32d17cf844..955c1fc213 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussArray.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussArray.java @@ -25,17 +25,30 @@ import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.variant.Variant; +import org.apache.fluss.utils.MapUtils; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.TimestampData; import java.lang.reflect.Array; +import java.lang.reflect.Method; +import java.util.concurrent.ConcurrentHashMap; import static org.apache.fluss.flink.row.FlinkAsFlussRow.fromFlinkDecimal; /** Wraps a Flink {@link ArrayData} as a Fluss {@link InternalArray}. */ public class FlinkAsFlussArray implements InternalArray { + /** + * Cache of reflected methods keyed by the concrete ArrayData class. The entry is a {@code + * Method[3]} array {@code [getVariantMethod, getMetadataMethod, getValueMethod]}. + * + * @see FlinkAsFlussRow#VARIANT_METHOD_CACHE for why we use direct byte passthrough + */ + private static final ConcurrentHashMap, Method[]> VARIANT_METHOD_CACHE = + MapUtils.newConcurrentHashMap(); + private final ArrayData flinkArray; public FlinkAsFlussArray(ArrayData flinkArray) { @@ -176,6 +189,41 @@ public InternalRow getRow(int pos, int numFields) { return new FlinkAsFlussRow(flinkArray.getRow(pos, numFields)); } + @Override + public Variant getVariant(int pos) { + try { + Method[] methods = + VARIANT_METHOD_CACHE.computeIfAbsent( + flinkArray.getClass(), + clazz -> { + try { + Method getVariantMethod = + clazz.getMethod("getVariant", int.class); + Class variantClass = + Class.forName( + "org.apache.flink.types.variant.BinaryVariant"); + Method getMetadataMethod = + variantClass.getMethod("getMetadata"); + Method getValueMethod = variantClass.getMethod("getValue"); + return new Method[] { + getVariantMethod, getMetadataMethod, getValueMethod + }; + } catch (Exception e) { + throw new UnsupportedOperationException( + "Variant type requires Flink 2.1 or later.", e); + } + }); + Object flinkVariant = methods[0].invoke(flinkArray, pos); + byte[] metadata = (byte[]) methods[1].invoke(flinkVariant); + byte[] value = (byte[]) methods[2].invoke(flinkVariant); + return new Variant(metadata, value); + } catch (UnsupportedOperationException e) { + throw e; + } catch (Exception e) { + throw new UnsupportedOperationException("Variant type requires Flink 2.1 or later.", e); + } + } + @SuppressWarnings("unchecked") public T[] toObjectArray(DataType elementType) { Class elementClass = (Class) InternalRow.getDataClass(elementType); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java index eba5abe9ba..87006d9f65 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java @@ -24,14 +24,32 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.variant.Variant; +import org.apache.fluss.utils.MapUtils; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.TimestampData; +import java.lang.reflect.Method; +import java.util.concurrent.ConcurrentHashMap; + /** Wraps a Flink {@link RowData} as a Fluss {@link InternalRow}. */ public class FlinkAsFlussRow implements InternalRow { + /** + * Cache of reflected methods keyed by the concrete class. The entry is a {@code Method[3]} + * array {@code [getVariantMethod, getMetadataMethod, getValueMethod]}. Using a per-class cache + * avoids repeated {@link Class#getMethod} calls on the hot read path. + * + *

We directly pass the raw byte arrays from Flink's {@code BinaryVariant} (metadata and + * value) into Fluss's {@code Variant}, similar to Paimon's {@code FlinkRowWrapper}. Fluss's + * {@code VariantUtil} supports Parquet Variant spec variable-width encoding, so the raw bytes + * are compatible. + */ + private static final ConcurrentHashMap, Method[]> VARIANT_METHOD_CACHE = + MapUtils.newConcurrentHashMap(); + private RowData flinkRow; public FlinkAsFlussRow() {} @@ -153,4 +171,43 @@ public InternalMap getMap(int pos) { public InternalRow getRow(int pos, int numFields) { return new FlinkAsFlussRow(flinkRow.getRow(pos, numFields)); } + + @Override + public Variant getVariant(int pos) { + try { + // Resolve and cache reflected methods per RowData concrete class to avoid + // per-call Method lookup overhead on the hot read path. + Method[] methods = + VARIANT_METHOD_CACHE.computeIfAbsent( + flinkRow.getClass(), + clazz -> { + try { + Method getVariantMethod = + clazz.getMethod("getVariant", int.class); + Class variantClass = + Class.forName( + "org.apache.flink.types.variant.BinaryVariant"); + Method getMetadataMethod = + variantClass.getMethod("getMetadata"); + Method getValueMethod = variantClass.getMethod("getValue"); + return new Method[] { + getVariantMethod, getMetadataMethod, getValueMethod + }; + } catch (Exception e) { + throw new UnsupportedOperationException( + "Variant type requires Flink 2.1 or later.", e); + } + }); + Object flinkVariant = methods[0].invoke(flinkRow, pos); + // Directly pass raw byte arrays from Flink's BinaryVariant to Fluss's Variant. + // VariantUtil supports variable-width Parquet Variant encoding. + byte[] metadata = (byte[]) methods[1].invoke(flinkVariant); + byte[] value = (byte[]) methods[2].invoke(flinkVariant); + return new Variant(metadata, value); + } catch (UnsupportedOperationException e) { + throw e; + } catch (Exception e) { + throw new UnsupportedOperationException("Variant type requires Flink 2.1 or later.", e); + } + } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlussSinkBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlussSinkBuilder.java index c8808f12f8..1b88ccc6dc 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlussSinkBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlussSinkBuilder.java @@ -204,6 +204,7 @@ public FlussSink build() { int numBucket = tableInfo.getNumBuckets(); List bucketKeys = tableInfo.getBucketKeys(); List partitionKeys = tableInfo.getPartitionKeys(); + // Use user-visible row type (excludes internal shredded columns like $v.x) RowType tableRowType = toFlinkRowType(tableInfo.getRowType()); DataLakeFormat lakeFormat = tableInfo.getTableConfig().getDataLakeFormat().orElse(null); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java index 87efc01cd2..c49235349c 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java @@ -265,6 +265,7 @@ public FlussSource build() { } if (this.projectedFieldNames != null && this.projectedFieldNames.length > 0) { + // Use user-visible row type (excludes internal shredded columns like $v.x) RowType rowType = tableInfo.getRowType(); List allFieldNames = rowType.getFieldNames(); @@ -300,6 +301,7 @@ public FlussSource build() { boolean isPartitioned = !tableInfo.getPartitionKeys().isEmpty(); boolean hasPrimaryKey = !tableInfo.getPrimaryKeys().isEmpty(); + // Use user-visible row type (excludes internal shredded columns like $v.x) RowType sourceOutputType = projectedFields != null ? tableInfo.getRowType().project(projectedFields) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java index 90f2b6e9a6..6c0e25b57e 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java @@ -136,6 +136,7 @@ public FlinkSourceSplitReader( this.projectedFields = projectedFields; this.flinkSourceReaderMetrics = flinkSourceReaderMetrics; + // Validate against user-visible schema (excludes internal shredded columns like $v.x) sanityCheck(table.getTableInfo().getRowType(), projectedFields); this.logScanner = table.newScan() diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverter.java index 2b471f1136..343fcaad1d 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverter.java @@ -30,6 +30,7 @@ import org.apache.fluss.row.encode.RowEncoder; import org.apache.fluss.row.indexed.IndexedRow; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.variant.Variant; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; @@ -41,6 +42,7 @@ import org.apache.flink.table.types.logical.RowType; import java.io.Serializable; +import java.lang.reflect.Method; /** * A converter to convert Flink's {@link RowData} to Fluss's {@link InternalRow}. @@ -187,6 +189,33 @@ private FlussSerializationConverter createInternalConverter(LogicalType flinkDat case ROW: return flinkField -> new FlinkAsFlussRow((RowData) flinkField); default: + // Check for Variant type by name since it's only available in Flink 2.1+ + if ("VARIANT".equals(flinkDataType.getTypeRoot().name())) { + // Resolve BinaryVariant accessor methods once at converter construction time. + // The concrete BinaryVariant class is the same for all rows, so we look it up + // from the declared return type of the known class. + final Method resolvedMetadataMethod; + final Method resolvedValueMethod; + try { + Class binaryVariantClass = + Class.forName("org.apache.flink.types.variant.BinaryVariant"); + resolvedMetadataMethod = binaryVariantClass.getMethod("getMetadata"); + resolvedValueMethod = binaryVariantClass.getMethod("getValue"); + } catch (Exception e) { + throw new UnsupportedOperationException( + "Variant type requires Flink 2.1 or later.", e); + } + return flinkField -> { + try { + byte[] metadata = (byte[]) resolvedMetadataMethod.invoke(flinkField); + byte[] value = (byte[]) resolvedValueMethod.invoke(flinkField); + return new Variant(metadata, value); + } catch (Exception e) { + throw new RuntimeException( + "Failed to read Flink BinaryVariant fields.", e); + } + }; + } throw new UnsupportedOperationException( "Fluss Unsupported data type: " + flinkDataType); } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkTypeToFlussType.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkTypeToFlussType.java index 8fd3af27a6..3698a67d9f 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkTypeToFlussType.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkTypeToFlussType.java @@ -21,6 +21,7 @@ import org.apache.fluss.types.DataField; import org.apache.fluss.types.DataType; import org.apache.fluss.types.StringType; +import org.apache.fluss.types.VariantType; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; @@ -170,6 +171,11 @@ public DataType visit(RowType rowType) { @Override protected DataType defaultMethod(LogicalType logicalType) { + // Check for Variant type by name since it's only available in Flink 2.1+ + // and not present in the LogicalTypeRoot enum of older Flink versions + if ("VARIANT".equals(logicalType.getTypeRoot().name())) { + return new VariantType(logicalType.isNullable()); + } throw new UnsupportedOperationException("Unsupported data type: " + logicalType); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverter.java index 7661691596..945e877afd 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverter.java @@ -29,6 +29,7 @@ import org.apache.fluss.types.DataType; import org.apache.fluss.types.MapType; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.Variant; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; @@ -40,6 +41,7 @@ import org.apache.flink.types.RowKind; import java.io.Serializable; +import java.lang.reflect.Constructor; import java.util.HashMap; import java.util.Map; @@ -210,6 +212,31 @@ static FlussDeserializationConverter createInternalConverter(DataType flussDataT } return flinkRowData; }; + case VARIANT: + // NOTE: FlussVariant (shredded fast-path) is disabled because Flink 2.2's + // VariantSerializer.toBinaryVariant() requires instanceof BinaryVariant and + // BinaryVariant is final — custom Variant implementations are rejected during + // serialisation / copy. Always produce BinaryVariant for now. + final Constructor binaryVariantConstructor; + try { + Class binaryVariantClass = + Class.forName("org.apache.flink.types.variant.BinaryVariant"); + binaryVariantConstructor = + binaryVariantClass.getConstructor(byte[].class, byte[].class); + } catch (Exception e) { + throw new UnsupportedOperationException( + "Variant type requires Flink 2.1 or later.", e); + } + return (flussField) -> { + Variant variant = (Variant) flussField; + try { + return binaryVariantConstructor.newInstance( + variant.value(), variant.metadata()); + } catch (Exception e) { + throw new RuntimeException( + "Failed to create Flink BinaryVariant instance.", e); + } + }; default: throw new UnsupportedOperationException("Unsupported data type: " + flussDataType); } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussTypeToFlinkType.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussTypeToFlinkType.java index a3b527966d..ce0f1fd837 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussTypeToFlinkType.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussTypeToFlinkType.java @@ -38,9 +38,13 @@ import org.apache.fluss.types.TimeType; import org.apache.fluss.types.TimestampType; import org.apache.fluss.types.TinyIntType; +import org.apache.fluss.types.VariantType; import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.AtomicDataType; import org.apache.flink.table.types.DataType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; @@ -48,6 +52,8 @@ /** Convert Fluss's {@link org.apache.fluss.types.DataType} to Flink's {@link DataType}. */ class FlussTypeToFlinkType implements DataTypeVisitor { + private static final Logger LOG = LoggerFactory.getLogger(FlussTypeToFlinkType.class); + static final FlussTypeToFlinkType INSTANCE = new FlussTypeToFlinkType(); @Override @@ -168,6 +174,26 @@ public DataType visit(RowType rowType) { return withNullability(DataTypes.ROW(dataFields), rowType.isNullable()); } + @Override + public DataType visit(VariantType variantType) { + try { + // Use reflection to create Flink's VariantType which is only available in Flink 2.1+ + Class flinkVariantTypeClass = + Class.forName("org.apache.flink.table.types.logical.VariantType"); + Object flinkVariantLogicalType = + flinkVariantTypeClass + .getConstructor(boolean.class) + .newInstance(variantType.isNullable()); + return new AtomicDataType( + (org.apache.flink.table.types.logical.LogicalType) flinkVariantLogicalType); + } catch (Exception e) { + throw new UnsupportedOperationException( + "Variant type requires Flink 2.1 or later. " + + "Please upgrade your Flink version to support Variant type.", + e); + } + } + private DataType withNullability(DataType flinkType, boolean nullable) { if (flinkType.getLogicalType().isNullable() != nullable) { return nullable ? flinkType.nullable() : flinkType.notNull(); diff --git a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/FlussDataTypeToIcebergDataType.java b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/FlussDataTypeToIcebergDataType.java index fb8f819574..345693bf6a 100644 --- a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/FlussDataTypeToIcebergDataType.java +++ b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/FlussDataTypeToIcebergDataType.java @@ -38,6 +38,7 @@ import org.apache.fluss.types.TimeType; import org.apache.fluss.types.TimestampType; import org.apache.fluss.types.TinyIntType; +import org.apache.fluss.types.VariantType; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -208,4 +209,9 @@ public Type visit(RowType rowType) { return Types.StructType.of(fields); } + + @Override + public Type visit(VariantType variantType) { + throw new UnsupportedOperationException("Variant type is not supported in Iceberg."); + } } diff --git a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergArrayAsFlussArray.java b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergArrayAsFlussArray.java index 1b83e8f2b4..dfbf6b5c6b 100644 --- a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergArrayAsFlussArray.java +++ b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergArrayAsFlussArray.java @@ -25,6 +25,7 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.BytesUtils; import java.math.BigDecimal; @@ -158,6 +159,11 @@ public InternalRow getRow(int pos, int numFields) { throw new UnsupportedOperationException(); } + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("Variant type is not supported in Iceberg."); + } + @Override public boolean[] toBooleanArray() { boolean[] result = new boolean[icebergList.size()]; diff --git a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordAsFlussRow.java b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordAsFlussRow.java index be990f7c64..33bbf2a307 100644 --- a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordAsFlussRow.java +++ b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordAsFlussRow.java @@ -25,6 +25,7 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.BytesUtils; import org.apache.iceberg.data.Record; @@ -202,4 +203,9 @@ public InternalRow getRow(int pos, int numFields) { + value.getClass().getName()); } } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("Variant type is not supported in Iceberg."); + } } diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/FlussDataTypeToPaimonDataType.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/FlussDataTypeToPaimonDataType.java index e97b4f67a6..de6af4f98e 100644 --- a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/FlussDataTypeToPaimonDataType.java +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/FlussDataTypeToPaimonDataType.java @@ -38,6 +38,7 @@ import org.apache.fluss.types.TimeType; import org.apache.fluss.types.TimestampType; import org.apache.fluss.types.TinyIntType; +import org.apache.fluss.types.VariantType; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -160,6 +161,11 @@ public DataType visit(RowType rowType) { return withNullability(rowTypeBuilder.build(), rowType.isNullable()); } + @Override + public DataType visit(VariantType variantType) { + throw new UnsupportedOperationException("Variant type is not supported in Paimon."); + } + private DataType withNullability(DataType paimon, boolean nullable) { if (paimon.isNullable() != nullable) { return nullable ? paimon.nullable() : paimon.notNull(); diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArray.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArray.java index b5df153d74..171b016ee5 100644 --- a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArray.java +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArray.java @@ -24,6 +24,7 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.variant.Variant; import org.apache.paimon.data.Timestamp; @@ -149,6 +150,11 @@ public InternalRow getRow(int pos, int numFields) { return new PaimonRowAsFlussRow(paimonArray.getRow(pos, numFields)); } + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("Variant type is not supported in Paimon."); + } + @Override public boolean[] toBooleanArray() { return paimonArray.toBooleanArray(); diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonRowAsFlussRow.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonRowAsFlussRow.java index fe956561a6..f9fd2d505c 100644 --- a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonRowAsFlussRow.java +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonRowAsFlussRow.java @@ -24,6 +24,7 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.variant.Variant; import org.apache.paimon.data.Timestamp; @@ -156,4 +157,9 @@ public InternalMap getMap(int pos) { public InternalRow getRow(int pos, int numFields) { return new PaimonRowAsFlussRow(paimonRow.getRow(pos, numFields)); } + + @Override + public Variant getVariant(int pos) { + throw new UnsupportedOperationException("Variant type is not supported in Paimon."); + } } diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index 9ed8e312d5..c649b6bbf3 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -821,6 +821,15 @@ message PbFetchLogReqForTable { // the schema against which the predicate was compiled. optional PbPredicate filter_predicate = 5; optional int32 filter_schema_id = 6; + // Variant sub-field projection hints. Each entry specifies the sub-fields + // to project within a Variant column. Empty means no sub-field projection. + repeated PbVariantFieldProjection variant_field_projections = 7; +} + +// Specifies which sub-fields within a Variant column should be projected. +message PbVariantFieldProjection { + required int32 column_index = 1; + repeated string field_names = 2; } message PbFetchLogReqForBucket { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/FetchReqInfo.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/FetchReqInfo.java index 0f8df68002..dd98274d2a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/entity/FetchReqInfo.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/entity/FetchReqInfo.java @@ -22,6 +22,8 @@ import javax.annotation.Nullable; import java.util.Arrays; +import java.util.List; +import java.util.Map; import java.util.Objects; /** The structure of fetch data. */ @@ -30,19 +32,34 @@ public final class FetchReqInfo { private final long tableId; private final long fetchOffset; @Nullable private final int[] projectFields; + /** + * Variant sub-field projection hints. Maps Variant column index to the list of sub-field names + * to project. Null means no sub-field projection. + */ + @Nullable private final Map> variantFieldProjection; private int maxBytes; public FetchReqInfo(long tableId, long fetchOffset, int maxBytes) { - this(tableId, fetchOffset, maxBytes, null); + this(tableId, fetchOffset, maxBytes, null, null); } public FetchReqInfo( long tableId, long fetchOffset, int maxBytes, @Nullable int[] projectFields) { + this(tableId, fetchOffset, maxBytes, projectFields, null); + } + + public FetchReqInfo( + long tableId, + long fetchOffset, + int maxBytes, + @Nullable int[] projectFields, + @Nullable Map> variantFieldProjection) { this.tableId = tableId; this.fetchOffset = fetchOffset; this.maxBytes = maxBytes; this.projectFields = projectFields; + this.variantFieldProjection = variantFieldProjection; } public long getTableId() { @@ -66,6 +83,11 @@ public int[] getProjectFields() { return projectFields; } + @Nullable + public Map> getVariantFieldProjection() { + return variantFieldProjection; + } + @Override public String toString() { return "FetchData{" diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java index 084529de2a..b3fa8688bb 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java @@ -26,6 +26,7 @@ import javax.annotation.Nullable; +import java.util.List; import java.util.Map; import java.util.Objects; @@ -106,6 +107,26 @@ public void setCurrentFetch( ArrowCompressionInfo compressionInfo, @Nullable int[] projectedFields, ProjectionPushdownCache projectionCache) { + setCurrentFetch( + tableId, + fetchOffset, + maxFetchBytes, + schemaGetter, + compressionInfo, + projectedFields, + null, + projectionCache); + } + + public void setCurrentFetch( + long tableId, + long fetchOffset, + int maxFetchBytes, + SchemaGetter schemaGetter, + ArrowCompressionInfo compressionInfo, + @Nullable int[] projectedFields, + @Nullable Map> variantFieldProjection, + ProjectionPushdownCache projectionCache) { this.fetchOffset = fetchOffset; this.maxFetchBytes = maxFetchBytes; if (projectedFields != null) { @@ -115,7 +136,11 @@ public void setCurrentFetch( } fileLogProjection.setCurrentProjection( - tableId, schemaGetter, compressionInfo, projectedFields); + tableId, + schemaGetter, + compressionInfo, + projectedFields, + variantFieldProjection); } else { projectionEnabled = false; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index 19ab3f936a..47ded154c7 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -1406,6 +1406,7 @@ public Map readFromLog( replica.getSchemaGetter(), replica.getArrowCompressionInfo(), fetchReqInfo.getProjectFields(), + fetchReqInfo.getVariantFieldProjection(), projectionsCache); LogReadInfo readInfo = replica.fetchRecords(fetchParams); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index d37c36148e..bcd95925a4 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -155,6 +155,7 @@ import org.apache.fluss.rpc.messages.PbTableStatsRespForBucket; import org.apache.fluss.rpc.messages.PbValue; import org.apache.fluss.rpc.messages.PbValueList; +import org.apache.fluss.rpc.messages.PbVariantFieldProjection; import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.PrefixLookupResponse; import org.apache.fluss.rpc.messages.ProduceLogRequest; @@ -936,6 +937,19 @@ public static Map getFetchLogData(FetchLogRequest req projectionFields = null; } + // Extract variant sub-field projection hints + final Map> variantFieldProjection; + if (fetchLogReqForTable.getVariantFieldProjectionsCount() > 0) { + variantFieldProjection = new HashMap<>(); + for (int vi = 0; vi < fetchLogReqForTable.getVariantFieldProjectionsCount(); vi++) { + PbVariantFieldProjection vfp = + fetchLogReqForTable.getVariantFieldProjectionAt(vi); + variantFieldProjection.put(vfp.getColumnIndex(), vfp.getFieldNamesList()); + } + } else { + variantFieldProjection = null; + } + List bucketsReqsList = fetchLogReqForTable.getBucketsReqsList(); for (PbFetchLogReqForBucket fetchLogReqForBucket : bucketsReqsList) { int bucketId = fetchLogReqForBucket.getBucketId(); @@ -950,7 +964,8 @@ public static Map getFetchLogData(FetchLogRequest req tableId, fetchLogReqForBucket.getFetchOffset(), fetchLogReqForBucket.getMaxFetchBytes(), - projectionFields)); + projectionFields, + variantFieldProjection)); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java index 0d9f5bc565..8827cc0761 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -20,11 +20,14 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.ServerNode; import org.apache.fluss.cluster.ServerType; +import org.apache.fluss.compression.ArrowCompressionInfo; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidRequiredAcksException; import org.apache.fluss.exception.NotLeaderOrFollowerException; import org.apache.fluss.exception.UnknownTableOrBucketException; +import org.apache.fluss.memory.ManagedPagedOutputView; +import org.apache.fluss.memory.TestingMemorySegmentPool; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.PhysicalTablePath; @@ -34,6 +37,7 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.DefaultLogRecordBatch; import org.apache.fluss.record.DefaultValueRecordBatch; import org.apache.fluss.record.KvRecord; import org.apache.fluss.record.KvRecordBatch; @@ -42,8 +46,11 @@ import org.apache.fluss.record.LogRecordReadContext; import org.apache.fluss.record.LogRecords; import org.apache.fluss.record.MemoryLogRecords; +import org.apache.fluss.record.MemoryLogRecordsArrowBuilder; import org.apache.fluss.record.TestingSchemaGetter; import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.arrow.ArrowWriter; +import org.apache.fluss.row.arrow.ArrowWriterPool; import org.apache.fluss.row.encode.CompactedKeyEncoder; import org.apache.fluss.row.encode.ValueDecoder; import org.apache.fluss.row.encode.ValueEncoder; @@ -74,10 +81,15 @@ import org.apache.fluss.server.testutils.KvTestUtils; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.RootAllocator; import org.apache.fluss.testutils.DataTestUtils; import org.apache.fluss.types.DataField; import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; +import org.apache.fluss.types.variant.ShreddedField; +import org.apache.fluss.types.variant.ShreddingSchema; +import org.apache.fluss.types.variant.Variant; import org.apache.fluss.utils.CloseableIterator; import org.apache.fluss.utils.types.Tuple2; @@ -126,6 +138,12 @@ import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static org.apache.fluss.record.TestData.EXPECTED_LOG_RESULTS_FOR_DATA_1_WITH_PK; +import static org.apache.fluss.record.TestData.VARIANT_DATA; +import static org.apache.fluss.record.TestData.VARIANT_PHYSICAL_TABLE_PATH; +import static org.apache.fluss.record.TestData.VARIANT_ROW_TYPE; +import static org.apache.fluss.record.TestData.VARIANT_SCHEMA; +import static org.apache.fluss.record.TestData.VARIANT_TABLE_ID; +import static org.apache.fluss.record.TestData.VARIANT_TABLE_PATH; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.server.metadata.PartitionMetadata.DELETED_PARTITION_ID; import static org.apache.fluss.server.metadata.TableMetadata.DELETED_TABLE_ID; @@ -2213,6 +2231,206 @@ private void verifyPrefixLookup( } } + @Test + void testProduceAndFetchVariantLog() throws Exception { + SchemaGetter schemaGetter = + serverMetadataCache.subscribeWithInitialSchema( + VARIANT_TABLE_PATH, VARIANT_TABLE_ID, DEFAULT_SCHEMA_ID, VARIANT_SCHEMA); + TableBucket tb = new TableBucket(VARIANT_TABLE_ID, 0); + makeLogTableAsLeader(VARIANT_PHYSICAL_TABLE_PATH, tb); + + // Produce variant data. + CompletableFuture> future = new CompletableFuture<>(); + replicaManager.appendRecordsToLog( + 20000, + 1, + Collections.singletonMap( + tb, + genMemoryLogRecordsByObject( + VARIANT_ROW_TYPE, DEFAULT_SCHEMA_ID, (byte) 2, VARIANT_DATA)), + null, + future::complete); + assertThat(future.get()) + .containsOnly(new ProduceLogResultForBucket(tb, 0, (long) VARIANT_DATA.size())); + + // Fetch and verify variant data. + CompletableFuture> fetchFuture = + new CompletableFuture<>(); + replicaManager.fetchLogRecords( + buildFetchParams(-1), + Collections.singletonMap(tb, new FetchReqInfo(tb.getTableId(), 0L, 1024 * 1024)), + null, + fetchFuture::complete); + Map result = fetchFuture.get(); + assertThat(result).hasSize(1); + FetchLogResultForBucket resultForBucket = result.get(tb); + assertThat(resultForBucket.getHighWatermark()).isEqualTo((long) VARIANT_DATA.size()); + LogRecords records = resultForBucket.records(); + assertThat(records).isNotNull(); + assertLogRecordsEquals(VARIANT_ROW_TYPE, records, VARIANT_DATA, schemaGetter); + } + + @Test + void testFetchVariantLogWithColumnProjection() throws Exception { + SchemaGetter schemaGetter = + serverMetadataCache.subscribeWithInitialSchema( + VARIANT_TABLE_PATH, VARIANT_TABLE_ID, DEFAULT_SCHEMA_ID, VARIANT_SCHEMA); + TableBucket tb = new TableBucket(VARIANT_TABLE_ID, 1); + makeLogTableAsLeader(VARIANT_PHYSICAL_TABLE_PATH, tb); + + // Produce variant data. + CompletableFuture> future = new CompletableFuture<>(); + replicaManager.appendRecordsToLog( + 20000, + 1, + Collections.singletonMap( + tb, + genMemoryLogRecordsByObject( + VARIANT_ROW_TYPE, DEFAULT_SCHEMA_ID, (byte) 2, VARIANT_DATA)), + null, + future::complete); + assertThat(future.get()) + .containsOnly(new ProduceLogResultForBucket(tb, 0, (long) VARIANT_DATA.size())); + + // Fetch with column projection [1] (only the variant column "data"). + CompletableFuture> fetchFuture = + new CompletableFuture<>(); + replicaManager.fetchLogRecords( + buildFetchParams(-1), + Collections.singletonMap( + tb, new FetchReqInfo(tb.getTableId(), 0L, 1024 * 1024, new int[] {1})), + null, + fetchFuture::complete); + Map result = fetchFuture.get(); + assertThat(result).hasSize(1); + FetchLogResultForBucket resultForBucket = result.get(tb); + assertThat(resultForBucket.getHighWatermark()).isEqualTo((long) VARIANT_DATA.size()); + + // Verify projected records only contain the variant column. + RowType projectedType = DataTypes.ROW(new DataField("data", DataTypes.VARIANT())); + LogRecords records = resultForBucket.records(); + assertThat(records).isNotNull(); + LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + projectedType, DEFAULT_SCHEMA_ID, schemaGetter, true); + int i = 0; + for (LogRecordBatch batch : records.batches()) { + try (CloseableIterator iter = batch.records(readContext)) { + while (iter.hasNext()) { + LogRecord record = iter.next(); + Variant expectedVariant = (Variant) VARIANT_DATA.get(i)[1]; + Variant actualVariant = record.getRow().getVariant(0); + assertThat(actualVariant).isEqualTo(expectedVariant); + i++; + } + } + } + readContext.close(); + assertThat(i).isEqualTo(VARIANT_DATA.size()); + } + + @Test + void testFetchVariantLogWithShreddingProjection() throws Exception { + SchemaGetter schemaGetter = + serverMetadataCache.subscribeWithInitialSchema( + VARIANT_TABLE_PATH, VARIANT_TABLE_ID, DEFAULT_SCHEMA_ID, VARIANT_SCHEMA); + TableBucket tb = new TableBucket(VARIANT_TABLE_ID, 2); + makeLogTableAsLeader(VARIANT_PHYSICAL_TABLE_PATH, tb); + + // Create shredded MemoryLogRecords with shredding for "name"(STRING) and "age"(INT). + ShreddingSchema shreddingSchema = + new ShreddingSchema( + "data", + Arrays.asList( + new ShreddedField("name", DataTypes.STRING()), + new ShreddedField("age", DataTypes.INT()))); + Map shreddingSchemas = + Collections.singletonMap("data", shreddingSchema); + + MemoryLogRecords shreddedRecords; + try (BufferAllocator allocator = new RootAllocator(Integer.MAX_VALUE); + ArrowWriterPool pool = new ArrowWriterPool(allocator)) { + ArrowWriter writer = + pool.getOrCreateWriter( + VARIANT_TABLE_ID, + DEFAULT_SCHEMA_ID, + Integer.MAX_VALUE, + VARIANT_ROW_TYPE, + ArrowCompressionInfo.DEFAULT_COMPRESSION, + shreddingSchemas); + MemoryLogRecordsArrowBuilder builder = + MemoryLogRecordsArrowBuilder.builder( + 0L, + (byte) 2, + DEFAULT_SCHEMA_ID, + writer, + new ManagedPagedOutputView(new TestingMemorySegmentPool(10 * 1024))); + for (Object[] objs : VARIANT_DATA) { + builder.append(ChangeType.APPEND_ONLY, DataTestUtils.row(VARIANT_ROW_TYPE, objs)); + } + builder.close(); + shreddedRecords = MemoryLogRecords.pointToBytesView(builder.build()); + ((DefaultLogRecordBatch) shreddedRecords.batches().iterator().next()) + .setCommitTimestamp(System.currentTimeMillis()); + shreddedRecords.ensureValid((byte) 2); + } + + // Produce shredded variant data. + CompletableFuture> future = new CompletableFuture<>(); + replicaManager.appendRecordsToLog( + 20000, 1, Collections.singletonMap(tb, shreddedRecords), null, future::complete); + assertThat(future.get()) + .containsOnly(new ProduceLogResultForBucket(tb, 0, (long) VARIANT_DATA.size())); + + // Fetch with column projection [1] + variant sub-field projection {1 -> ["name"]}. + Map> variantFieldProjection = new HashMap<>(); + variantFieldProjection.put(1, Collections.singletonList("name")); + + CompletableFuture> fetchFuture = + new CompletableFuture<>(); + replicaManager.fetchLogRecords( + buildFetchParams(-1), + Collections.singletonMap( + tb, + new FetchReqInfo( + tb.getTableId(), + 0L, + 1024 * 1024, + new int[] {1}, + variantFieldProjection)), + null, + fetchFuture::complete); + Map result = fetchFuture.get(); + assertThat(result).hasSize(1); + FetchLogResultForBucket resultForBucket = result.get(tb); + assertThat(resultForBucket.getHighWatermark()).isEqualTo((long) VARIANT_DATA.size()); + + // Verify projected records contain the variant column with full value. + RowType projectedType = DataTypes.ROW(new DataField("data", DataTypes.VARIANT())); + LogRecords records = resultForBucket.records(); + assertThat(records).isNotNull(); + LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + projectedType, DEFAULT_SCHEMA_ID, schemaGetter, true); + int i = 0; + for (LogRecordBatch batch : records.batches()) { + try (CloseableIterator iter = batch.records(readContext)) { + while (iter.hasNext()) { + LogRecord record = iter.next(); + Variant actualVariant = record.getRow().getVariant(0); + // The full variant value should be preserved. + Variant expectedVariant = (Variant) VARIANT_DATA.get(i)[1]; + // Compare semantically since ShreddedVariant merge may produce + // a different binary encoding order than the original Variant. + assertThat(actualVariant.toJson()).isEqualTo(expectedVariant.toJson()); + i++; + } + } + } + readContext.close(); + assertThat(i).isEqualTo(VARIANT_DATA.size()); + } + @Test void testKvFormatV2TableRejectV0Client() throws Exception { // Test that old client version (version 0) is rejected for new format tables diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java index f829ca8924..dffac797ff 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java @@ -111,6 +111,10 @@ import static org.apache.fluss.record.TestData.DATA3_TABLE_ID_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA3_TABLE_PATH_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; +import static org.apache.fluss.record.TestData.VARIANT_SCHEMA; +import static org.apache.fluss.record.TestData.VARIANT_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.VARIANT_TABLE_ID; +import static org.apache.fluss.record.TestData.VARIANT_TABLE_PATH; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.server.replica.ReplicaManager.HIGH_WATERMARK_CHECKPOINT_FILE_NAME; import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_BUCKET_EPOCH; @@ -282,6 +286,12 @@ private void registerTableInZkClient() throws Exception { DEFAULT_REMOTE_DATA_DIR, DATA3_TABLE_DESCRIPTOR_PK_AUTO_INC)); zkClient.registerFirstSchema(DATA3_TABLE_PATH_PK_AUTO_INC, DATA3_SCHEMA_PK_AUTO_INC); + + zkClient.registerTable( + VARIANT_TABLE_PATH, + TableRegistration.newTable( + VARIANT_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, VARIANT_TABLE_DESCRIPTOR)); + zkClient.registerFirstSchema(VARIANT_TABLE_PATH, VARIANT_SCHEMA); } protected long registerTableInZkClient( @@ -382,6 +392,22 @@ protected void makeLogTableAsLeader(TableBucket tb, boolean partitionTable) { partitionTable); } + /** Makes a log table as leader for any table path (not limited to DATA1). */ + protected void makeLogTableAsLeader(PhysicalTablePath physicalTablePath, TableBucket tb) { + makeLeaderAndFollower( + Collections.singletonList( + new NotifyLeaderAndIsrData( + physicalTablePath, + tb, + Collections.singletonList(TABLET_SERVER_ID), + new LeaderAndIsr( + TABLET_SERVER_ID, + INITIAL_LEADER_EPOCH, + Collections.singletonList(TABLET_SERVER_ID), + INITIAL_COORDINATOR_EPOCH, + INITIAL_BUCKET_EPOCH)))); + } + protected void makeLogTableAsLeader( TableBucket tb, List replicas, List isr, boolean partitionTable) { makeLeaderAndFollower( diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/row/SparkAsFlussArray.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/row/SparkAsFlussArray.scala index 17d45ec028..2c6237a4b8 100644 --- a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/row/SparkAsFlussArray.scala +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/row/SparkAsFlussArray.scala @@ -18,6 +18,7 @@ package org.apache.fluss.spark.row import org.apache.fluss.row.{BinaryString, Decimal, InternalArray => FlussInternalArray, InternalMap, InternalRow => FlussInternalRow, TimestampLtz, TimestampNtz} +import org.apache.fluss.types.variant.Variant import org.apache.spark.sql.catalyst.util.{ArrayData => SparkArrayData} import org.apache.spark.sql.types.{ArrayType => SparkArrayType, DataType => SparkDataType, StructType} @@ -134,4 +135,9 @@ class SparkAsFlussArray(arrayData: SparkArrayData, elementType: SparkDataType) override def getMap(pos: Int): InternalMap = { throw new UnsupportedOperationException() } + + /** Returns the variant value at the given position. */ + override def getVariant(pos: Int): Variant = { + throw new UnsupportedOperationException("Variant type is not supported in Spark 3.x.") + } } diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/row/SparkAsFlussRow.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/row/SparkAsFlussRow.scala index 3a5c9613c2..0dc45438b4 100644 --- a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/row/SparkAsFlussRow.scala +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/row/SparkAsFlussRow.scala @@ -18,6 +18,7 @@ package org.apache.fluss.spark.row import org.apache.fluss.row.{BinaryString, Decimal, InternalMap, InternalRow => FlussInternalRow, TimestampLtz, TimestampNtz} +import org.apache.fluss.types.variant.Variant import org.apache.spark.sql.catalyst.{InternalRow => SparkInternalRow} import org.apache.spark.sql.types.StructType @@ -129,4 +130,9 @@ class SparkAsFlussRow(schema: StructType) extends FlussInternalRow with Serializ override def getMap(pos: Int): InternalMap = { throw new UnsupportedOperationException() } + + /** Returns the variant value at the given position. */ + override def getVariant(pos: Int): Variant = { + throw new UnsupportedOperationException("Variant type is not supported in Spark 3.x.") + } } diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/types/FlussToSparkTypeVisitor.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/types/FlussToSparkTypeVisitor.scala index f9df1e9c06..83e62be486 100644 --- a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/types/FlussToSparkTypeVisitor.scala +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/types/FlussToSparkTypeVisitor.scala @@ -113,4 +113,8 @@ object FlussToSparkTypeVisitor extends DataTypeVisitor[SparkDataType] { } SparkDataTypes.createStructType(sparkFields.toArray) } + + override def visit(variantType: VariantType): SparkDataType = { + throw new UnsupportedOperationException("Variant type is not supported in Spark 3.x.") + } }